You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by bo...@apache.org on 2017/09/13 07:37:29 UTC

[1/5] incubator-airflow git commit: [AIRFLOW-1582] Improve logging within Airflow

Repository: incubator-airflow
Updated Branches:
  refs/heads/master 5de632e07 -> a7a518902


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/plugins_manager.py
----------------------------------------------------------------------
diff --git a/airflow/plugins_manager.py b/airflow/plugins_manager.py
index 83aae23..7c1d246 100644
--- a/airflow/plugins_manager.py
+++ b/airflow/plugins_manager.py
@@ -20,13 +20,14 @@ from __future__ import unicode_literals
 from builtins import object
 import imp
 import inspect
-import logging
 import os
 import re
 import sys
 
 from airflow import configuration
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
+log = LoggingMixin().logger
 
 class AirflowPluginException(Exception):
     pass
@@ -72,7 +73,7 @@ for root, dirs, files in os.walk(plugins_folder, followlinks=True):
             if file_ext != '.py':
                 continue
 
-            logging.debug('Importing plugin module ' + filepath)
+            log.debug('Importing plugin module %s', filepath)
             # normalize root path as namespace
             namespace = '_'.join([re.sub(norm_pattern, '__', root), mod_name])
 
@@ -87,12 +88,12 @@ for root, dirs, files in os.walk(plugins_folder, followlinks=True):
                         plugins.append(obj)
 
         except Exception as e:
-            logging.exception(e)
-            logging.error('Failed to import plugin ' + filepath)
+            log.exception(e)
+            log.error('Failed to import plugin %s', filepath)
 
 
 def make_module(name, objects):
-    logging.debug('Creating module ' + name)
+    log.debug('Creating module %s', name)
     name = name.lower()
     module = imp.new_module(name)
     module._name = name.split('.')[-1]

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/security/kerberos.py
----------------------------------------------------------------------
diff --git a/airflow/security/kerberos.py b/airflow/security/kerberos.py
index bac5c46..a9687b3 100644
--- a/airflow/security/kerberos.py
+++ b/airflow/security/kerberos.py
@@ -15,18 +15,17 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
+import socket
 import subprocess
 import sys
 import time
-import socket
 
-from airflow import configuration
-
-LOG = logging.getLogger(__name__)
+from airflow import configuration, LoggingMixin
 
 NEED_KRB181_WORKAROUND = None
 
+log = LoggingMixin().logger
+
 
 def renew_from_kt():
     # The config is specified in seconds. But we ask for that same amount in
@@ -37,10 +36,10 @@ def renew_from_kt():
     cmdv = [configuration.get('kerberos', 'kinit_path'),
             "-r", renewal_lifetime,
             "-k",  # host ticket
-            "-t", configuration.get('kerberos', 'keytab'),   # specify keytab
-            "-c", configuration.get('kerberos', 'ccache'),   # specify credentials cache
+            "-t", configuration.get('kerberos', 'keytab'),  # specify keytab
+            "-c", configuration.get('kerberos', 'ccache'),  # specify credentials cache
             principal]
-    LOG.info("Reinitting kerberos from keytab: " + " ".join(cmdv))
+    log.info("Reinitting kerberos from keytab: " + " ".join(cmdv))
 
     subp = subprocess.Popen(cmdv,
                             stdout=subprocess.PIPE,
@@ -50,7 +49,7 @@ def renew_from_kt():
                             universal_newlines=True)
     subp.wait()
     if subp.returncode != 0:
-        LOG.error("Couldn't reinit from keytab! `kinit' exited with %s.\n%s\n%s" % (
+        log.error("Couldn't reinit from keytab! `kinit' exited with %s.\n%s\n%s" % (
             subp.returncode,
             "\n".join(subp.stdout.readlines()),
             "\n".join(subp.stderr.readlines())))
@@ -71,7 +70,7 @@ def perform_krb181_workaround():
             "-c", configuration.get('kerberos', 'ccache'),
             "-R"]  # Renew ticket_cache
 
-    LOG.info("Renewing kerberos ticket to work around kerberos 1.8.1: " +
+    log.info("Renewing kerberos ticket to work around kerberos 1.8.1: " +
              " ".join(cmdv))
 
     ret = subprocess.call(cmdv)
@@ -80,7 +79,7 @@ def perform_krb181_workaround():
         principal = "%s/%s" % (configuration.get('kerberos', 'principal'), socket.getfqdn())
         fmt_dict = dict(princ=principal,
                         ccache=configuration.get('kerberos', 'principal'))
-        LOG.error("Couldn't renew kerberos ticket in order to work around "
+        log.error("Couldn't renew kerberos ticket in order to work around "
                   "Kerberos 1.8.1 issue. Please check that the ticket for "
                   "'%(princ)s' is still renewable:\n"
                   "  $ kinit -f -c %(ccache)s\n"
@@ -105,8 +104,8 @@ def detect_conf_var():
 
 
 def run():
-    if configuration.get('kerberos','keytab') is None:
-        LOG.debug("Keytab renewer not starting, no keytab configured")
+    if configuration.get('kerberos', 'keytab') is None:
+        log.debug("Keytab renewer not starting, no keytab configured")
         sys.exit(0)
 
     while True:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/settings.py
----------------------------------------------------------------------
diff --git a/airflow/settings.py b/airflow/settings.py
index 9567020..cf1eca4 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -27,7 +27,9 @@ from sqlalchemy.orm import scoped_session, sessionmaker
 from sqlalchemy.pool import NullPool
 
 from airflow import configuration as conf
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
+log = LoggingMixin().logger
 
 class DummyStatsLogger(object):
 
@@ -130,8 +132,9 @@ def configure_logging(log_format=LOG_FORMAT):
     try:
         _configure_logging(logging_level)
     except ValueError:
-        logging.warning("Logging level {} is not defined. "
-                        "Use default.".format(logging_level))
+        logging.warning(
+            "Logging level %s is not defined. Use default.", logging_level
+        )
         _configure_logging(logging.INFO)
 
 
@@ -162,7 +165,7 @@ def configure_orm(disable_connection_pool=False):
 
 try:
     from airflow_local_settings import *
-    logging.info("Loaded airflow_local_settings.")
+    log.info("Loaded airflow_local_settings.")
 except:
     pass
 
@@ -174,11 +177,13 @@ configure_orm()
 logging_config_path = conf.get('core', 'logging_config_path')
 try:
     from logging_config_path import LOGGING_CONFIG
-    logging.debug("Successfully imported user-defined logging config.")
+    log.debug("Successfully imported user-defined logging config.")
 except Exception as e:
     # Import default logging configurations.
-    logging.debug("Unable to load custom logging config file: {}."
-                  " Using default airflow logging config instead".format(str(e)))
+    log.debug(
+        "Unable to load custom logging config file: %s. Using default airflow logging config instead",
+        e
+    )
     from airflow.config_templates.default_airflow_logging import \
         DEFAULT_LOGGING_CONFIG as LOGGING_CONFIG
 logging.config.dictConfig(LOGGING_CONFIG)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/task_runner/base_task_runner.py
----------------------------------------------------------------------
diff --git a/airflow/task_runner/base_task_runner.py b/airflow/task_runner/base_task_runner.py
index 8ca8f1a..7794f4a 100644
--- a/airflow/task_runner/base_task_runner.py
+++ b/airflow/task_runner/base_task_runner.py
@@ -19,8 +19,9 @@ import json
 import subprocess
 import threading
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 from airflow import configuration as conf
-from airflow.utils.logging import LoggingMixin
 from tempfile import mkstemp
 
 
@@ -53,7 +54,7 @@ class BaseTaskRunner(LoggingMixin):
         # Add sudo commands to change user if we need to. Needed to handle SubDagOperator
         # case using a SequentialExecutor.
         if self.run_as_user and (self.run_as_user != getpass.getuser()):
-            self.logger.debug("Planning to run as the {} user".format(self.run_as_user))
+            self.logger.debug("Planning to run as the %s user", self.run_as_user)
             cfg_dict = conf.as_dict(display_sensitive=True)
             cfg_subset = {
                 'core': cfg_dict.get('core', {}),
@@ -94,7 +95,7 @@ class BaseTaskRunner(LoggingMixin):
                 line = line.decode('utf-8')
             if len(line) == 0:
                 break
-            self.logger.info(u'Subtask: {}'.format(line.rstrip('\n')))
+            self.logger.info('Subtask: %s', line.rstrip('\n'))
 
     def run_command(self, run_with, join_args=False):
         """
@@ -111,7 +112,7 @@ class BaseTaskRunner(LoggingMixin):
         """
         cmd = [" ".join(self._command)] if join_args else self._command
         full_cmd = run_with + cmd
-        self.logger.info('Running: {}'.format(full_cmd))
+        self.logger.info('Running: %s', full_cmd)
         proc = subprocess.Popen(
             full_cmd,
             stdout=subprocess.PIPE,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/dag_processing.py
----------------------------------------------------------------------
diff --git a/airflow/utils/dag_processing.py b/airflow/utils/dag_processing.py
index 2e975c1..6497fcc 100644
--- a/airflow/utils/dag_processing.py
+++ b/airflow/utils/dag_processing.py
@@ -17,19 +17,17 @@ from __future__ import division
 from __future__ import print_function
 from __future__ import unicode_literals
 
-import logging
 import os
 import re
 import time
 import zipfile
-
 from abc import ABCMeta, abstractmethod
 from collections import defaultdict
 from datetime import datetime
 
-from airflow.exceptions import AirflowException
 from airflow.dag.base_dag import BaseDag, BaseDagBag
-from airflow.utils.logging import LoggingMixin
+from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
 class SimpleDag(BaseDag):
@@ -207,7 +205,8 @@ def list_py_file_paths(directory, safe_mode=True):
 
                     file_paths.append(file_path)
                 except Exception:
-                    logging.exception("Error while examining %s", f)
+                    log = LoggingMixin().logger
+                    log.exception("Error while examining %s", f)
     return file_paths
 
 
@@ -444,7 +443,7 @@ class DagFileProcessorManager(LoggingMixin):
             if file_path in new_file_paths:
                 filtered_processors[file_path] = processor
             else:
-                self.logger.warning("Stopping processor for {}".format(file_path))
+                self.logger.warning("Stopping processor for %s", file_path)
                 processor.stop()
         self._processors = filtered_processors
 
@@ -512,17 +511,18 @@ class DagFileProcessorManager(LoggingMixin):
         log_directory = self._get_log_directory()
         latest_log_directory_path = os.path.join(
             self._child_process_log_directory, "latest")
-        if (os.path.isdir(log_directory)):
+        if os.path.isdir(log_directory):
             # if symlink exists but is stale, update it
-            if (os.path.islink(latest_log_directory_path)):
-                if(os.readlink(latest_log_directory_path) != log_directory):
+            if os.path.islink(latest_log_directory_path):
+                if os.readlink(latest_log_directory_path) != log_directory:
                     os.unlink(latest_log_directory_path)
                     os.symlink(log_directory, latest_log_directory_path)
             elif (os.path.isdir(latest_log_directory_path) or
                     os.path.isfile(latest_log_directory_path)):
-                self.logger.warning("{} already exists as a dir/file. "
-                                    "Skip creating symlink."
-                                    .format(latest_log_directory_path))
+                self.logger.warning(
+                    "%s already exists as a dir/file. Skip creating symlink.",
+                    latest_log_directory_path
+                )
             else:
                 os.symlink(log_directory, latest_log_directory_path)
 
@@ -558,7 +558,7 @@ class DagFileProcessorManager(LoggingMixin):
 
         for file_path, processor in self._processors.items():
             if processor.done:
-                self.logger.info("Processor for {} finished".format(file_path))
+                self.logger.info("Processor for %s finished", file_path)
                 now = datetime.now()
                 finished_processors[file_path] = processor
                 self._last_runtime[file_path] = (now -
@@ -573,11 +573,10 @@ class DagFileProcessorManager(LoggingMixin):
         simple_dags = []
         for file_path, processor in finished_processors.items():
             if processor.result is None:
-                self.logger.warning("Processor for {} exited with return code "
-                                    "{}. See {} for details."
-                                    .format(processor.file_path,
-                                            processor.exit_code,
-                                            processor.log_file))
+                self.logger.warning(
+                    "Processor for %s exited with return code %s. See %s for details.",
+                    processor.file_path, processor.exit_code, processor.log_file
+                )
             else:
                 for simple_dag in processor.result:
                     simple_dags.append(simple_dag)
@@ -607,12 +606,15 @@ class DagFileProcessorManager(LoggingMixin):
                                         set(files_paths_at_run_limit))
 
             for file_path, processor in self._processors.items():
-                self.logger.debug("File path {} is still being processed (started: {})"
-                                  .format(processor.file_path,
-                                          processor.start_time.isoformat()))
+                self.logger.debug(
+                    "File path %s is still being processed (started: %s)",
+                    processor.file_path, processor.start_time.isoformat()
+                )
 
-            self.logger.debug("Queuing the following files for processing:\n\t{}"
-                              .format("\n\t".join(files_paths_to_queue)))
+            self.logger.debug(
+                "Queuing the following files for processing:\n\t%s",
+                "\n\t".join(files_paths_to_queue)
+            )
 
             self._file_path_queue.extend(files_paths_to_queue)
 
@@ -624,9 +626,10 @@ class DagFileProcessorManager(LoggingMixin):
             processor = self._processor_factory(file_path, log_file_path)
 
             processor.start()
-            self.logger.info("Started a process (PID: {}) to generate "
-                             "tasks for {} - logging into {}"
-                             .format(processor.pid, file_path, log_file_path))
+            self.logger.info(
+                "Started a process (PID: %s) to generate tasks for %s - logging into %s",
+                processor.pid, file_path, log_file_path
+            )
 
             self._processors[file_path] = processor
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/db.py
----------------------------------------------------------------------
diff --git a/airflow/utils/db.py b/airflow/utils/db.py
index 35c187c..b3c8a4d 100644
--- a/airflow/utils/db.py
+++ b/airflow/utils/db.py
@@ -19,13 +19,16 @@ from __future__ import unicode_literals
 
 from datetime import datetime
 from functools import wraps
-import logging
+
 import os
 
 from sqlalchemy import event, exc
 from sqlalchemy.pool import Pool
 
 from airflow import settings
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
+log = LoggingMixin().logger
 
 def provide_session(func):
     """
@@ -308,7 +311,8 @@ def upgradedb():
     from alembic import command
     from alembic.config import Config
 
-    logging.info("Creating tables")
+    log.info("Creating tables")
+
     current_dir = os.path.dirname(os.path.abspath(__file__))
     package_dir = os.path.normpath(os.path.join(current_dir, '..'))
     directory = os.path.join(package_dir, 'migrations')
@@ -326,7 +330,8 @@ def resetdb():
     # alembic adds significant import time, so we import it lazily
     from alembic.migration import MigrationContext
 
-    logging.info("Dropping tables that exist")
+    log.info("Dropping tables that exist")
+
     models.Base.metadata.drop_all(settings.engine)
     mc = MigrationContext.configure(settings.engine)
     if mc._version.exists(settings.engine):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/email.py
----------------------------------------------------------------------
diff --git a/airflow/utils/email.py b/airflow/utils/email.py
index 57219c3..f252d55 100644
--- a/airflow/utils/email.py
+++ b/airflow/utils/email.py
@@ -21,7 +21,6 @@ from builtins import str
 from past.builtins import basestring
 
 import importlib
-import logging
 import os
 import smtplib
 
@@ -32,6 +31,7 @@ from email.utils import formatdate
 
 from airflow import configuration
 from airflow.exceptions import AirflowConfigException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
 def send_email(to, subject, html_content, files=None, dryrun=False, cc=None, bcc=None, mime_subtype='mixed'):
@@ -88,6 +88,8 @@ def send_email_smtp(to, subject, html_content, files=None, dryrun=False, cc=None
 
 
 def send_MIME_email(e_from, e_to, mime_msg, dryrun=False):
+    log = LoggingMixin().logger
+
     SMTP_HOST = configuration.get('smtp', 'SMTP_HOST')
     SMTP_PORT = configuration.getint('smtp', 'SMTP_PORT')
     SMTP_STARTTLS = configuration.getboolean('smtp', 'SMTP_STARTTLS')
@@ -99,7 +101,7 @@ def send_MIME_email(e_from, e_to, mime_msg, dryrun=False):
         SMTP_USER = configuration.get('smtp', 'SMTP_USER')
         SMTP_PASSWORD = configuration.get('smtp', 'SMTP_PASSWORD')
     except AirflowConfigException:
-        logging.debug("No user/password found for SMTP, so logging in with no authentication.")
+        log.debug("No user/password found for SMTP, so logging in with no authentication.")
 
     if not dryrun:
         s = smtplib.SMTP_SSL(SMTP_HOST, SMTP_PORT) if SMTP_SSL else smtplib.SMTP(SMTP_HOST, SMTP_PORT)
@@ -107,7 +109,7 @@ def send_MIME_email(e_from, e_to, mime_msg, dryrun=False):
             s.starttls()
         if SMTP_USER and SMTP_PASSWORD:
             s.login(SMTP_USER, SMTP_PASSWORD)
-        logging.info("Sent an alert email to " + str(e_to))
+        log.info("Sent an alert email to %s", e_to)
         s.sendmail(e_from, e_to, mime_msg.as_string())
         s.quit()
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/log/LoggingMixin.py
----------------------------------------------------------------------
diff --git a/airflow/utils/log/LoggingMixin.py b/airflow/utils/log/LoggingMixin.py
new file mode 100644
index 0000000..4572d63
--- /dev/null
+++ b/airflow/utils/log/LoggingMixin.py
@@ -0,0 +1,45 @@
+# -*- coding: utf-8 -*-
+#
+# 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.
+#
+from __future__ import absolute_import
+from __future__ import division
+from __future__ import print_function
+from __future__ import unicode_literals
+
+import logging
+from builtins import object
+
+
+class LoggingMixin(object):
+    """
+    Convenience super-class to have a logger configured with the class name
+    """
+
+    @property
+    def logger(self):
+        try:
+            return self._logger
+        except AttributeError:
+            self._logger = logging.root.getChild(self.__class__.__module__ + '.' + self.__class__.__name__)
+            return self._logger
+
+    def set_logger_contexts(self, task_instance):
+        """
+        Set the context for all handlers of current logger.
+        """
+        for handler in self.logger.handlers:
+            try:
+                handler.set_context(task_instance)
+            except AttributeError:
+                pass

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/log/file_task_handler.py
----------------------------------------------------------------------
diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py
index 7392aae..b31c968 100644
--- a/airflow/utils/log/file_task_handler.py
+++ b/airflow/utils/log/file_task_handler.py
@@ -14,6 +14,7 @@
 
 import logging
 import os
+import requests
 
 from jinja2 import Template
 
@@ -65,16 +66,16 @@ class FileTaskHandler(logging.Handler):
     def close(self):
         if self.handler is not None:
             self.handler.close()
-            
+
     def _render_filename(self, ti, try_number):
         if self.filename_jinja_template:
             jinja_context = ti.get_template_context()
             jinja_context['try_number'] = try_number
-            return self.filename_jinja_template.render(**jinja_context) 
-            
-        return self.filename_template.format(dag_id=ti.dag_id, 
+            return self.filename_jinja_template.render(**jinja_context)
+
+        return self.filename_template.format(dag_id=ti.dag_id,
                                              task_id=ti.task_id,
-                                             execution_date=ti.execution_date.isoformat(), 
+                                             execution_date=ti.execution_date.isoformat(),
                                              try_number=try_number)
 
     def _read(self, ti, try_number):
@@ -89,32 +90,37 @@ class FileTaskHandler(logging.Handler):
         # initializing the handler. Thus explicitly getting log location
         # is needed to get correct log path.
         log_relative_path = self._render_filename(ti, try_number + 1)
-        loc = os.path.join(self.local_base, log_relative_path)
+        location = os.path.join(self.local_base, log_relative_path)
+
         log = ""
 
-        if os.path.exists(loc):
+        if os.path.exists(location):
             try:
-                with open(loc) as f:
+                with open(location) as f:
                     log += "*** Reading local log.\n" + "".join(f.readlines())
             except Exception as e:
-                log = "*** Failed to load local log file: {}. {}\n".format(loc, str(e))
+                log = "*** Failed to load local log file: {}. {}\n".format(location, str(e))
         else:
-            url = os.path.join("http://{ti.hostname}:{worker_log_server_port}/log",
-                               log_relative_path).format(
+            url = os.path.join(
+                "http://{ti.hostname}:{worker_log_server_port}/log", log_relative_path
+            ).format(
                 ti=ti,
-                worker_log_server_port=conf.get('celery', 'WORKER_LOG_SERVER_PORT'))
+                worker_log_server_port=conf.get('celery', 'WORKER_LOG_SERVER_PORT')
+            )
             log += "*** Log file isn't local.\n"
             log += "*** Fetching here: {url}\n".format(**locals())
             try:
-                import requests
                 timeout = None  # No timeout
                 try:
                     timeout = conf.getint('webserver', 'log_fetch_timeout_sec')
                 except (AirflowConfigException, ValueError):
                     pass
 
-                response = requests.get(url, timeout=timeout)
+                response = requests.get(url, timeout=self.timeout)
+
+                # Check if the resource was properly fetched
                 response.raise_for_status()
+
                 log += '\n' + response.text
             except Exception as e:
                 log += "*** Failed to fetch log file from worker. {}\n".format(str(e))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/log/gcs_task_handler.py
----------------------------------------------------------------------
diff --git a/airflow/utils/log/gcs_task_handler.py b/airflow/utils/log/gcs_task_handler.py
index c340f10..0bc0b5e 100644
--- a/airflow/utils/log/gcs_task_handler.py
+++ b/airflow/utils/log/gcs_task_handler.py
@@ -11,28 +11,46 @@
 # 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 os
-import warnings
 
-from airflow import configuration as conf
-from airflow.utils import logging as logging_utils
+from airflow import configuration
+from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.utils.log.file_task_handler import FileTaskHandler
 
 
-class GCSTaskHandler(FileTaskHandler):
+class GCSTaskHandler(FileTaskHandler, LoggingMixin):
     """
     GCSTaskHandler is a python log handler that handles and reads
     task instance logs. It extends airflow FileTaskHandler and
     uploads to and reads from GCS remote storage. Upon log reading
     failure, it reads from host machine's local disk.
     """
-
     def __init__(self, base_log_folder, gcs_log_folder, filename_template):
         super(GCSTaskHandler, self).__init__(base_log_folder, filename_template)
         self.remote_base = gcs_log_folder
         self.log_relative_path = ''
-        self.closed = False
+        self._hook = None
+
+    def _build_hook(self):
+        remote_conn_id = configuration.get('core', 'REMOTE_LOG_CONN_ID')
+        try:
+            from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
+            return GoogleCloudStorageHook(
+                google_cloud_storage_conn_id=remote_conn_id
+            )
+        except:
+            self.logger.error(
+                'Could not create a GoogleCloudStorageHook with connection id '
+                '"%s". Please make sure that airflow[gcp_api] is installed '
+                'and the GCS connection exists.', remote_conn_id
+            )
+
+    @property
+    def hook(self):
+        if self._hook is None:
+            self._hook = self._build_hook()
+        return self._hook
 
     def set_context(self, ti):
         super(GCSTaskHandler, self).set_context(ti)
@@ -49,7 +67,7 @@ class GCSTaskHandler(FileTaskHandler):
         # calling close method. Here we check if logger is already
         # closed to prevent uploading the log to remote storage multiple
         # times when `logging.shutdown` is called.
-        if self.closed:
+        if self._hook is None:
             return
 
         super(GCSTaskHandler, self).close()
@@ -60,9 +78,10 @@ class GCSTaskHandler(FileTaskHandler):
             # read log and remove old logs to get just the latest additions
             with open(local_loc, 'r') as logfile:
                 log = logfile.read()
-            logging_utils.GCSLog().write(log, remote_loc)
+            self.gcs_write(log, remote_loc)
 
-        self.closed = True
+        # Unset variable
+        self._hook = None
 
     def _read(self, ti, try_number):
         """
@@ -77,15 +96,95 @@ class GCSTaskHandler(FileTaskHandler):
         log_relative_path = self._render_filename(ti, try_number + 1)
         remote_loc = os.path.join(self.remote_base, log_relative_path)
 
-        gcs_log = logging_utils.GCSLog()
-        if gcs_log.log_exists(remote_loc):
+        if self.gcs_log_exists(remote_loc):
             # If GCS remote file exists, we do not fetch logs from task instance
             # local machine even if there are errors reading remote logs, as
             # remote_log will contain error message.
-            remote_log = gcs_log.read(remote_loc, return_error=True)
+            remote_log = self.gcs_read(remote_loc, return_error=True)
             log = '*** Reading remote log from {}.\n{}\n'.format(
                 remote_loc, remote_log)
         else:
             log = super(GCSTaskHandler, self)._read(ti, try_number)
 
         return log
+
+    def gcs_log_exists(self, remote_log_location):
+        """
+        Check if remote_log_location exists in remote storage
+        :param remote_log_location: log's location in remote storage
+        :return: True if location exists else False
+        """
+        try:
+            bkt, blob = self.parse_gcs_url(remote_log_location)
+            return self.hook.exists(bkt, blob)
+        except Exception:
+            pass
+        return False
+
+    def gcs_read(self, remote_log_location, return_error=False):
+        """
+        Returns the log found at the remote_log_location.
+        :param remote_log_location: the log's location in remote storage
+        :type remote_log_location: string (path)
+        :param return_error: if True, returns a string error message if an
+            error occurs. Otherwise returns '' when an error occurs.
+        :type return_error: bool
+        """
+        try:
+            bkt, blob = self.parse_gcs_url(remote_log_location)
+            return self.hook.download(bkt, blob).decode()
+        except:
+            # return error if needed
+            if return_error:
+                msg = 'Could not read logs from {}'.format(remote_log_location)
+                self.logger.error(msg)
+                return msg
+
+    def gcs_write(self, log, remote_log_location, append=True):
+        """
+        Writes the log to the remote_log_location. Fails silently if no hook
+        was created.
+        :param log: the log to write to the remote_log_location
+        :type log: string
+        :param remote_log_location: the log's location in remote storage
+        :type remote_log_location: string (path)
+        :param append: if False, any existing log file is overwritten. If True,
+            the new log is appended to any existing logs.
+        :type append: bool
+        """
+        if append:
+            old_log = self.read(remote_log_location)
+            log = '\n'.join([old_log, log])
+
+        try:
+            bkt, blob = self.parse_gcs_url(remote_log_location)
+            from tempfile import NamedTemporaryFile
+            with NamedTemporaryFile(mode='w+') as tmpfile:
+                tmpfile.write(log)
+                # Force the file to be flushed, since we're doing the
+                # upload from within the file context (it hasn't been
+                # closed).
+                tmpfile.flush()
+                self.hook.upload(bkt, blob, tmpfile.name)
+        except:
+            self.logger.error('Could not write logs to %s', remote_log_location)
+
+    def parse_gcs_url(self, gsurl):
+        """
+        Given a Google Cloud Storage URL (gs://<bucket>/<blob>), returns a
+        tuple containing the corresponding bucket and blob.
+        """
+        # Python 3
+        try:
+            from urllib.parse import urlparse
+        # Python 2
+        except ImportError:
+            from urlparse import urlparse
+
+        parsed_url = urlparse(gsurl)
+        if not parsed_url.netloc:
+            raise AirflowException('Please provide a bucket name')
+        else:
+            bucket = parsed_url.netloc
+            blob = parsed_url.path.strip('/')
+            return bucket, blob

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/log/s3_task_handler.py
----------------------------------------------------------------------
diff --git a/airflow/utils/log/s3_task_handler.py b/airflow/utils/log/s3_task_handler.py
index 51baaac..71fc149 100644
--- a/airflow/utils/log/s3_task_handler.py
+++ b/airflow/utils/log/s3_task_handler.py
@@ -11,25 +11,42 @@
 # 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 os
 
-from airflow.utils import logging as logging_utils
+from airflow import configuration
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.utils.log.file_task_handler import FileTaskHandler
 
 
-class S3TaskHandler(FileTaskHandler):
+class S3TaskHandler(FileTaskHandler, LoggingMixin):
     """
     S3TaskHandler is a python log handler that handles and reads
     task instance logs. It extends airflow FileTaskHandler and
     uploads to and reads from S3 remote storage.
     """
-
     def __init__(self, base_log_folder, s3_log_folder, filename_template):
         super(S3TaskHandler, self).__init__(base_log_folder, filename_template)
         self.remote_base = s3_log_folder
         self.log_relative_path = ''
-        self.closed = False
+        self._hook = None
+
+    def _build_hook(self):
+        remote_conn_id = configuration.get('core', 'REMOTE_LOG_CONN_ID')
+        try:
+            from airflow.hooks.S3_hook import S3Hook
+            return S3Hook(remote_conn_id)
+        except:
+            self.logger.error(
+                'Could not create an S3Hook with connection id "%s". '
+                'Please make sure that airflow[s3] is installed and '
+                'the S3 connection exists.', remote_conn_id
+            )
+
+    @property
+    def hook(self):
+        if self._hook is None:
+            self._hook = self._build_hook()
+        return self._hook
 
     def set_context(self, ti):
         super(S3TaskHandler, self).set_context(ti)
@@ -45,7 +62,7 @@ class S3TaskHandler(FileTaskHandler):
         # calling close method. Here we check if logger is already
         # closed to prevent uploading the log to remote storage multiple
         # times when `logging.shutdown` is called.
-        if self.closed:
+        if self._hook is None:
             return
 
         super(S3TaskHandler, self).close()
@@ -56,9 +73,9 @@ class S3TaskHandler(FileTaskHandler):
             # read log and remove old logs to get just the latest additions
             with open(local_loc, 'r') as logfile:
                 log = logfile.read()
-            logging_utils.S3Log().write(log, remote_loc)
+            self.s3_write(log, remote_loc)
 
-        self.closed = True
+        self._hook = None
 
     def _read(self, ti, try_number):
         """
@@ -73,15 +90,73 @@ class S3TaskHandler(FileTaskHandler):
         log_relative_path = self._render_filename(ti, try_number + 1)
         remote_loc = os.path.join(self.remote_base, log_relative_path)
 
-        s3_log = logging_utils.S3Log()
-        if s3_log.log_exists(remote_loc):
+        if self.s3_log_exists(remote_loc):
             # If S3 remote file exists, we do not fetch logs from task instance
             # local machine even if there are errors reading remote logs, as
             # returned remote_log will contain error messages.
-            remote_log = s3_log.read(remote_loc, return_error=True)
+            remote_log = self.s3_log_read(remote_loc, return_error=True)
             log = '*** Reading remote log from {}.\n{}\n'.format(
                 remote_loc, remote_log)
         else:
             log = super(S3TaskHandler, self)._read(ti, try_number)
 
         return log
+
+    def s3_log_exists(self, remote_log_location):
+        """
+        Check if remote_log_location exists in remote storage
+        :param remote_log_location: log's location in remote storage
+        :return: True if location exists else False
+        """
+        try:
+            return self.hook.get_key(remote_log_location) is not None
+        except Exception:
+            pass
+        return False
+
+    def s3_log_read(self, remote_log_location, return_error=False):
+        """
+        Returns the log found at the remote_log_location. Returns '' if no
+        logs are found or there is an error.
+        :param remote_log_location: the log's location in remote storage
+        :type remote_log_location: string (path)
+        :param return_error: if True, returns a string error message if an
+            error occurs. Otherwise returns '' when an error occurs.
+        :type return_error: bool
+        """
+        try:
+            s3_key = self.hook.get_key(remote_log_location)
+            if s3_key:
+                return s3_key.get_contents_as_string().decode()
+        except:
+            # return error if needed
+            if return_error:
+                msg = 'Could not read logs from {}'.format(remote_log_location)
+                self.logger.error(msg)
+                return msg
+
+    def s3_write(self, log, remote_log_location, append=True):
+        """
+        Writes the log to the remote_log_location. Fails silently if no hook
+        was created.
+        :param log: the log to write to the remote_log_location
+        :type log: string
+        :param remote_log_location: the log's location in remote storage
+        :type remote_log_location: string (path)
+        :param append: if False, any existing log file is overwritten. If True,
+            the new log is appended to any existing logs.
+        :type append: bool
+        """
+        if append:
+            old_log = self.read(remote_log_location)
+            log = '\n'.join([old_log, log])
+
+        try:
+            self.hook.load_string(
+                log,
+                key=remote_log_location,
+                replace=True,
+                encrypt=configuration.getboolean('core', 'ENCRYPT_S3_LOGS'),
+            )
+        except:
+            self.logger.error('Could not write logs to %s', remote_log_location)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/logging.py
----------------------------------------------------------------------
diff --git a/airflow/utils/logging.py b/airflow/utils/logging.py
deleted file mode 100644
index c550c88..0000000
--- a/airflow/utils/logging.py
+++ /dev/null
@@ -1,252 +0,0 @@
-# -*- coding: utf-8 -*-
-#
-# 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.
-#
-from __future__ import absolute_import
-from __future__ import division
-from __future__ import print_function
-from __future__ import unicode_literals
-
-from builtins import object
-
-import dateutil.parser
-import logging
-import six
-
-from airflow import configuration
-from airflow.exceptions import AirflowException
-
-
-class LoggingMixin(object):
-    """
-    Convenience super-class to have a logger configured with the class name
-    """
-
-    @property
-    def logger(self):
-        try:
-            return self._logger
-        except AttributeError:
-            self._logger = logging.root.getChild(self.__class__.__module__ + '.' + self.__class__.__name__)
-            return self._logger
-
-    def set_logger_contexts(self, task_instance):
-        """
-        Set the context for all handlers of current logger.
-        """
-        for handler in self.logger.handlers:
-            try:
-                handler.set_context(task_instance)
-            except AttributeError:
-                pass
-
-
-class S3Log(object):
-    """
-    Utility class for reading and writing logs in S3.
-    Requires airflow[s3] and setting the REMOTE_BASE_LOG_FOLDER and
-    REMOTE_LOG_CONN_ID configuration options in airflow.cfg.
-    """
-    def __init__(self):
-        remote_conn_id = configuration.get('core', 'REMOTE_LOG_CONN_ID')
-        try:
-            from airflow.hooks.S3_hook import S3Hook
-            self.hook = S3Hook(remote_conn_id)
-        except:
-            self.hook = None
-            logging.error(
-                'Could not create an S3Hook with connection id "{}". '
-                'Please make sure that airflow[s3] is installed and '
-                'the S3 connection exists.'.format(remote_conn_id))
-
-    def log_exists(self, remote_log_location):
-        """
-        Check if remote_log_location exists in remote storage
-        :param remote_log_location: log's location in remote storage
-        :return: True if location exists else False
-        """
-        if self.hook:
-            try:
-                return self.hook.get_key(remote_log_location) is not None
-            except Exception:
-                pass
-        return False
-
-    def read(self, remote_log_location, return_error=False):
-        """
-        Returns the log found at the remote_log_location. Returns '' if no
-        logs are found or there is an error.
-
-        :param remote_log_location: the log's location in remote storage
-        :type remote_log_location: string (path)
-        :param return_error: if True, returns a string error message if an
-            error occurs. Otherwise returns '' when an error occurs.
-        :type return_error: bool
-        """
-        if self.hook:
-            try:
-                s3_key = self.hook.get_key(remote_log_location)
-                if s3_key:
-                    return s3_key.get_contents_as_string().decode()
-            except:
-                pass
-
-        # return error if needed
-        if return_error:
-            msg = 'Could not read logs from {}'.format(remote_log_location)
-            logging.error(msg)
-            return msg
-
-        return ''
-
-    def write(self, log, remote_log_location, append=True):
-        """
-        Writes the log to the remote_log_location. Fails silently if no hook
-        was created.
-
-        :param log: the log to write to the remote_log_location
-        :type log: string
-        :param remote_log_location: the log's location in remote storage
-        :type remote_log_location: string (path)
-        :param append: if False, any existing log file is overwritten. If True,
-            the new log is appended to any existing logs.
-        :type append: bool
-        """
-        if self.hook:
-            if append:
-                old_log = self.read(remote_log_location)
-                log = '\n'.join([old_log, log])
-
-            try:
-                self.hook.load_string(
-                    log,
-                    key=remote_log_location,
-                    replace=True,
-                    encrypt=configuration.getboolean('core', 'ENCRYPT_S3_LOGS'),
-                )
-            except:
-                logging.error('Could not write logs to {}'.format(remote_log_location))
-
-
-class GCSLog(object):
-    """
-    Utility class for reading and writing logs in GCS. Requires
-    airflow[gcp_api] and setting the REMOTE_BASE_LOG_FOLDER and
-    REMOTE_LOG_CONN_ID configuration options in airflow.cfg.
-    """
-    def __init__(self):
-        """
-        Attempt to create hook with airflow[gcp_api].
-        """
-        remote_conn_id = configuration.get('core', 'REMOTE_LOG_CONN_ID')
-        self.hook = None
-
-        try:
-            from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
-            self.hook = GoogleCloudStorageHook(
-                google_cloud_storage_conn_id=remote_conn_id)
-        except:
-            logging.error(
-                'Could not create a GoogleCloudStorageHook with connection id '
-                '"{}". Please make sure that airflow[gcp_api] is installed '
-                'and the GCS connection exists.'.format(remote_conn_id))
-
-    def log_exists(self, remote_log_location):
-        """
-        Check if remote_log_location exists in remote storage
-        :param remote_log_location: log's location in remote storage
-        :return: True if location exists else False
-        """
-        if self.hook:
-            try:
-                bkt, blob = self.parse_gcs_url(remote_log_location)
-                return self.hook.exists(bkt, blob)
-            except Exception:
-                pass
-        return False
-
-    def read(self, remote_log_location, return_error=False):
-        """
-        Returns the log found at the remote_log_location.
-
-        :param remote_log_location: the log's location in remote storage
-        :type remote_log_location: string (path)
-        :param return_error: if True, returns a string error message if an
-            error occurs. Otherwise returns '' when an error occurs.
-        :type return_error: bool
-        """
-        if self.hook:
-            try:
-                bkt, blob = self.parse_gcs_url(remote_log_location)
-                return self.hook.download(bkt, blob).decode()
-            except:
-                pass
-
-        # return error if needed
-        if return_error:
-            msg = 'Could not read logs from {}'.format(remote_log_location)
-            logging.error(msg)
-            return msg
-
-        return ''
-
-    def write(self, log, remote_log_location, append=True):
-        """
-        Writes the log to the remote_log_location. Fails silently if no hook
-        was created.
-
-        :param log: the log to write to the remote_log_location
-        :type log: string
-        :param remote_log_location: the log's location in remote storage
-        :type remote_log_location: string (path)
-        :param append: if False, any existing log file is overwritten. If True,
-            the new log is appended to any existing logs.
-        :type append: bool
-        """
-        if self.hook:
-            if append:
-                old_log = self.read(remote_log_location)
-                log = '\n'.join([old_log, log])
-
-            try:
-                bkt, blob = self.parse_gcs_url(remote_log_location)
-                from tempfile import NamedTemporaryFile
-                with NamedTemporaryFile(mode='w+') as tmpfile:
-                    tmpfile.write(log)
-                    # Force the file to be flushed, since we're doing the
-                    # upload from within the file context (it hasn't been
-                    # closed).
-                    tmpfile.flush()
-                    self.hook.upload(bkt, blob, tmpfile.name)
-            except:
-                logging.error('Could not write logs to {}'.format(remote_log_location))
-
-    def parse_gcs_url(self, gsurl):
-        """
-        Given a Google Cloud Storage URL (gs://<bucket>/<blob>), returns a
-        tuple containing the corresponding bucket and blob.
-        """
-        # Python 3
-        try:
-            from urllib.parse import urlparse
-        # Python 2
-        except ImportError:
-            from urlparse import urlparse
-
-        parsed_url = urlparse(gsurl)
-        if not parsed_url.netloc:
-            raise AirflowException('Please provide a bucket name')
-        else:
-            bucket = parsed_url.netloc
-            blob = parsed_url.path.strip('/')
-            return (bucket, blob)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/utils/timeout.py
----------------------------------------------------------------------
diff --git a/airflow/utils/timeout.py b/airflow/utils/timeout.py
index 62af9db..53f2149 100644
--- a/airflow/utils/timeout.py
+++ b/airflow/utils/timeout.py
@@ -17,24 +17,23 @@ from __future__ import division
 from __future__ import print_function
 from __future__ import unicode_literals
 
-import logging
 import signal
 
-from builtins import object
-
 from airflow.exceptions import AirflowTaskTimeout
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class timeout(object):
+class timeout(LoggingMixin):
     """
     To be used in a ``with`` block and timeout its content.
     """
+
     def __init__(self, seconds=1, error_message='Timeout'):
         self.seconds = seconds
         self.error_message = error_message
 
     def handle_timeout(self, signum, frame):
-        logging.error("Process timed out")
+        self.logger.error("Process timed out")
         raise AirflowTaskTimeout(self.error_message)
 
     def __enter__(self):
@@ -42,12 +41,12 @@ class timeout(object):
             signal.signal(signal.SIGALRM, self.handle_timeout)
             signal.alarm(self.seconds)
         except ValueError as e:
-            logging.warning("timeout can't be used in the current context")
-            logging.exception(e)
+            self.logger.warning("timeout can't be used in the current context")
+            self.logger.exception(e)
 
     def __exit__(self, type, value, traceback):
         try:
             signal.alarm(0)
         except ValueError as e:
-            logging.warning("timeout can't be used in the current context")
-            logging.exception(e)
+            self.logger.warning("timeout can't be used in the current context")
+            self.logger.exception(e)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/www/api/experimental/endpoints.py
----------------------------------------------------------------------
diff --git a/airflow/www/api/experimental/endpoints.py b/airflow/www/api/experimental/endpoints.py
index 3e0ff46..4e5892d 100644
--- a/airflow/www/api/experimental/endpoints.py
+++ b/airflow/www/api/experimental/endpoints.py
@@ -11,9 +11,6 @@
 # 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
-
 import airflow.api
 
 from airflow.api.common.experimental import pool as pool_api
@@ -21,6 +18,7 @@ from airflow.api.common.experimental import trigger_dag as trigger
 from airflow.api.common.experimental.get_task import get_task
 from airflow.api.common.experimental.get_task_instance import get_task_instance
 from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.www.app import csrf
 
 from flask import (
@@ -29,7 +27,7 @@ from flask import (
 )
 from datetime import datetime
 
-_log = logging.getLogger(__name__)
+_log = LoggingMixin().logger
 
 requires_authentication = airflow.api.api_auth.requires_authentication
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/www/app.py
----------------------------------------------------------------------
diff --git a/airflow/www/app.py b/airflow/www/app.py
index 1ae2731..f280713 100644
--- a/airflow/www/app.py
+++ b/airflow/www/app.py
@@ -12,7 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-import logging
 import socket
 import six
 
@@ -23,7 +22,7 @@ from flask_wtf.csrf import CSRFProtect
 csrf = CSRFProtect()
 
 import airflow
-from airflow import models
+from airflow import models, LoggingMixin
 from airflow.settings import Session
 
 from airflow.www.blueprints import routes
@@ -114,16 +113,17 @@ def create_app(config=None, testing=False):
 
         def integrate_plugins():
             """Integrate plugins to the context"""
+            log = LoggingMixin().logger
             from airflow.plugins_manager import (
                 admin_views, flask_blueprints, menu_links)
             for v in admin_views:
-                logging.debug('Adding view ' + v.name)
+                log.debug('Adding view %s', v.name)
                 admin.add_view(v)
             for bp in flask_blueprints:
-                logging.debug('Adding blueprint ' + bp.name)
+                log.debug('Adding blueprint %s', bp.name)
                 app.register_blueprint(bp)
             for ml in sorted(menu_links, key=lambda x: x.name):
-                logging.debug('Adding menu link ' + ml.name)
+                log.debug('Adding menu link %s', ml.name)
                 admin.add_link(ml)
 
         integrate_plugins()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/www/views.py
----------------------------------------------------------------------
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 655d95a..447c19f 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -68,17 +68,14 @@ from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, SCHEDULER_DEPS
 from airflow.models import BaseOperator
 from airflow.operators.subdag_operator import SubDagOperator
 
-from airflow.utils.logging import LoggingMixin
 from airflow.utils.json import json_ser
 from airflow.utils.state import State
 from airflow.utils.db import provide_session
 from airflow.utils.helpers import alchemy_to_dict
-from airflow.utils import logging as log_utils
 from airflow.utils.dates import infer_time_unit, scale_time_units
 from airflow.www import utils as wwwutils
 from airflow.www.forms import DateTimeForm, DateTimeWithNumRunsForm
 from airflow.www.validators import GreaterEqualThan
-from airflow.configuration import AirflowConfigException
 
 QUERY_LIMIT = 100000
 CHART_LIMIT = 200000
@@ -2604,7 +2601,7 @@ class UserModelView(wwwutils.SuperUserMixin, AirflowModelView):
     column_default_sort = 'username'
 
 
-class VersionView(wwwutils.SuperUserMixin, LoggingMixin, BaseView):
+class VersionView(wwwutils.SuperUserMixin, BaseView):
     @expose('/')
     def version(self):
         # Look at the version from setup.py
@@ -2612,7 +2609,7 @@ class VersionView(wwwutils.SuperUserMixin, LoggingMixin, BaseView):
             airflow_version = pkg_resources.require("apache-airflow")[0].version
         except Exception as e:
             airflow_version = None
-            self.logger.error(e)
+            logging.error(e)
 
         # Get the Git repo and git hash
         git_version = None
@@ -2620,7 +2617,7 @@ class VersionView(wwwutils.SuperUserMixin, LoggingMixin, BaseView):
             with open(os.path.join(*[settings.AIRFLOW_HOME, 'airflow', 'git_version'])) as f:
                 git_version = f.readline()
         except Exception as e:
-            self.logger.error(e)
+            logging.error(e)
 
         # Render information
         title = "Version Info"

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/setup.py
----------------------------------------------------------------------
diff --git a/setup.py b/setup.py
index b22e4e8..0ddc0f0 100644
--- a/setup.py
+++ b/setup.py
@@ -99,16 +99,6 @@ def write_version(filename=os.path.join(*['airflow',
     with open(filename, 'w') as a:
         a.write(text)
 
-
-def check_previous():
-    installed_packages = ([package.project_name for package
-                           in pip.get_installed_distributions()])
-    if 'airflow' in installed_packages:
-        print("An earlier non-apache version of Airflow was installed, "
-              "please uninstall it first. Then reinstall.")
-        sys.exit(1)
-
-
 async = [
     'greenlet>=0.4.9',
     'eventlet>= 0.9.7',
@@ -206,7 +196,6 @@ devel_all = devel + all_dbs + doc + samba + s3 + slack + crypto + oracle + docke
 
 
 def do_setup():
-    check_previous()
     write_version()
     setup(
         name='apache-airflow',

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/contrib/hooks/test_databricks_hook.py
----------------------------------------------------------------------
diff --git a/tests/contrib/hooks/test_databricks_hook.py b/tests/contrib/hooks/test_databricks_hook.py
index 56288a1..e091067 100644
--- a/tests/contrib/hooks/test_databricks_hook.py
+++ b/tests/contrib/hooks/test_databricks_hook.py
@@ -108,18 +108,17 @@ class DatabricksHookTest(unittest.TestCase):
         with self.assertRaises(AssertionError):
             DatabricksHook(retry_limit = 0)
 
-    @mock.patch('airflow.contrib.hooks.databricks_hook.logging')
     @mock.patch('airflow.contrib.hooks.databricks_hook.requests')
-    def test_do_api_call_with_error_retry(self, mock_requests, mock_logging):
+    def test_do_api_call_with_error_retry(self, mock_requests):
         for exception in [requests_exceptions.ConnectionError, requests_exceptions.Timeout]:
-            mock_requests.reset_mock()
-            mock_logging.reset_mock()
-            mock_requests.post.side_effect = exception()
+            with mock.patch.object(self.hook.logger, 'error') as mock_errors:
+                mock_requests.reset_mock()
+                mock_requests.post.side_effect = exception()
 
-            with self.assertRaises(AirflowException):
-                self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {})
+                with self.assertRaises(AirflowException):
+                    self.hook._do_api_call(SUBMIT_RUN_ENDPOINT, {})
 
-            self.assertEquals(len(mock_logging.error.mock_calls), self.hook.retry_limit)
+                self.assertEquals(len(mock_errors.mock_calls), self.hook.retry_limit)
 
     @mock.patch('airflow.contrib.hooks.databricks_hook.requests')
     def test_do_api_call_with_bad_status_code(self, mock_requests):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/contrib/operators/test_dataproc_operator.py
----------------------------------------------------------------------
diff --git a/tests/contrib/operators/test_dataproc_operator.py b/tests/contrib/operators/test_dataproc_operator.py
index 71edf58..89ad258 100644
--- a/tests/contrib/operators/test_dataproc_operator.py
+++ b/tests/contrib/operators/test_dataproc_operator.py
@@ -27,6 +27,7 @@ from copy import deepcopy
 from mock import Mock
 from mock import patch
 
+
 TASK_ID = 'test-dataproc-operator'
 CLUSTER_NAME = 'test-cluster-name'
 PROJECT_ID = 'test-project-id'
@@ -53,6 +54,7 @@ class DataprocClusterCreateOperatorTest(unittest.TestCase):
         # instantiate two different test cases with different labels.
         self.labels = [LABEL1, LABEL2]
         self.dataproc_operators = []
+        self.mock_conn = Mock()
         for labels in self.labels:
              self.dataproc_operators.append(
                 DataprocClusterCreateOperator(
@@ -120,8 +122,8 @@ class DataprocClusterCreateOperatorTest(unittest.TestCase):
             self.assertEqual(cluster_data['labels'], merged_labels)
 
     def test_cluster_name_log_no_sub(self):
-        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') \
-            as mock_hook, patch('logging.info') as l:
+        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook:
+            mock_hook.return_value.get_conn = self.mock_conn
             dataproc_task = DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name=CLUSTER_NAME,
@@ -130,14 +132,14 @@ class DataprocClusterCreateOperatorTest(unittest.TestCase):
                 zone=ZONE,
                 dag=self.dag
             )
-
-            with self.assertRaises(TypeError) as _:
-                dataproc_task.execute(None)
-            l.assert_called_with(('Creating cluster: ' + CLUSTER_NAME))
+            with patch.object(dataproc_task.logger, 'info') as mock_info:
+                with self.assertRaises(TypeError) as _:
+                    dataproc_task.execute(None)
+                mock_info.assert_called_with('Creating cluster: %s', CLUSTER_NAME)
 
     def test_cluster_name_log_sub(self):
-        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') \
-            as mock_hook, patch('logging.info') as l:
+        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook:
+            mock_hook.return_value.get_conn = self.mock_conn
             dataproc_task = DataprocClusterCreateOperator(
                 task_id=TASK_ID,
                 cluster_name='smoke-cluster-{{ ts_nodash }}',
@@ -146,14 +148,14 @@ class DataprocClusterCreateOperatorTest(unittest.TestCase):
                 zone=ZONE,
                 dag=self.dag
             )
+            with patch.object(dataproc_task.logger, 'info') as mock_info:
+                context = { 'ts_nodash' : 'testnodash'}
 
-            context = { 'ts_nodash' : 'testnodash'}
-
-            rendered = dataproc_task.render_template('cluster_name', getattr(dataproc_task,'cluster_name'), context)
-            setattr(dataproc_task, 'cluster_name', rendered)
-            with self.assertRaises(TypeError) as _:
-                dataproc_task.execute(None)
-            l.assert_called_with(('Creating cluster: smoke-cluster-testnodash'))
+                rendered = dataproc_task.render_template('cluster_name', getattr(dataproc_task,'cluster_name'), context)
+                setattr(dataproc_task, 'cluster_name', rendered)
+                with self.assertRaises(TypeError) as _:
+                    dataproc_task.execute(None)
+                mock_info.assert_called_with('Creating cluster: %s', u'smoke-cluster-testnodash')
 
 class DataprocClusterDeleteOperatorTest(unittest.TestCase):
     # Unitest for the DataprocClusterDeleteOperator
@@ -180,8 +182,7 @@ class DataprocClusterDeleteOperatorTest(unittest.TestCase):
             schedule_interval='@daily')
 
     def test_cluster_name_log_no_sub(self):
-        with patch('airflow.contrib.hooks.gcp_dataproc_hook.DataProcHook') \
-            as mock_hook, patch('logging.info') as l:
+        with patch('airflow.contrib.hooks.gcp_dataproc_hook.DataProcHook') as mock_hook:
             mock_hook.return_value.get_conn = self.mock_conn
             dataproc_task = DataprocClusterDeleteOperator(
                 task_id=TASK_ID,
@@ -189,14 +190,13 @@ class DataprocClusterDeleteOperatorTest(unittest.TestCase):
                 project_id=PROJECT_ID,
                 dag=self.dag
             )
-
-            with self.assertRaises(TypeError) as _:
-                dataproc_task.execute(None)
-            l.assert_called_with(('Deleting cluster: ' + CLUSTER_NAME))
+            with patch.object(dataproc_task.logger, 'info') as mock_info:
+                with self.assertRaises(TypeError) as _:
+                    dataproc_task.execute(None)
+                mock_info.assert_called_with('Deleting cluster: %s', CLUSTER_NAME)
 
     def test_cluster_name_log_sub(self):
-        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') \
-            as mock_hook, patch('logging.info') as l:
+        with patch('airflow.contrib.operators.dataproc_operator.DataProcHook') as mock_hook:
             mock_hook.return_value.get_conn = self.mock_conn
             dataproc_task = DataprocClusterDeleteOperator(
                 task_id=TASK_ID,
@@ -205,10 +205,11 @@ class DataprocClusterDeleteOperatorTest(unittest.TestCase):
                 dag=self.dag
             )
 
-            context = { 'ts_nodash' : 'testnodash'}
+            with patch.object(dataproc_task.logger, 'info') as mock_info:
+                context = { 'ts_nodash' : 'testnodash'}
 
-            rendered = dataproc_task.render_template('cluster_name', getattr(dataproc_task,'cluster_name'), context)
-            setattr(dataproc_task, 'cluster_name', rendered)
-            with self.assertRaises(TypeError) as _:
-                dataproc_task.execute(None)
-            l.assert_called_with(('Deleting cluster: smoke-cluster-testnodash'))
+                rendered = dataproc_task.render_template('cluster_name', getattr(dataproc_task,'cluster_name'), context)
+                setattr(dataproc_task, 'cluster_name', rendered)
+                with self.assertRaises(TypeError) as _:
+                    dataproc_task.execute(None)
+                mock_info.assert_called_with('Deleting cluster: %s', u'smoke-cluster-testnodash')

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/core.py
----------------------------------------------------------------------
diff --git a/tests/core.py b/tests/core.py
index 8eeec82..acc543b 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -29,7 +29,6 @@ from datetime import datetime, time, timedelta
 from email.mime.multipart import MIMEMultipart
 from email.mime.application import MIMEApplication
 import signal
-from time import time as timetime
 from time import sleep
 import warnings
 
@@ -37,7 +36,7 @@ from dateutil.relativedelta import relativedelta
 import sqlalchemy
 
 from airflow import configuration
-from airflow.executors import SequentialExecutor, LocalExecutor
+from airflow.executors import SequentialExecutor
 from airflow.models import Variable
 from tests.test_utils.fake_datetime import FakeDatetime
 
@@ -53,13 +52,11 @@ from airflow.operators.http_operator import SimpleHttpOperator
 from airflow.operators import sensors
 from airflow.hooks.base_hook import BaseHook
 from airflow.hooks.sqlite_hook import SqliteHook
-from airflow.hooks.postgres_hook import PostgresHook
 from airflow.bin import cli
 from airflow.www import app as application
 from airflow.settings import Session
 from airflow.utils.state import State
 from airflow.utils.dates import infer_time_unit, round_time, scale_time_units
-from airflow.utils.logging import LoggingMixin
 from lxml import html
 from airflow.exceptions import AirflowException
 from airflow.configuration import AirflowConfigException, run_command
@@ -805,17 +802,6 @@ class CoreTest(unittest.TestCase):
         # restore the envvar back to the original state
         del os.environ[key]
 
-    def test_class_with_logger_should_have_logger_with_correct_name(self):
-
-        # each class should automatically receive a logger with a correct name
-
-        class Blah(LoggingMixin):
-            pass
-
-        self.assertEqual("tests.core.Blah", Blah().logger.name)
-        self.assertEqual("airflow.executors.sequential_executor.SequentialExecutor", SequentialExecutor().logger.name)
-        self.assertEqual("airflow.executors.local_executor.LocalExecutor", LocalExecutor().logger.name)
-
     def test_round_time(self):
 
         rt1 = round_time(datetime(2015, 1, 1, 6), timedelta(days=1))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/operators/sensors.py
----------------------------------------------------------------------
diff --git a/tests/operators/sensors.py b/tests/operators/sensors.py
index 9a40a05..9b256e6 100644
--- a/tests/operators/sensors.py
+++ b/tests/operators/sensors.py
@@ -11,29 +11,32 @@
 # 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
-import os
 import sys
 import time
 import unittest
-
-from mock import patch
 from datetime import datetime, timedelta
+from mock import patch
 
-from airflow import DAG, configuration, jobs, settings
-from airflow.jobs import BackfillJob, SchedulerJob
-from airflow.models import TaskInstance, DagModel, DagBag
-from airflow.operators.sensors import HttpSensor, BaseSensorOperator, HdfsSensor, ExternalTaskSensor
-from airflow.operators.bash_operator import BashOperator
-from airflow.operators.dummy_operator import DummyOperator
-from airflow.utils.decorators import apply_defaults
+from airflow import DAG, configuration, settings
 from airflow.exceptions import (AirflowException,
                                 AirflowSensorTimeout,
                                 AirflowSkipException)
+from airflow.models import TaskInstance
+from airflow.operators.bash_operator import BashOperator
+from airflow.operators.dummy_operator import DummyOperator
+from airflow.operators.sensors import HttpSensor, BaseSensorOperator, HdfsSensor, ExternalTaskSensor
+from airflow.utils.decorators import apply_defaults
 from airflow.utils.state import State
-from tests.core import TEST_DAG_FOLDER
+
+try:
+    from unittest import mock
+except ImportError:
+    try:
+        import mock
+    except ImportError:
+        mock = None
+
 configuration.load_test_config()
 
 DEFAULT_DATE = datetime(2015, 1, 1)
@@ -72,7 +75,7 @@ class TimeoutTestSensor(BaseSensorOperator):
                 else:
                     raise AirflowSensorTimeout('Snap. Time is OUT.')
             time.sleep(self.poke_interval)
-        logging.info("Success criteria met. Exiting.")
+        self.logger.info("Success criteria met. Exiting.")
 
 
 class SensorTimeoutTest(unittest.TestCase):
@@ -158,10 +161,8 @@ class HttpSensorTests(unittest.TestCase):
         self.assertTrue(prep_request.method, received_request.method)
 
     @patch("airflow.hooks.http_hook.requests.Session.send")
-    @patch("airflow.hooks.http_hook.logging.error")
     def test_logging_head_error_request(
         self,
-        mock_error_logging,
         mock_session_send
     ):
 
@@ -183,13 +184,15 @@ class HttpSensorTests(unittest.TestCase):
             method='HEAD',
             response_check=resp_check,
             timeout=5,
-            poke_interval=1)
+            poke_interval=1
+        )
 
-        with self.assertRaises(AirflowSensorTimeout):
-            task.execute(None)
+        with mock.patch.object(task.hook.logger, 'error') as mock_errors:
+            with self.assertRaises(AirflowSensorTimeout):
+                task.execute(None)
 
-        self.assertTrue(mock_error_logging.called)
-        mock_error_logging.assert_called_with('HTTP error: Not Found')
+            self.assertTrue(mock_errors.called)
+            mock_errors.assert_called_with('HTTP error: %s', 'Not Found')
 
 
 class HdfsSensorTests(unittest.TestCase):
@@ -199,8 +202,6 @@ class HdfsSensorTests(unittest.TestCase):
             raise unittest.SkipTest('HdfsSensor won\'t work with python3. No need to test anything here')
         from tests.core import FakeHDFSHook
         self.hook = FakeHDFSHook
-        self.logger = logging.getLogger()
-        self.logger.setLevel(logging.DEBUG)
 
     def test_legacy_file_exist(self):
         """
@@ -208,7 +209,7 @@ class HdfsSensorTests(unittest.TestCase):
         :return:
         """
         # Given
-        self.logger.info("Test for existing file with the legacy behaviour")
+        logging.info("Test for existing file with the legacy behaviour")
         # When
         task = HdfsSensor(task_id='Should_be_file_legacy',
                           filepath='/datadirectory/datafile',
@@ -227,7 +228,7 @@ class HdfsSensorTests(unittest.TestCase):
         :return:
         """
         # Given
-        self.logger.info("Test for existing file with the legacy behaviour")
+        logging.info("Test for existing file with the legacy behaviour")
         # When
         task = HdfsSensor(task_id='Should_be_file_legacy',
                           filepath='/datadirectory/datafile',
@@ -248,7 +249,7 @@ class HdfsSensorTests(unittest.TestCase):
         :return:
         """
         # Given
-        self.logger.info("Test for non existing file with the legacy behaviour")
+        logging.info("Test for non existing file with the legacy behaviour")
         task = HdfsSensor(task_id='Should_not_be_file_legacy',
                           filepath='/datadirectory/not_existing_file_or_directory',
                           timeout=1,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/utils/log/test_logging.py
----------------------------------------------------------------------
diff --git a/tests/utils/log/test_logging.py b/tests/utils/log/test_logging.py
new file mode 100644
index 0000000..7e05c7d
--- /dev/null
+++ b/tests/utils/log/test_logging.py
@@ -0,0 +1,108 @@
+# -*- coding: utf-8 -*-
+#
+# 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 mock
+import unittest
+
+from airflow.utils.log.s3_task_handler import S3TaskHandler
+
+
+class TestS3TaskHandler(unittest.TestCase):
+
+    def setUp(self):
+        super(S3TaskHandler, self).setUp()
+        self.remote_log_location = 'remote/log/location'
+        self.hook_patcher = mock.patch("airflow.hooks.S3_hook.S3Hook")
+        self.hook_mock = self.hook_patcher.start()
+        self.hook_inst_mock = self.hook_mock.return_value
+        self.hook_key_mock = self.hook_inst_mock.get_key.return_value
+        self.hook_key_mock.get_contents_as_string.return_value.decode.\
+            return_value = 'content'
+
+    def tearDown(self):
+        self.hook_patcher.stop()
+        super(S3TaskHandler, self).tearDown()
+
+    def test_init(self):
+        S3TaskHandler()
+        self.hook_mock.assert_called_once_with('')
+
+    def test_init_raises(self):
+        self.hook_mock.side_effect = Exception('Failed to connect')
+        handler = S3TaskHandler()
+        with mock.patch.object(handler.logger, 'error') as mock_error:
+            # Initialize the hook
+            handler.hook()
+            mock_error.assert_called_once_with(
+                'Could not create an S3Hook with connection id "". Please make '
+                'sure that airflow[s3] is installed and the S3 connection exists.'
+            )
+
+    def test_log_exists(self):
+        self.assertTrue(S3TaskHandler().log_exists(self.remote_log_location))
+
+    def test_log_exists_none(self):
+        self.hook_inst_mock.get_key.return_value = None
+        self.assertFalse(S3TaskHandler().log_exists(self.remote_log_location))
+
+    def test_log_exists_raises(self):
+        self.hook_inst_mock.get_key.side_effect = Exception('error')
+        self.assertFalse(S3TaskHandler().log_exists(self.remote_log_location))
+
+    def test_log_exists_no_hook(self):
+        self.hook_mock.side_effect = Exception('Failed to connect')
+        self.assertFalse(S3TaskHandler().log_exists(self.remote_log_location))
+
+    def test_read(self):
+        self.assertEqual(
+            S3TaskHandler().read(self.remote_log_location),
+            'content'
+        )
+
+    def test_read_key_empty(self):
+        self.hook_inst_mock.get_key.return_value = None
+        self.assertEqual(S3TaskHandler().read(self.remote_log_location), '')
+
+    def test_read_raises(self):
+        self.hook_inst_mock.get_key.side_effect = Exception('error')
+        self.assertEqual(S3TaskHandler().read(self.remote_log_location), '')
+
+    def test_read_raises_return_error(self):
+        self.hook_inst_mock.get_key.side_effect = Exception('error')
+        handler = S3TaskHandler()
+        with mock.patch.object(handler.logger, 'error') as mock_error:
+            result = handler.s3_log_read(
+                self.remote_log_location,
+                return_error=True
+            )
+            msg = 'Could not read logs from %s' % self.remote_log_location
+            self.assertEqual(result, msg)
+            mock_error.assert_called_once_with(msg)
+
+    def test_write(self):
+        S3TaskHandler().write('text', self.remote_log_location)
+        self.hook_inst_mock.load_string.assert_called_once_with(
+            'content\ntext',
+            key=self.remote_log_location,
+            replace=True,
+            encrypt=False,
+        )
+
+    def test_write_raises(self):
+        self.hook_inst_mock.load_string.side_effect = Exception('error')
+        handler = S3TaskHandler()
+        with mock.patch.object(handler.logger, 'error') as mock_error:
+            handler.write('text', self.remote_log_location)
+            msg = 'Could not write logs to %s' % self.remote_log_location
+            mock_error.assert_called_once_with(msg)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/tests/utils/test_logging.py
----------------------------------------------------------------------
diff --git a/tests/utils/test_logging.py b/tests/utils/test_logging.py
deleted file mode 100644
index 72c5d49..0000000
--- a/tests/utils/test_logging.py
+++ /dev/null
@@ -1,103 +0,0 @@
-# -*- coding: utf-8 -*-
-#
-# 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 mock
-import unittest
-
-from airflow.utils import logging
-from datetime import datetime
-
-DEFAULT_DATE = datetime(2016, 1, 1)
-
-
-class TestS3Log(unittest.TestCase):
-
-    def setUp(self):
-        super(TestS3Log, self).setUp()
-        self.remote_log_location = 'remote/log/location'
-        self.hook_patcher = mock.patch("airflow.hooks.S3_hook.S3Hook")
-        self.hook_mock = self.hook_patcher.start()
-        self.hook_inst_mock = self.hook_mock.return_value
-        self.hook_key_mock = self.hook_inst_mock.get_key.return_value
-        self.hook_key_mock.get_contents_as_string.return_value.decode.\
-            return_value = 'content'
-        self.logging_patcher = mock.patch("airflow.utils.logging.logging")
-        self.logging_mock = self.logging_patcher.start()
-
-    def tearDown(self):
-        self.logging_patcher.stop()
-        self.hook_patcher.stop()
-        super(TestS3Log, self).tearDown()
-
-    def test_init(self):
-        logging.S3Log()
-        self.hook_mock.assert_called_once_with('')
-
-    def test_init_raises(self):
-        self.hook_mock.side_effect = Exception('Failed to connect')
-        logging.S3Log()
-        self.logging_mock.error.assert_called_once_with(
-            'Could not create an S3Hook with connection id "". Please make '
-            'sure that airflow[s3] is installed and the S3 connection exists.'
-        )
-
-    def test_log_exists(self):
-        self.assertTrue(logging.S3Log().log_exists(self.remote_log_location))
-
-    def test_log_exists_none(self):
-        self.hook_inst_mock.get_key.return_value = None
-        self.assertFalse(logging.S3Log().log_exists(self.remote_log_location))
-
-    def test_log_exists_raises(self):
-        self.hook_inst_mock.get_key.side_effect = Exception('error')
-        self.assertFalse(logging.S3Log().log_exists(self.remote_log_location))
-
-    def test_log_exists_no_hook(self):
-        self.hook_mock.side_effect = Exception('Failed to connect')
-        self.assertFalse(logging.S3Log().log_exists(self.remote_log_location))
-
-    def test_read(self):
-        self.assertEqual(logging.S3Log().read(self.remote_log_location),
-                         'content')
-
-    def test_read_key_empty(self):
-        self.hook_inst_mock.get_key.return_value = None
-        self.assertEqual(logging.S3Log().read(self.remote_log_location), '')
-
-    def test_read_raises(self):
-        self.hook_inst_mock.get_key.side_effect = Exception('error')
-        self.assertEqual(logging.S3Log().read(self.remote_log_location), '')
-
-    def test_read_raises_return_error(self):
-        self.hook_inst_mock.get_key.side_effect = Exception('error')
-        result = logging.S3Log().read(self.remote_log_location,
-                                      return_error=True)
-        msg = 'Could not read logs from %s' % self.remote_log_location
-        self.assertEqual(result, msg)
-        self.logging_mock.error.assert_called_once_with(msg)
-
-    def test_write(self):
-        logging.S3Log().write('text', self.remote_log_location)
-        self.hook_inst_mock.load_string.assert_called_once_with(
-            'content\ntext',
-            key=self.remote_log_location,
-            replace=True,
-            encrypt=False,
-        )
-
-    def test_write_raises(self):
-        self.hook_inst_mock.load_string.side_effect = Exception('error')
-        logging.S3Log().write('text', self.remote_log_location)
-        msg = 'Could not write logs to %s' % self.remote_log_location
-        self.logging_mock.error.assert_called_once_with(msg)


[3/5] incubator-airflow git commit: [AIRFLOW-1582] Improve logging within Airflow

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/local_executor.py
----------------------------------------------------------------------
diff --git a/airflow/executors/local_executor.py b/airflow/executors/local_executor.py
index a58c3d7..9730737 100644
--- a/airflow/executors/local_executor.py
+++ b/airflow/executors/local_executor.py
@@ -20,14 +20,13 @@ from builtins import range
 
 from airflow import configuration
 from airflow.executors.base_executor import BaseExecutor
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.utils.state import State
-from airflow.utils.logging import LoggingMixin
 
 PARALLELISM = configuration.get('core', 'PARALLELISM')
 
 
 class LocalWorker(multiprocessing.Process, LoggingMixin):
-
     def __init__(self, task_queue, result_queue):
         multiprocessing.Process.__init__(self)
         self.task_queue = task_queue
@@ -41,15 +40,15 @@ class LocalWorker(multiprocessing.Process, LoggingMixin):
                 # Received poison pill, no more tasks to run
                 self.task_queue.task_done()
                 break
-            self.logger.info("{} running {}".format(
-                self.__class__.__name__, command))
+            self.logger.info("%s running %s", self.__class__.__name__, command)
             command = "exec bash -c '{0}'".format(command)
             try:
                 subprocess.check_call(command, shell=True)
                 state = State.SUCCESS
             except subprocess.CalledProcessError as e:
                 state = State.FAILED
-                self.logger.error("failed to execute task {}:".format(str(e)))
+                self.logger.error("Failed to execute task %s.", str(e))
+                # TODO: Why is this commented out?
                 # raise e
             self.result_queue.put((key, state))
             self.task_queue.task_done()
@@ -68,7 +67,7 @@ class LocalExecutor(BaseExecutor):
         self.result_queue = multiprocessing.Queue()
         self.workers = [
             LocalWorker(self.queue, self.result_queue)
-            for i in range(self.parallelism)
+            for _ in range(self.parallelism)
         ]
 
         for w in self.workers:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/sequential_executor.py
----------------------------------------------------------------------
diff --git a/airflow/executors/sequential_executor.py b/airflow/executors/sequential_executor.py
index 43180cc..7d08a4b 100644
--- a/airflow/executors/sequential_executor.py
+++ b/airflow/executors/sequential_executor.py
@@ -37,14 +37,14 @@ class SequentialExecutor(BaseExecutor):
 
     def sync(self):
         for key, command in self.commands_to_run:
-            self.logger.info("Executing command: {}".format(command))
+            self.logger.info("Executing command: %s", command)
 
             try:
                 subprocess.check_call(command, shell=True)
                 self.change_state(key, State.SUCCESS)
             except subprocess.CalledProcessError as e:
                 self.change_state(key, State.FAILED)
-                self.logger.error("Failed to execute task {}:".format(str(e)))
+                self.logger.error("Failed to execute task %s.", str(e))
 
         self.commands_to_run = []
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/S3_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/S3_hook.py b/airflow/hooks/S3_hook.py
index caaa575..2f7e6ee 100644
--- a/airflow/hooks/S3_hook.py
+++ b/airflow/hooks/S3_hook.py
@@ -13,9 +13,12 @@
 # limitations under the License.
 
 from __future__ import division
+
 from future import standard_library
+
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 standard_library.install_aliases()
-import logging
 import re
 import fnmatch
 import configparser
@@ -27,8 +30,8 @@ import warnings
 import boto
 from boto.s3.connection import S3Connection, NoHostProvided
 from boto.sts import STSConnection
+
 boto.set_stream_logger('boto')
-logging.getLogger("boto").setLevel(logging.INFO)
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
@@ -84,7 +87,8 @@ def _parse_s3_config(config_file_name, config_format='boto', profile=None):
             if Config.has_option(cred_section, 'calling_format'):
                 calling_format = Config.get(cred_section, 'calling_format')
         except:
-            logging.warning("Option Error in parsing s3 config file")
+            log = LoggingMixin().logger
+            log.warning("Option Error in parsing s3 config file")
             raise
         return (access_key, secret_key, calling_format)
 
@@ -94,8 +98,8 @@ class S3Hook(BaseHook):
     Interact with S3. This class is a wrapper around the boto library.
     """
     def __init__(
-            self,
-            s3_conn_id='s3_default'):
+        self,
+        s3_conn_id='s3_default'):
         self.s3_conn_id = s3_conn_id
         self.s3_conn = self.get_connection(s3_conn_id)
         self.extra_params = self.s3_conn.extra_dejson
@@ -164,8 +168,8 @@ class S3Hook(BaseHook):
         a_key = s_key = None
         if self._creds_in_config_file:
             a_key, s_key, calling_format = _parse_s3_config(self.s3_config_file,
-                                                self.s3_config_format,
-                                                self.profile)
+                                                            self.s3_config_format,
+                                                            self.profile)
         elif self._creds_in_conn:
             a_key = self._a_key
             s_key = self._s_key
@@ -185,14 +189,14 @@ class S3Hook(BaseHook):
             assumed_role_object = sts_connection.assume_role(
                 role_arn=self.role_arn,
                 role_session_name="Airflow_" + self.s3_conn_id
-                )
+            )
             creds = assumed_role_object.credentials
             connection = S3Connection(
                 aws_access_key_id=creds.access_key,
                 aws_secret_access_key=creds.secret_key,
                 calling_format=calling_format,
                 security_token=creds.session_token
-                )
+            )
         else:
             connection = S3Connection(aws_access_key_id=a_key,
                                       aws_secret_access_key=s_key,
@@ -323,13 +327,13 @@ class S3Hook(BaseHook):
         return False if plist is None else prefix in plist
 
     def load_file(
-            self,
-            filename,
-            key,
-            bucket_name=None,
-            replace=False,
-            multipart_bytes=5 * (1024 ** 3),
-            encrypt=False):
+        self,
+        filename,
+        key,
+        bucket_name=None,
+        replace=False,
+        multipart_bytes=5 * (1024 ** 3),
+        encrypt=False):
         """
         Loads a local file to S3
 
@@ -373,10 +377,8 @@ class S3Hook(BaseHook):
                 for chunk in range(total_chunks):
                     offset = chunk * multipart_bytes
                     bytes = min(multipart_bytes, key_size - offset)
-                    with FileChunkIO(
-                            filename, 'r', offset=offset, bytes=bytes) as fp:
-                        logging.info('Sending chunk {c} of {tc}...'.format(
-                            c=chunk + 1, tc=total_chunks))
+                    with FileChunkIO(filename, 'r', offset=offset, bytes=bytes) as fp:
+                        self.logger.info('Sending chunk %s of %s...', chunk + 1, total_chunks)
                         mp.upload_part_from_file(fp, part_num=chunk + 1)
             except:
                 mp.cancel_upload()
@@ -389,8 +391,9 @@ class S3Hook(BaseHook):
             key_size = key_obj.set_contents_from_filename(filename,
                                                           replace=replace,
                                                           encrypt_key=encrypt)
-        logging.info("The key {key} now contains"
-                     " {key_size} bytes".format(**locals()))
+        self.logger.info(
+            "The key {key} now contains {key_size} bytes".format(**locals())
+        )
 
     def load_string(self, string_data,
                     key, bucket_name=None,
@@ -429,5 +432,6 @@ class S3Hook(BaseHook):
         key_size = key_obj.set_contents_from_string(string_data,
                                                     replace=replace,
                                                     encrypt_key=encrypt)
-        logging.info("The key {key} now contains"
-                     " {key_size} bytes".format(**locals()))
+        self.logger.info(
+            "The key {key} now contains {key_size} bytes".format(**locals())
+        )

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/base_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/base_hook.py b/airflow/hooks/base_hook.py
index cef8c97..4617b98 100644
--- a/airflow/hooks/base_hook.py
+++ b/airflow/hooks/base_hook.py
@@ -17,19 +17,18 @@ from __future__ import division
 from __future__ import print_function
 from __future__ import unicode_literals
 
-from builtins import object
-import logging
 import os
 import random
 
 from airflow import settings
 from airflow.models import Connection
 from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 CONN_ENV_PREFIX = 'AIRFLOW_CONN_'
 
 
-class BaseHook(object):
+class BaseHook(LoggingMixin):
     """
     Abstract base class for hooks, hooks are meant as an interface to
     interact with external systems. MySqlHook, HiveHook, PigHook return
@@ -40,6 +39,7 @@ class BaseHook(object):
     def __init__(self, source):
         pass
 
+
     @classmethod
     def _get_connections_from_db(cls, conn_id):
         session = settings.Session()
@@ -76,7 +76,8 @@ class BaseHook(object):
     def get_connection(cls, conn_id):
         conn = random.choice(cls.get_connections(conn_id))
         if conn.host:
-            logging.info("Using connection to: " + conn.host)
+            log = LoggingMixin().logger
+            log.info("Using connection to: %s", conn.host)
         return conn
 
     @classmethod

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/dbapi_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/dbapi_hook.py b/airflow/hooks/dbapi_hook.py
index fe85153..85eebd0 100644
--- a/airflow/hooks/dbapi_hook.py
+++ b/airflow/hooks/dbapi_hook.py
@@ -16,8 +16,6 @@ from builtins import str
 from past.builtins import basestring
 from datetime import datetime
 from contextlib import closing
-import numpy
-import logging
 import sys
 
 from sqlalchemy import create_engine
@@ -88,8 +86,8 @@ class DbApiHook(BaseHook):
         if sys.version_info[0] < 3:
             sql = sql.encode('utf-8')
         import pandas.io.sql as psql
-        
-        with closing(self.get_conn()) as conn:        
+
+        with closing(self.get_conn()) as conn:
             return psql.read_sql(sql, con=conn, params=parameters)
 
     def get_records(self, sql, parameters=None):
@@ -104,7 +102,7 @@ class DbApiHook(BaseHook):
         """
         if sys.version_info[0] < 3:
             sql = sql.encode('utf-8')
-            
+
         with closing(self.get_conn()) as conn:
             with closing(conn.cursor()) as cur:
                 if parameters is not None:
@@ -125,7 +123,7 @@ class DbApiHook(BaseHook):
         """
         if sys.version_info[0] < 3:
             sql = sql.encode('utf-8')
-        
+
         with closing(self.get_conn()) as conn:
             with closing(conn.cursor()) as cur:
                 if parameters is not None:
@@ -151,21 +149,21 @@ class DbApiHook(BaseHook):
         """
         if isinstance(sql, basestring):
             sql = [sql]
-        
+
         with closing(self.get_conn()) as conn:
             if self.supports_autocommit:
                 self.set_autocommit(conn, autocommit)
-            
+
             with closing(conn.cursor()) as cur:
                 for s in sql:
                     if sys.version_info[0] < 3:
                         s = s.encode('utf-8')
-                    logging.info(s)
+                    self.logger.info(s)
                     if parameters is not None:
                         cur.execute(s, parameters)
                     else:
                         cur.execute(s)
-            
+
             conn.commit()
 
     def set_autocommit(self, conn, autocommit):
@@ -197,13 +195,13 @@ class DbApiHook(BaseHook):
             target_fields = "({})".format(target_fields)
         else:
             target_fields = ''
-            
+
         with closing(self.get_conn()) as conn:
             if self.supports_autocommit:
                 self.set_autocommit(conn, False)
-            
+
             conn.commit()
-            
+
             with closing(conn.cursor()) as cur:
                 for i, row in enumerate(rows, 1):
                     l = []
@@ -218,11 +216,12 @@ class DbApiHook(BaseHook):
                     cur.execute(sql, values)
                     if commit_every and i % commit_every == 0:
                         conn.commit()
-                        logging.info(
-                            "Loaded {i} into {table} rows so far".format(**locals()))
-            
+                        self.logger.info(
+                            "Loaded {i} into {table} rows so far".format(**locals())
+                        )
+
             conn.commit()
-        logging.info(
+        self.logger.info(
             "Done loading. Loaded a total of {i} rows".format(**locals()))
 
     @staticmethod

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/druid_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/druid_hook.py b/airflow/hooks/druid_hook.py
index 350c230..af3ae9b 100644
--- a/airflow/hooks/druid_hook.py
+++ b/airflow/hooks/druid_hook.py
@@ -14,7 +14,6 @@
 
 from __future__ import print_function
 
-import logging
 import requests
 import time
 
@@ -33,7 +32,6 @@ class DruidHook(BaseHook):
     :param max_ingestion_time: The maximum ingestion time before assuming the job failed
     :type max_ingestion_time: int
     """
-
     def __init__(
             self,
             druid_ingest_conn_id='druid_ingest_default',
@@ -70,12 +68,12 @@ class DruidHook(BaseHook):
         while running:
             req_status = requests.get("{0}/{1}/status".format(url, druid_task_id))
 
-            logging.info("Job still running for {0} seconds...".format(sec))
+            self.logger.info("Job still running for %s seconds...", sec)
 
             sec = sec + 1
 
             if sec > self.max_ingestion_time:
-                raise AirflowException('Druid ingestion took more than {} seconds'.format(self.max_ingestion_time))
+                raise AirflowException('Druid ingestion took more than %s seconds', self.max_ingestion_time)
 
             time.sleep(self.timeout)
 
@@ -87,6 +85,6 @@ class DruidHook(BaseHook):
             elif status == 'FAILED':
                 raise AirflowException('Druid indexing job failed, check console for more info')
             else:
-                raise AirflowException('Could not get status of the job, got {0}'.format(status))
+                raise AirflowException('Could not get status of the job, got %s', status)
 
-        logging.info('Successful index')
+        self.logger.info('Successful index')

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/hive_hooks.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/hive_hooks.py b/airflow/hooks/hive_hooks.py
index d120769..70d7642 100644
--- a/airflow/hooks/hive_hooks.py
+++ b/airflow/hooks/hive_hooks.py
@@ -17,10 +17,8 @@ from __future__ import print_function
 from builtins import zip
 from past.builtins import basestring
 
-import collections
 import unicodecsv as csv
 import itertools
-import logging
 import re
 import subprocess
 import time
@@ -38,7 +36,6 @@ HIVE_QUEUE_PRIORITIES = ['VERY_HIGH', 'HIGH', 'NORMAL', 'LOW', 'VERY_LOW']
 
 
 class HiveCliHook(BaseHook):
-
     """Simple wrapper around the hive CLI.
 
     It also supports the ``beeline``
@@ -204,7 +201,7 @@ class HiveCliHook(BaseHook):
                 hive_cmd.extend(['-f', f.name])
 
                 if verbose:
-                    logging.info(" ".join(hive_cmd))
+                    self.logger.info(" ".join(hive_cmd))
                 sp = subprocess.Popen(
                     hive_cmd,
                     stdout=subprocess.PIPE,
@@ -218,7 +215,7 @@ class HiveCliHook(BaseHook):
                         break
                     stdout += line.decode('UTF-8')
                     if verbose:
-                        logging.info(line.decode('UTF-8').strip())
+                        self.logger.info(line.decode('UTF-8').strip())
                 sp.wait()
 
                 if sp.returncode:
@@ -249,7 +246,7 @@ class HiveCliHook(BaseHook):
             for query in query_set:
 
                 query_preview = ' '.join(query.split())[:50]
-                logging.info("Testing HQL [{0} (...)]".format(query_preview))
+                self.logger.info("Testing HQL [%s (...)]", query_preview)
                 if query_set == insert:
                     query = other + '; explain ' + query
                 else:
@@ -258,16 +255,16 @@ class HiveCliHook(BaseHook):
                     self.run_cli(query, verbose=False)
                 except AirflowException as e:
                     message = e.args[0].split('\n')[-2]
-                    logging.info(message)
+                    self.logger.info(message)
                     error_loc = re.search('(\d+):(\d+)', message)
                     if error_loc and error_loc.group(1).isdigit():
                         l = int(error_loc.group(1))
                         begin = max(l-2, 0)
                         end = min(l+3, len(query.split('\n')))
                         context = '\n'.join(query.split('\n')[begin:end])
-                        logging.info("Context :\n {0}".format(context))
+                        self.logger.info("Context :\n %s", context)
                 else:
-                    logging.info("SUCCESS")
+                    self.logger.info("SUCCESS")
 
     def load_df(
             self,
@@ -356,7 +353,7 @@ class HiveCliHook(BaseHook):
         final destination using a ``HiveOperator``.
 
         :param filepath: local filepath of the file to load
-        :type filepath: str       
+        :type filepath: str
         :param table: target Hive table, use dot notation to target a
             specific database
         :type table: str
@@ -398,9 +395,9 @@ class HiveCliHook(BaseHook):
                 tprops = ", ".join(
                     ["'{0}'='{1}'".format(k, v) for k, v in tblproperties.items()])
                 hql += "TBLPROPERTIES({tprops})\n"
-        hql += ";" 
+        hql += ";"
         hql = hql.format(**locals())
-        logging.info(hql)
+        self.logger.info(hql)
         self.run_cli(hql)
         hql = "LOAD DATA LOCAL INPATH '{filepath}' "
         if overwrite:
@@ -411,7 +408,7 @@ class HiveCliHook(BaseHook):
                 ["{0}='{1}'".format(k, v) for k, v in partition.items()])
             hql += "PARTITION ({pvals});"
         hql = hql.format(**locals())
-        logging.info(hql)
+        self.logger.info(hql)
         self.run_cli(hql)
 
     def kill(self):
@@ -665,8 +662,10 @@ class HiveServer2Hook(BaseHook):
 
         # impyla uses GSSAPI instead of KERBEROS as a auth_mechanism identifier
         if auth_mechanism == 'KERBEROS':
-            logging.warning("Detected deprecated 'KERBEROS' for authMechanism for %s. Please use 'GSSAPI' instead",
-                            self.hiveserver2_conn_id)
+            self.logger.warning(
+                "Detected deprecated 'KERBEROS' for authMechanism for %s. Please use 'GSSAPI' instead",
+                self.hiveserver2_conn_id
+            )
             auth_mechanism = 'GSSAPI'
 
         from impala.dbapi import connect
@@ -697,7 +696,7 @@ class HiveServer2Hook(BaseHook):
                     # may be `SET` or DDL
                     records = cur.fetchall()
                 except ProgrammingError:
-                    logging.debug("get_results returned no records")
+                    self.logger.debug("get_results returned no records")
                 if records:
                     results = {
                         'data': records,
@@ -717,7 +716,7 @@ class HiveServer2Hook(BaseHook):
         schema = schema or 'default'
         with self.get_conn(schema) as conn:
             with conn.cursor() as cur:
-                logging.info("Running query: " + hql)
+                self.logger.info("Running query: %s", hql)
                 cur.execute(hql)
                 schema = cur.description
                 with open(csv_filepath, 'wb') as f:
@@ -735,8 +734,8 @@ class HiveServer2Hook(BaseHook):
 
                         writer.writerows(rows)
                         i += len(rows)
-                        logging.info("Written {0} rows so far.".format(i))
-                    logging.info("Done. Loaded a total of {0} rows.".format(i))
+                        self.logger.info("Written %s rows so far.", i)
+                    self.logger.info("Done. Loaded a total of %s rows.", i)
 
     def get_records(self, hql, schema='default'):
         """

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/http_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/http_hook.py b/airflow/hooks/http_hook.py
index a144143..f168bc8 100644
--- a/airflow/hooks/http_hook.py
+++ b/airflow/hooks/http_hook.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 from builtins import str
-import logging
 
 import requests
 
@@ -83,7 +82,7 @@ class HttpHook(BaseHook):
                                    headers=headers)
 
         prepped_request = session.prepare_request(req)
-        logging.info("Sending '" + self.method + "' to url: " + url)
+        self.logger.info("Sending '%s' to url: %s", self.method, url)
         return self.run_and_check(session, prepped_request, extra_options)
 
     def run_and_check(self, session, prepped_request, extra_options):
@@ -108,12 +107,12 @@ class HttpHook(BaseHook):
             # Tried rewrapping, but not supported. This way, it's possible
             # to get reason and code for failure by checking first 3 chars
             # for the code, or do a split on ':'
-            logging.error("HTTP error: " + response.reason)
+            self.logger.error("HTTP error: %s", response.reason)
             if self.method not in ('GET', 'HEAD'):
                 # The sensor uses GET, so this prevents filling up the log
                 # with the body every time the GET 'misses'.
                 # That's ok to do, because GETs should be repeatable and
                 # all data should be visible in the log (no post data)
-                logging.error(response.text)
+                self.logger.error(response.text)
             raise AirflowException(str(response.status_code)+":"+response.reason)
         return response

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/oracle_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/oracle_hook.py b/airflow/hooks/oracle_hook.py
index 1386d6d..f439daa 100644
--- a/airflow/hooks/oracle_hook.py
+++ b/airflow/hooks/oracle_hook.py
@@ -19,7 +19,6 @@ from builtins import str
 from past.builtins import basestring
 from datetime import datetime
 import numpy
-import logging
 
 
 class OracleHook(DbApiHook):
@@ -102,11 +101,11 @@ class OracleHook(DbApiHook):
             cur.execute(sql)
             if i % commit_every == 0:
                 conn.commit()
-                logging.info('Loaded {i} into {table} rows so far'.format(**locals()))
+                self.logger.info('Loaded {i} into {table} rows so far'.format(**locals()))
         conn.commit()
         cur.close()
         conn.close()
-        logging.info('Done loading. Loaded a total of {i} rows'.format(**locals()))
+        self.logger.info('Done loading. Loaded a total of {i} rows'.format(**locals()))
 
     def bulk_insert_rows(self, table, rows, target_fields=None, commit_every=5000):
         """A performant bulk insert for cx_Oracle that uses prepared statements via `executemany()`.
@@ -130,13 +129,13 @@ class OracleHook(DbApiHook):
                 cursor.prepare(prepared_stm)
                 cursor.executemany(None, row_chunk)
                 conn.commit()
-                logging.info('[%s] inserted %s rows', table, row_count)
+                self.logger.info('[%s] inserted %s rows', table, row_count)
                 # Empty chunk
                 row_chunk = []
         # Commit the leftover chunk
         cursor.prepare(prepared_stm)
         cursor.executemany(None, row_chunk)
         conn.commit()
-        logging.info('[%s] inserted %s rows', table, row_count)
+        self.logger.info('[%s] inserted %s rows', table, row_count)
         cursor.close()
         conn.close()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/pig_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/pig_hook.py b/airflow/hooks/pig_hook.py
index 7201b9f..29beb54 100644
--- a/airflow/hooks/pig_hook.py
+++ b/airflow/hooks/pig_hook.py
@@ -13,14 +13,12 @@
 # limitations under the License.
 
 from __future__ import print_function
-import logging
 import subprocess
 from tempfile import NamedTemporaryFile
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
 from airflow.utils.file import TemporaryDirectory
-from airflow import configuration
 
 
 class PigCliHook(BaseHook):
@@ -64,7 +62,7 @@ class PigCliHook(BaseHook):
                     pig_properties_list = self.pig_properties.split()
                     pig_cmd.extend(pig_properties_list)
                 if verbose:
-                    logging.info(" ".join(pig_cmd))
+                    self.logger.info(" ".join(pig_cmd))
                 sp = subprocess.Popen(
                     pig_cmd,
                     stdout=subprocess.PIPE,
@@ -75,7 +73,7 @@ class PigCliHook(BaseHook):
                 for line in iter(sp.stdout.readline, ''):
                     stdout += line
                     if verbose:
-                        logging.info(line.strip())
+                        self.logger.info(line.strip())
                 sp.wait()
 
                 if sp.returncode:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/presto_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/presto_hook.py b/airflow/hooks/presto_hook.py
index 768ff3f..b9f30a6 100644
--- a/airflow/hooks/presto_hook.py
+++ b/airflow/hooks/presto_hook.py
@@ -13,16 +13,12 @@
 # limitations under the License.
 
 from builtins import str
-import logging
 
 from pyhive import presto
 from pyhive.exc import DatabaseError
 
 from airflow.hooks.dbapi_hook import DbApiHook
 
-logging.getLogger("pyhive").setLevel(logging.INFO)
-
-
 class PrestoException(Exception):
     pass
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/webhdfs_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/webhdfs_hook.py b/airflow/hooks/webhdfs_hook.py
index 5e2a28d..e7df328 100644
--- a/airflow/hooks/webhdfs_hook.py
+++ b/airflow/hooks/webhdfs_hook.py
@@ -14,16 +14,18 @@
 
 from airflow.hooks.base_hook import BaseHook
 from airflow import configuration
-import logging
 
 from hdfs import InsecureClient, HdfsError
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 _kerberos_security_mode = configuration.get("core", "security") == "kerberos"
 if _kerberos_security_mode:
     try:
         from hdfs.ext.kerberos import KerberosClient
     except ImportError:
-        logging.error("Could not load the Kerberos extension for the WebHDFSHook.")
+        log = LoggingMixin().logger
+        log.error("Could not load the Kerberos extension for the WebHDFSHook.")
         raise
 from airflow.exceptions import AirflowException
 
@@ -47,7 +49,7 @@ class WebHDFSHook(BaseHook):
         nn_connections = self.get_connections(self.webhdfs_conn_id)
         for nn in nn_connections:
             try:
-                logging.debug('Trying namenode {}'.format(nn.host))
+                self.logger.debug('Trying namenode %s', nn.host)
                 connection_str = 'http://{nn.host}:{nn.port}'.format(nn=nn)
                 if _kerberos_security_mode:
                     client = KerberosClient(connection_str)
@@ -55,11 +57,12 @@ class WebHDFSHook(BaseHook):
                     proxy_user = self.proxy_user or nn.login
                     client = InsecureClient(connection_str, user=proxy_user)
                 client.status('/')
-                logging.debug('Using namenode {} for hook'.format(nn.host))
+                self.logger.debug('Using namenode %s for hook', nn.host)
                 return client
             except HdfsError as e:
-                logging.debug("Read operation on namenode {nn.host} failed with"
-                              " error: {e.message}".format(**locals()))
+                self.logger.debug(
+                    "Read operation on namenode {nn.host} failed witg error: {e.message}".format(**locals())
+                )
         nn_hosts = [c.host for c in nn_connections]
         no_nn_error = "Read operations failed on the namenodes below:\n{}".format("\n".join(nn_hosts))
         raise AirflowWebHDFSHookException(no_nn_error)
@@ -98,4 +101,4 @@ class WebHDFSHook(BaseHook):
                  overwrite=overwrite,
                  n_threads=parallelism,
                  **kwargs)
-        logging.debug("Uploaded file {} to {}".format(source, destination))
+        self.logger.debug("Uploaded file %s to %s", source, destination)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/hooks/zendesk_hook.py
----------------------------------------------------------------------
diff --git a/airflow/hooks/zendesk_hook.py b/airflow/hooks/zendesk_hook.py
index 907d1e8..4634b22 100644
--- a/airflow/hooks/zendesk_hook.py
+++ b/airflow/hooks/zendesk_hook.py
@@ -12,19 +12,15 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-
-"""
-A hook to talk to Zendesk
-"""
-
-import logging
 import time
 from zdesk import Zendesk, RateLimitError, ZendeskError
 from airflow.hooks.base_hook import BaseHook
 
 
 class ZendeskHook(BaseHook):
+    """
+    A hook to talk to Zendesk
+    """
     def __init__(self, zendesk_conn_id):
         self.__zendesk_conn_id = zendesk_conn_id
         self.__url = None
@@ -41,10 +37,10 @@ class ZendeskHook(BaseHook):
         """
         retry_after = int(
             rate_limit_exception.response.headers.get('Retry-After', 60))
-        logging.info(
-            "Hit Zendesk API rate limit. Pausing for {} "
-            "seconds".format(
-                retry_after))
+        self.logger.info(
+            "Hit Zendesk API rate limit. Pausing for %s seconds",
+            retry_after
+        )
         time.sleep(retry_after)
 
     def call(self, path, query=None, get_all_pages=True):
@@ -79,7 +75,7 @@ class ZendeskHook(BaseHook):
                     # `github.zendesk...`
                     # in it, but the call function needs it removed.
                     next_url = next_page.split(self.__url)[1]
-                    logging.info("Calling {}".format(next_url))
+                    self.logger.info("Calling %s", next_url)
                     more_res = zendesk.call(next_url)
                     results.extend(more_res[key])
                     if next_page == more_res['next_page']:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/jobs.py
----------------------------------------------------------------------
diff --git a/airflow/jobs.py b/airflow/jobs.py
index 138a055..904609c 100644
--- a/airflow/jobs.py
+++ b/airflow/jobs.py
@@ -17,48 +17,43 @@ from __future__ import division
 from __future__ import print_function
 from __future__ import unicode_literals
 
-from past.builtins import basestring
-from collections import defaultdict, Counter
-
-from datetime import datetime
-
 import getpass
-import logging
-import socket
 import multiprocessing
 import os
+import psutil
 import signal
 import six
+import socket
 import sys
 import threading
 import time
-from time import sleep
-
-import psutil
+from collections import defaultdict
+from datetime import datetime
+from past.builtins import basestring
 from sqlalchemy import (
     Column, Integer, String, DateTime, func, Index, or_, and_, not_)
 from sqlalchemy.exc import OperationalError
 from sqlalchemy.orm.session import make_transient
 from tabulate import tabulate
+from time import sleep
 
-from airflow import executors, models, settings
 from airflow import configuration as conf
+from airflow import executors, models, settings
 from airflow.exceptions import AirflowException
 from airflow.models import DAG, DagRun
 from airflow.settings import Stats
 from airflow.task_runner import get_task_runner
 from airflow.ti_deps.dep_context import DepContext, QUEUE_DEPS, RUN_DEPS
-from airflow.utils.state import State
-from airflow.utils.db import provide_session, pessimistic_connection_handling
+from airflow.utils import asciiart
 from airflow.utils.dag_processing import (AbstractDagFileProcessor,
                                           DagFileProcessorManager,
                                           SimpleDag,
                                           SimpleDagBag,
                                           list_py_file_paths)
+from airflow.utils.db import provide_session, pessimistic_connection_handling
 from airflow.utils.email import send_email
-from airflow.utils.logging import LoggingMixin
-from airflow.utils import asciiart
-
+from airflow.utils.log.LoggingMixin import LoggingMixin
+from airflow.utils.state import State
 
 Base = models.Base
 ID_LEN = models.ID_LEN
@@ -276,12 +271,14 @@ class BaseJob(Base, LoggingMixin):
             ["{}".format(x) for x in reset_tis])
         session.commit()
 
-        self.logger.info("Reset the following {} TaskInstances:\n\t{}"
-                         .format(len(reset_tis), task_instance_str))
+        self.logger.info(
+            "Reset the following %s TaskInstances:\n\t%s",
+            len(reset_tis), task_instance_str
+        )
         return reset_tis
 
 
-class DagFileProcessor(AbstractDagFileProcessor):
+class DagFileProcessor(AbstractDagFileProcessor, LoggingMixin):
     """Helps call SchedulerJob.process_file() in a separate process."""
 
     # Counter that increments everytime an instance of this class is created
@@ -361,6 +358,8 @@ class DagFileProcessor(AbstractDagFileProcessor):
             # responsive file tailing
             parent_dir, _ = os.path.split(log_file)
 
+            _log = LoggingMixin().logger
+
             # Create the parent directory for the log file if necessary.
             if not os.path.isdir(parent_dir):
                 os.makedirs(parent_dir)
@@ -385,7 +384,7 @@ class DagFileProcessor(AbstractDagFileProcessor):
                 threading.current_thread().name = thread_name
                 start_time = time.time()
 
-                logging.info("Started process (PID=%s) to work on %s",
+                _log.info("Started process (PID=%s) to work on %s",
                              os.getpid(),
                              file_path)
                 scheduler_job = SchedulerJob(dag_ids=dag_id_white_list)
@@ -393,12 +392,13 @@ class DagFileProcessor(AbstractDagFileProcessor):
                                                     pickle_dags)
                 result_queue.put(result)
                 end_time = time.time()
-                logging.info("Processing %s took %.3f seconds",
-                             file_path,
-                             end_time - start_time)
+                _log.info(
+                    "Processing %s took %.3f seconds",
+                    file_path, end_time - start_time
+                )
             except:
                 # Log exceptions through the logging framework.
-                logging.exception("Got an exception! Propagating...")
+                _log.exception("Got an exception! Propagating...")
                 raise
             finally:
                 sys.stdout = original_stdout
@@ -438,7 +438,7 @@ class DagFileProcessor(AbstractDagFileProcessor):
         # Arbitrarily wait 5s for the process to die
         self._process.join(5)
         if sigkill and self._process.is_alive():
-            logging.warning("Killing PID %s", self._process.pid)
+            self.logger.warning("Killing PID %s", self._process.pid)
             os.kill(self._process.pid, signal.SIGKILL)
 
     @property
@@ -478,7 +478,7 @@ class DagFileProcessor(AbstractDagFileProcessor):
         if not self._result_queue.empty():
             self._result = self._result_queue.get_nowait()
             self._done = True
-            logging.debug("Waiting for %s", self._process)
+            self.logger.debug("Waiting for %s", self._process)
             self._process.join()
             return True
 
@@ -488,7 +488,7 @@ class DagFileProcessor(AbstractDagFileProcessor):
             # Get the object from the queue or else join() can hang.
             if not self._result_queue.empty():
                 self._result = self._result_queue.get_nowait()
-            logging.debug("Waiting for %s", self._process)
+            self.logger.debug("Waiting for %s", self._process)
             self._process.join()
             return True
 
@@ -610,8 +610,10 @@ class SchedulerJob(BaseJob):
         tasks that should have succeeded in the past hour.
         """
         if not any([ti.sla for ti in dag.tasks]):
-            self.logger.info("Skipping SLA check for {} because "
-                             "no tasks in DAG have SLAs".format(dag))
+            self.logger.info(
+                "Skipping SLA check for %s because no tasks in DAG have SLAs",
+                dag
+            )
             return
 
         TI = models.TaskInstance
@@ -841,8 +843,10 @@ class SchedulerJob(BaseJob):
                 task_start_dates = [t.start_date for t in dag.tasks]
                 if task_start_dates:
                     next_run_date = dag.normalize_schedule(min(task_start_dates))
-                    self.logger.debug("Next run date based on tasks {}"
-                                      .format(next_run_date))
+                    self.logger.debug(
+                        "Next run date based on tasks %s",
+                        next_run_date
+                    )
             else:
                 next_run_date = dag.following_schedule(last_scheduled_run)
 
@@ -859,8 +863,10 @@ class SchedulerJob(BaseJob):
                 if next_run_date == dag.start_date:
                     next_run_date = dag.normalize_schedule(dag.start_date)
 
-                self.logger.debug("Dag start date: {}. Next run date: {}"
-                                  .format(dag.start_date, next_run_date))
+                self.logger.debug(
+                    "Dag start date: %s. Next run date: %s",
+                    dag.start_date, next_run_date
+                )
 
             # don't ever schedule in the future
             if next_run_date > datetime.now():
@@ -908,11 +914,13 @@ class SchedulerJob(BaseJob):
         dag_runs = DagRun.find(dag_id=dag.dag_id, state=State.RUNNING, session=session)
         active_dag_runs = []
         for run in dag_runs:
-            self.logger.info("Examining DAG run {}".format(run))
+            self.logger.info("Examining DAG run %s", run)
             # don't consider runs that are executed in the future
             if run.execution_date > datetime.now():
-                self.logger.error("Execution date is in future: {}"
-                                  .format(run.execution_date))
+                self.logger.error(
+                    "Execution date is in future: %s",
+                    run.execution_date
+                )
                 continue
 
             if len(active_dag_runs) >= dag.max_active_runs:
@@ -933,7 +941,7 @@ class SchedulerJob(BaseJob):
                 active_dag_runs.append(run)
 
         for run in active_dag_runs:
-            self.logger.debug("Examining active DAG run {}".format(run))
+            self.logger.debug("Examining active DAG run: %s", run)
             # this needs a fresh session sometimes tis get detached
             tis = run.get_task_instances(state=(State.NONE,
                                                 State.UP_FOR_RETRY))
@@ -954,7 +962,7 @@ class SchedulerJob(BaseJob):
                 if ti.are_dependencies_met(
                         dep_context=DepContext(flag_upstream_failed=True),
                         session=session):
-                    self.logger.debug('Queuing task: {}'.format(ti))
+                    self.logger.debug('Queuing task: %s', ti)
                     queue.append(ti.key)
 
         session.close()
@@ -1012,9 +1020,10 @@ class SchedulerJob(BaseJob):
             session.commit()
 
         if tis_changed > 0:
-            self.logger.warning("Set {} task instances to state={} as their associated "
-                                "DagRun was not in RUNNING state".format(
-                tis_changed, new_state))
+            self.logger.warning(
+                "Set %s task instances to state=%s as their associated DagRun was not in RUNNING state",
+                tis_changed, new_state
+            )
 
     @provide_session
     def _find_executable_task_instances(self, simple_dag_bag, states, session=None):
@@ -1066,7 +1075,7 @@ class SchedulerJob(BaseJob):
         # Put one task instance on each line
         task_instance_str = "\n\t".join(
             ["{}".format(x) for x in task_instances_to_examine])
-        self.logger.info("Tasks up for execution:\n\t{}".format(task_instance_str))
+        self.logger.info("Tasks up for execution:\n\t%s", task_instance_str)
 
         # Get the pool settings
         pools = {p.pool: p for p in session.query(models.Pool).all()}
@@ -1087,9 +1096,12 @@ class SchedulerJob(BaseJob):
                 open_slots = pools[pool].open_slots(session=session)
 
             num_queued = len(task_instances)
-            self.logger.info("Figuring out tasks to run in Pool(name={pool}) "
-                             "with {open_slots} open slots and {num_queued} "
-                             "task instances in queue".format(**locals()))
+            self.logger.info(
+                "Figuring out tasks to run in Pool(name={pool}) with {open_slots} "
+                "open slots and {num_queued} task instances in queue".format(
+                    **locals()
+                )
+            )
 
             priority_sorted_task_instances = sorted(
                 task_instances, key=lambda ti: (-ti.priority_weight, ti.execution_date))
@@ -1099,8 +1111,10 @@ class SchedulerJob(BaseJob):
 
             for task_instance in priority_sorted_task_instances:
                 if open_slots <= 0:
-                    self.logger.info("Not scheduling since there are {} open slots in pool {}"
-                        .format(open_slots, pool))
+                    self.logger.info(
+                        "Not scheduling since there are %s open slots in pool %s",
+                        open_slots, pool
+                    )
                     # Can't schedule any more since there are no more open slots.
                     break
 
@@ -1119,25 +1133,23 @@ class SchedulerJob(BaseJob):
 
                 current_task_concurrency = dag_id_to_possibly_running_task_count[dag_id]
                 task_concurrency_limit = simple_dag_bag.get_dag(dag_id).concurrency
-                self.logger.info("DAG {} has {}/{} running and queued tasks"
-                                 .format(dag_id,
-                                         current_task_concurrency,
-                                         task_concurrency_limit))
+                self.logger.info(
+                    "DAG %s has %s/%s running and queued tasks",
+                    dag_id, current_task_concurrency, task_concurrency_limit
+                )
                 if current_task_concurrency >= task_concurrency_limit:
-                    self.logger.info("Not executing {} since the number "
-                                     "of tasks running or queued from DAG {}"
-                                     " is >= to the "
-                                     "DAG's task concurrency limit of {}"
-                                     .format(task_instance,
-                                             dag_id,
-                                             task_concurrency_limit))
+                    self.logger.info(
+                        "Not executing %s since the number of tasks running or queued from DAG %s"
+                        " is >= to the DAG's task concurrency limit of %s",
+                        task_instance, dag_id, task_concurrency_limit
+                    )
                     continue
 
-
                 if self.executor.has_task(task_instance):
-                    self.logger.debug(("Not handling task {} as the executor " +
-                                      "reports it is running")
-                                      .format(task_instance.key))
+                    self.logger.debug(
+                        "Not handling task %s as the executor reports it is running",
+                        task_instance.key
+                    )
                     continue
                 executable_tis.append(task_instance)
                 open_slots -= 1
@@ -1145,8 +1157,7 @@ class SchedulerJob(BaseJob):
 
         task_instance_str = "\n\t".join(
             ["{}".format(x) for x in executable_tis])
-        self.logger.info("Setting the follow tasks to queued state:\n\t{}"
-                         .format(task_instance_str))
+        self.logger.info("Setting the follow tasks to queued state:\n\t%s", task_instance_str)
         # so these dont expire on commit
         for ti in executable_tis:
             copy_dag_id = ti.dag_id
@@ -1225,8 +1236,7 @@ class SchedulerJob(BaseJob):
 
         task_instance_str = "\n\t".join(
             ["{}".format(x) for x in tis_to_be_queued])
-        self.logger.info("Setting the follow tasks to queued state:\n\t{}"
-                         .format(task_instance_str))
+        self.logger.info("Setting the follow tasks to queued state:\n\t%s", task_instance_str)
         return tis_to_be_queued
 
     def _enqueue_task_instances_with_queued_state(self, simple_dag_bag, task_instances):
@@ -1258,8 +1268,10 @@ class SchedulerJob(BaseJob):
 
             priority = task_instance.priority_weight
             queue = task_instance.queue
-            self.logger.info("Sending {} to executor with priority {} and queue {}"
-                             .format(task_instance.key, priority, queue))
+            self.logger.info(
+                "Sending %s to executor with priority %s and queue %s",
+                task_instance.key, priority, queue
+            )
 
             # save attributes so sqlalchemy doesnt expire them
             copy_dag_id = task_instance.dag_id
@@ -1345,20 +1357,18 @@ class SchedulerJob(BaseJob):
         for dag in dags:
             dag = dagbag.get_dag(dag.dag_id)
             if dag.is_paused:
-                self.logger.info("Not processing DAG {} since it's paused"
-                                 .format(dag.dag_id))
+                self.logger.info("Not processing DAG %s since it's paused", dag.dag_id)
                 continue
 
             if not dag:
-                self.logger.error("DAG ID {} was not found in the DagBag"
-                                  .format(dag.dag_id))
+                self.logger.error("DAG ID %s was not found in the DagBag", dag.dag_id)
                 continue
 
-            self.logger.info("Processing {}".format(dag.dag_id))
+            self.logger.info("Processing %s", dag.dag_id)
 
             dag_run = self.create_dag_run(dag)
             if dag_run:
-                self.logger.info("Created {}".format(dag_run))
+                self.logger.info("Created %s", dag_run)
             self._process_task_instances(dag, tis_out)
             self.manage_slas(dag)
 
@@ -1374,11 +1384,10 @@ class SchedulerJob(BaseJob):
         """
         for key, executor_state in list(self.executor.get_event_buffer().items()):
             dag_id, task_id, execution_date = key
-            self.logger.info("Executor reports {}.{} execution_date={} as {}"
-                             .format(dag_id,
-                                     task_id,
-                                     execution_date,
-                                     executor_state))
+            self.logger.info(
+                "Executor reports %s.%s execution_date=%s as %s",
+                dag_id, task_id, execution_date, executor_state
+            )
 
     def _log_file_processing_stats(self,
                                    known_file_paths,
@@ -1450,8 +1459,6 @@ class SchedulerJob(BaseJob):
         self.logger.info("Starting the scheduler")
         pessimistic_connection_handling()
 
-        logging.basicConfig(level=logging.DEBUG)
-
         # DAGs can be pickled for easier remote execution by some executors
         pickle_dags = False
         if self.do_pickle and self.executor.__class__ not in \
@@ -1462,22 +1469,16 @@ class SchedulerJob(BaseJob):
         # DAGs in parallel. By processing them in separate processes,
         # we can get parallelism and isolation from potentially harmful
         # user code.
-        self.logger.info("Processing files using up to {} processes at a time "
-                         .format(self.max_threads))
-        self.logger.info("Running execute loop for {} seconds"
-                         .format(self.run_duration))
-        self.logger.info("Processing each file at most {} times"
-                         .format(self.num_runs))
-        self.logger.info("Process each file at most once every {} seconds"
-                         .format(self.file_process_interval))
-        self.logger.info("Checking for new files in {} every {} seconds"
-                         .format(self.subdir, self.dag_dir_list_interval))
+        self.logger.info("Processing files using up to %s processes at a time", self.max_threads)
+        self.logger.info("Running execute loop for %s seconds", self.run_duration)
+        self.logger.info("Processing each file at most %s times", self.num_runs)
+        self.logger.info("Process each file at most once every %s seconds", self.file_process_interval)
+        self.logger.info("Checking for new files in %s every %s seconds", self.subdir, self.dag_dir_list_interval)
 
         # Build up a list of Python files that could contain DAGs
-        self.logger.info("Searching for files in {}".format(self.subdir))
+        self.logger.info("Searching for files in %s", self.subdir)
         known_file_paths = list_py_file_paths(self.subdir)
-        self.logger.info("There are {} files in {}"
-                         .format(len(known_file_paths), self.subdir))
+        self.logger.info("There are %s files in %s", len(known_file_paths), self.subdir)
 
         def processor_factory(file_path, log_file_path):
             return DagFileProcessor(file_path,
@@ -1510,23 +1511,22 @@ class SchedulerJob(BaseJob):
                 child_processes = [x for x in this_process.children(recursive=True)
                                    if x.is_running() and x.pid in pids_to_kill]
                 for child in child_processes:
-                    self.logger.info("Terminating child PID: {}".format(child.pid))
+                    self.logger.info("Terminating child PID: %s", child.pid)
                     child.terminate()
+                # TODO: Remove magic number
                 timeout = 5
-                self.logger.info("Waiting up to {}s for processes to exit..."
-                                 .format(timeout))
+                self.logger.info("Waiting up to %s seconds for processes to exit...", timeout)
                 try:
                     psutil.wait_procs(child_processes, timeout)
                 except psutil.TimeoutExpired:
-                    self.logger.debug("Ran out of time while waiting for "
-                                      "processes to exit")
+                    self.logger.debug("Ran out of time while waiting for processes to exit")
 
                 # Then SIGKILL
                 child_processes = [x for x in this_process.children(recursive=True)
                                    if x.is_running() and x.pid in pids_to_kill]
                 if len(child_processes) > 0:
                     for child in child_processes:
-                        self.logger.info("Killing child PID: {}".format(child.pid))
+                        self.logger.info("Killing child PID: %s", child.pid)
                         child.kill()
                         child.wait()
 
@@ -1568,11 +1568,10 @@ class SchedulerJob(BaseJob):
 
             if elapsed_time_since_refresh > self.dag_dir_list_interval:
                 # Build up a list of Python files that could contain DAGs
-                self.logger.info("Searching for files in {}".format(self.subdir))
+                self.logger.info("Searching for files in %s", self.subdir)
                 known_file_paths = list_py_file_paths(self.subdir)
                 last_dag_dir_refresh_time = datetime.now()
-                self.logger.info("There are {} files in {}"
-                                 .format(len(known_file_paths), self.subdir))
+                self.logger.info("There are %s files in %s", len(known_file_paths), self.subdir)
                 processor_manager.set_file_paths(known_file_paths)
 
                 self.logger.debug("Removing old import errors")
@@ -1585,8 +1584,7 @@ class SchedulerJob(BaseJob):
             if self.using_sqlite:
                 # For the sqlite case w/ 1 thread, wait until the processor
                 # is finished to avoid concurrent access to the DB.
-                self.logger.debug("Waiting for processors to finish since we're "
-                                  "using sqlite")
+                self.logger.debug("Waiting for processors to finish since we're using sqlite")
                 processor_manager.wait_until_finished()
 
             # Send tasks for execution if available
@@ -1638,16 +1636,13 @@ class SchedulerJob(BaseJob):
                 last_stat_print_time = datetime.now()
 
             loop_end_time = time.time()
-            self.logger.debug("Ran scheduling loop in {:.2f}s"
-                              .format(loop_end_time - loop_start_time))
-            self.logger.debug("Sleeping for {:.2f}s"
-                              .format(self._processor_poll_interval))
+            self.logger.debug("Ran scheduling loop in %.2f seconds", loop_end_time - loop_start_time)
+            self.logger.debug("Sleeping for %.2f seconds", self._processor_poll_interval)
             time.sleep(self._processor_poll_interval)
 
             # Exit early for a test mode
             if processor_manager.max_runs_reached():
-                self.logger.info("Exiting loop as all files have been processed "
-                                 "{} times".format(self.num_runs))
+                self.logger.info("Exiting loop as all files have been processed %s times", self.num_runs)
                 break
 
         # Stop any processors
@@ -1662,8 +1657,10 @@ class SchedulerJob(BaseJob):
                 all_files_processed = False
                 break
         if all_files_processed:
-            self.logger.info("Deactivating DAGs that haven't been touched since {}"
-                             .format(execute_start_time.isoformat()))
+            self.logger.info(
+                "Deactivating DAGs that haven't been touched since %s",
+                execute_start_time.isoformat()
+            )
             models.DAG.deactivate_stale_dags(execute_start_time)
 
         self.executor.end()
@@ -1696,23 +1693,21 @@ class SchedulerJob(BaseJob):
         :return: a list of SimpleDags made from the Dags found in the file
         :rtype: list[SimpleDag]
         """
-        self.logger.info("Processing file {} for tasks to queue".format(file_path))
+        self.logger.info("Processing file %s for tasks to queue", file_path)
         # As DAGs are parsed from this file, they will be converted into SimpleDags
         simple_dags = []
 
         try:
             dagbag = models.DagBag(file_path)
         except Exception:
-            self.logger.exception("Failed at reloading the DAG file {}".format(file_path))
+            self.logger.exception("Failed at reloading the DAG file %s", file_path)
             Stats.incr('dag_file_refresh_error', 1, 1)
             return []
 
         if len(dagbag.dags) > 0:
-            self.logger.info("DAG(s) {} retrieved from {}"
-                             .format(dagbag.dags.keys(),
-                                     file_path))
+            self.logger.info("DAG(s) %s retrieved from %s", dagbag.dags.keys(), file_path)
         else:
-            self.logger.warning("No viable dags retrieved from {}".format(file_path))
+            self.logger.warning("No viable dags retrieved from %s", file_path)
             self.update_import_errors(session, dagbag)
             return []
 
@@ -1783,7 +1778,7 @@ class SchedulerJob(BaseJob):
                 ti.state = State.SCHEDULED
 
             # Also save this task instance to the DB.
-            self.logger.info("Creating / updating {} in ORM".format(ti))
+            self.logger.info("Creating / updating %s in ORM", ti)
             session.merge(ti)
             session.commit()
 
@@ -1914,25 +1909,22 @@ class BackfillJob(BaseJob):
             ti.refresh_from_db()
             if ti.state == State.SUCCESS:
                 ti_status.succeeded.add(key)
-                self.logger.debug("Task instance {} succeeded. "
-                                  "Don't rerun.".format(ti))
+                self.logger.debug("Task instance %s succeeded. Don't rerun.", ti)
                 ti_status.started.pop(key)
                 continue
             elif ti.state == State.SKIPPED:
                 ti_status.skipped.add(key)
-                self.logger.debug("Task instance {} skipped. "
-                                  "Don't rerun.".format(ti))
+                self.logger.debug("Task instance %s skipped. Don't rerun.", ti)
                 ti_status.started.pop(key)
                 continue
             elif ti.state == State.FAILED:
-                self.logger.error("Task instance {} failed".format(ti))
+                self.logger.error("Task instance %s failed", ti)
                 ti_status.failed.add(key)
                 ti_status.started.pop(key)
                 continue
             # special case: if the task needs to run again put it back
             elif ti.state == State.UP_FOR_RETRY:
-                self.logger.warning("Task instance {} is up for retry"
-                                    .format(ti))
+                self.logger.warning("Task instance %s is up for retry", ti)
                 ti_status.started.pop(key)
                 ti_status.to_run[key] = ti
             # special case: The state of the task can be set to NONE by the task itself
@@ -1941,9 +1933,11 @@ class BackfillJob(BaseJob):
             # for that as otherwise those tasks would fall outside of the scope of
             # the backfill suddenly.
             elif ti.state == State.NONE:
-                self.logger.warning("FIXME: task instance {} state was set to "
-                                    "None externally or reaching concurrency limits. "
-                                    "Re-adding task to queue.".format(ti))
+                self.logger.warning(
+                    "FIXME: task instance %s state was set to none externally or "
+                    "reaching concurrency limits. Re-adding task to queue.",
+                    ti
+                )
                 session = settings.Session()
                 ti.set_state(State.SCHEDULED, session=session)
                 session.close()
@@ -1960,14 +1954,16 @@ class BackfillJob(BaseJob):
 
         for key, state in list(executor.get_event_buffer().items()):
             if key not in started:
-                self.logger.warning("{} state {} not in started={}"
-                                    .format(key, state, started.values()))
+                self.logger.warning(
+                    "%s state %s not in started=%s",
+                    key, state, started.values()
+                )
                 continue
 
             ti = started[key]
             ti.refresh_from_db()
 
-            self.logger.debug("Executor state: {} task {}".format(state, ti))
+            self.logger.debug("Executor state: %s task %s", state, ti)
 
             if state == State.FAILED or state == State.SUCCESS:
                 if ti.state == State.RUNNING or ti.state == State.QUEUED:
@@ -2090,9 +2086,10 @@ class BackfillJob(BaseJob):
             len(ti_status.not_ready))
         self.logger.info(msg)
 
-        self.logger.debug("Finished dag run loop iteration. "
-                          "Remaining tasks {}"
-                          .format(ti_status.to_run.values()))
+        self.logger.debug(
+            "Finished dag run loop iteration. Remaining tasks %s",
+            ti_status.to_run.values()
+        )
 
     @provide_session
     def _process_backfill_task_instances(self,
@@ -2142,43 +2139,41 @@ class BackfillJob(BaseJob):
                     ignore_depends_on_past = (
                         self.ignore_first_depends_on_past and
                         ti.execution_date == (start_date or ti.start_date))
-                    self.logger.debug("Task instance to run {} state {}"
-                                      .format(ti, ti.state))
+                    self.logger.debug("Task instance to run %s state %s", ti, ti.state)
 
                     # guard against externally modified tasks instances or
                     # in case max concurrency has been reached at task runtime
                     if ti.state == State.NONE:
-                        self.logger.warning("FIXME: task instance {} state was set to "
-                                            "None externally. This should not happen")
+                        self.logger.warning(
+                            "FIXME: task instance {} state was set to None externally. This should not happen"
+                        )
                         ti.set_state(State.SCHEDULED, session=session)
 
                     # The task was already marked successful or skipped by a
                     # different Job. Don't rerun it.
                     if ti.state == State.SUCCESS:
                         ti_status.succeeded.add(key)
-                        self.logger.debug("Task instance {} succeeded. "
-                                          "Don't rerun.".format(ti))
+                        self.logger.debug("Task instance %s succeeded. Don't rerun.", ti)
                         ti_status.to_run.pop(key)
                         if key in ti_status.started:
                             ti_status.started.pop(key)
                         continue
                     elif ti.state == State.SKIPPED:
                         ti_status.skipped.add(key)
-                        self.logger.debug("Task instance {} skipped. "
-                                          "Don't rerun.".format(ti))
+                        self.logger.debug("Task instance %s skipped. Don't rerun.", ti)
                         ti_status.to_run.pop(key)
                         if key in ti_status.started:
                             ti_status.started.pop(key)
                         continue
                     elif ti.state == State.FAILED:
-                        self.logger.error("Task instance {} failed".format(ti))
+                        self.logger.error("Task instance %s failed", ti)
                         ti_status.failed.add(key)
                         ti_status.to_run.pop(key)
                         if key in ti_status.started:
                             ti_status.started.pop(key)
                         continue
                     elif ti.state == State.UPSTREAM_FAILED:
-                        self.logger.error("Task instance {} upstream failed".format(ti))
+                        self.logger.error("Task instance %s upstream failed", ti)
                         ti_status.failed.add(key)
                         ti_status.to_run.pop(key)
                         if key in ti_status.started:
@@ -2200,10 +2195,12 @@ class BackfillJob(BaseJob):
                         ti.refresh_from_db(lock_for_update=True, session=session)
                         if ti.state == State.SCHEDULED or ti.state == State.UP_FOR_RETRY:
                             if executor.has_task(ti):
-                                self.logger.debug("Task Instance {} already in executor "
-                                                  "waiting for queue to clear".format(ti))
+                                self.logger.debug(
+                                    "Task Instance %s already in executor waiting for queue to clear",
+                                    ti
+                                )
                             else:
-                                self.logger.debug('Sending {} to executor'.format(ti))
+                                self.logger.debug('Sending %s to executor', ti)
                                 # Skip scheduled state, we are executing immediately
                                 ti.state = State.QUEUED
                                 session.merge(ti)
@@ -2220,7 +2217,7 @@ class BackfillJob(BaseJob):
                         continue
 
                     if ti.state == State.UPSTREAM_FAILED:
-                        self.logger.error("Task instance {} upstream failed".format(ti))
+                        self.logger.error("Task instance %s upstream failed", ti)
                         ti_status.failed.add(key)
                         ti_status.to_run.pop(key)
                         if key in ti_status.started:
@@ -2229,15 +2226,14 @@ class BackfillJob(BaseJob):
 
                     # special case
                     if ti.state == State.UP_FOR_RETRY:
-                        self.logger.debug("Task instance {} retry period not expired yet"
-                                          .format(ti))
+                        self.logger.debug("Task instance %s retry period not expired yet", ti)
                         if key in ti_status.started:
                             ti_status.started.pop(key)
                         ti_status.to_run[key] = ti
                         continue
 
                     # all remaining tasks
-                    self.logger.debug('Adding {} to not_ready'.format(ti))
+                    self.logger.debug('Adding %s to not_ready', ti)
                     ti_status.not_ready.add(key)
 
             # execute the tasks in the queue
@@ -2250,8 +2246,10 @@ class BackfillJob(BaseJob):
             if (ti_status.not_ready and
                     ti_status.not_ready == set(ti_status.to_run) and
                     len(ti_status.started) == 0):
-                self.logger.warning("Deadlock discovered for ti_status.to_run={}"
-                                    .format(ti_status.to_run.values()))
+                self.logger.warning(
+                    "Deadlock discovered for ti_status.to_run=%s",
+                    ti_status.to_run.values()
+                )
                 ti_status.deadlocked.update(ti_status.to_run.values())
                 ti_status.to_run.clear()
 
@@ -2284,7 +2282,7 @@ class BackfillJob(BaseJob):
         if ti_status.failed:
             err += (
                 "---------------------------------------------------\n"
-                "Some task instances failed:\n{}\n".format(ti_status.failed))
+                "Some task instances failed:\n%s\n".format(ti_status.failed))
         if ti_status.deadlocked:
             err += (
                 '---------------------------------------------------\n'
@@ -2367,8 +2365,7 @@ class BackfillJob(BaseJob):
         run_dates = self.dag.get_run_dates(start_date=start_date,
                                            end_date=self.bf_end_date)
         if len(run_dates) == 0:
-            self.logger.info("No run dates were found for the given dates and dag "
-                             "interval.")
+            self.logger.info("No run dates were found for the given dates and dag interval.")
             return
 
         # picklin'
@@ -2406,9 +2403,11 @@ class BackfillJob(BaseJob):
                     raise AirflowException(err)
 
                 if remaining_dates > 0:
-                    self.logger.info(("max_active_runs limit for dag {} has been reached "
-                                     " - waiting for other dag runs to finish")
-                                     .format(self.dag_id))
+                    self.logger.info(
+                        "max_active_runs limit for dag %s has been reached "
+                        " - waiting for other dag runs to finish",
+                        self.dag_id
+                    )
                     time.sleep(self.delay_on_limit_secs)
         finally:
             executor.end()
@@ -2454,8 +2453,8 @@ class LocalTaskJob(BaseJob):
         self.task_runner = get_task_runner(self)
 
         def signal_handler(signum, frame):
-            '''Setting kill signal handler'''
-            logging.error("Killing subprocess")
+            """Setting kill signal handler"""
+            self.logger.error("Killing subprocess")
             self.on_kill()
             raise AirflowException("LocalTaskJob received SIGTERM signal")
         signal.signal(signal.SIGTERM, signal_handler)
@@ -2468,8 +2467,8 @@ class LocalTaskJob(BaseJob):
                 ignore_ti_state=self.ignore_ti_state,
                 job_id=self.id,
                 pool=self.pool):
-            self.logger.info("Task is not able to be run") 
-            return 
+            self.logger.info("Task is not able to be run")
+            return
 
         try:
             self.task_runner.start()
@@ -2481,8 +2480,7 @@ class LocalTaskJob(BaseJob):
                 # Monitor the task to see if it's done
                 return_code = self.task_runner.return_code()
                 if return_code is not None:
-                    self.logger.info("Task exited with return code {}"
-                                     .format(return_code))
+                    self.logger.info("Task exited with return code %s", return_code)
                     return
 
                 # Periodically heartbeat so that the scheduler doesn't think this
@@ -2492,8 +2490,10 @@ class LocalTaskJob(BaseJob):
                     last_heartbeat_time = time.time()
                 except OperationalError:
                     Stats.incr('local_task_job_heartbeat_failure', 1, 1)
-                    self.logger.exception("Exception while trying to heartbeat! "
-                                          "Sleeping for {}s".format(self.heartrate))
+                    self.logger.exception(
+                        "Exception while trying to heartbeat! Sleeping for %s seconds",
+                        self.heartrate
+                    )
                     time.sleep(self.heartrate)
 
                 # If it's been too long since we've heartbeat, then it's possible that
@@ -2531,19 +2531,20 @@ class LocalTaskJob(BaseJob):
 
         if ti.state == State.RUNNING:
             if not same_hostname:
-                logging.warning("The recorded hostname {ti.hostname} "
+                self.logger.warning("The recorded hostname {ti.hostname} "
                                 "does not match this instance's hostname "
                                 "{fqdn}".format(**locals()))
                 raise AirflowException("Hostname of job runner does not match")
             elif not same_process:
                 current_pid = os.getpid()
-                logging.warning("Recorded pid {ti.pid} does not match the current pid "
+                self.logger.warning("Recorded pid {ti.pid} does not match the current pid "
                                 "{current_pid}".format(**locals()))
                 raise AirflowException("PID of job runner does not match")
         elif (self.task_runner.return_code() is None
               and hasattr(self.task_runner, 'process')):
-            logging.warning(
-                "State of this instance has been externally set to "
-                "{}. Taking the poison pill. So long.".format(ti.state))
+            self.logger.warning(
+                "State of this instance has been externally set to %s. Taking the poison pill.",
+                ti.state
+            )
             self.task_runner.terminate()
             self.terminating = True


[5/5] incubator-airflow git commit: [AIRFLOW-1582] Improve logging within Airflow

Posted by bo...@apache.org.
[AIRFLOW-1582] Improve logging within Airflow

Clean the way of logging within Airflow. Remove
the old logging.py and
move to the airflow.utils.log.* interface. Remove
setting the logging
outside of the settings/configuration code. Move
away from the string
format to logging_function(msg, *args).

Closes #2592 from Fokko/AIRFLOW-1582-Improve-
logging-structure


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

Branch: refs/heads/master
Commit: a7a518902dcf1e7fd4bf477cf57cee691f181b29
Parents: 5de632e
Author: Fokko Driesprong <fo...@godatadriven.com>
Authored: Wed Sep 13 09:36:58 2017 +0200
Committer: Bolke de Bruin <bo...@xs4all.nl>
Committed: Wed Sep 13 09:36:58 2017 +0200

----------------------------------------------------------------------
 UPDATING.md                                     |   4 +-
 airflow/__init__.py                             |  10 +-
 airflow/api/__init__.py                         |  12 +-
 airflow/api/auth/backend/kerberos_auth.py       |  14 +-
 airflow/api/common/experimental/get_task.py     |   4 -
 .../common/experimental/get_task_instance.py    |   4 -
 airflow/bin/airflow                             |   1 -
 airflow/bin/cli.py                              |  46 +--
 airflow/configuration.py                        |  25 +-
 .../auth/backends/github_enterprise_auth.py     |   5 +-
 airflow/contrib/auth/backends/google_auth.py    |  11 +-
 airflow/contrib/auth/backends/kerberos_auth.py  |   5 +-
 airflow/contrib/auth/backends/ldap_auth.py      |  32 +-
 airflow/contrib/auth/backends/password_auth.py  |  10 +-
 airflow/contrib/executors/mesos_executor.py     |  51 +--
 airflow/contrib/hooks/bigquery_hook.py          |  57 +--
 airflow/contrib/hooks/cloudant_hook.py          |  10 +-
 airflow/contrib/hooks/databricks_hook.py        |  15 +-
 airflow/contrib/hooks/datadog_hook.py           |  13 +-
 airflow/contrib/hooks/datastore_hook.py         |   3 +-
 airflow/contrib/hooks/ftp_hook.py               |  10 +-
 airflow/contrib/hooks/gcp_api_base_hook.py      |  13 +-
 airflow/contrib/hooks/gcp_dataflow_hook.py      |  29 +-
 airflow/contrib/hooks/gcp_dataproc_hook.py      |  28 +-
 airflow/contrib/hooks/gcp_mlengine_hook.py      |  48 ++-
 airflow/contrib/hooks/gcs_hook.py               |  10 +-
 airflow/contrib/hooks/jira_hook.py              |   9 +-
 airflow/contrib/hooks/qubole_hook.py            |  22 +-
 airflow/contrib/hooks/redis_hook.py             |  16 +-
 airflow/contrib/hooks/salesforce_hook.py        |  31 +-
 airflow/contrib/hooks/spark_sql_hook.py         |  10 +-
 airflow/contrib/hooks/spark_submit_hook.py      |  24 +-
 airflow/contrib/hooks/sqoop_hook.py             |  17 +-
 airflow/contrib/hooks/ssh_hook.py               |  34 +-
 airflow/contrib/operators/bigquery_operator.py  |   4 +-
 .../operators/bigquery_table_delete_operator.py |   4 +-
 .../contrib/operators/bigquery_to_bigquery.py   |   8 +-
 airflow/contrib/operators/bigquery_to_gcs.py    |   4 +-
 .../contrib/operators/databricks_operator.py    |  25 +-
 airflow/contrib/operators/dataproc_operator.py  |  30 +-
 .../operators/datastore_export_operator.py      |   5 +-
 .../operators/datastore_import_operator.py      |   6 +-
 airflow/contrib/operators/ecs_operator.py       |  24 +-
 .../contrib/operators/emr_add_steps_operator.py |   7 +-
 .../operators/emr_create_job_flow_operator.py   |  10 +-
 .../emr_terminate_job_flow_operator.py          |   7 +-
 airflow/contrib/operators/file_to_wasb.py       |  14 +-
 airflow/contrib/operators/fs_operator.py        |   4 +-
 .../contrib/operators/gcs_download_operator.py  |   6 +-
 airflow/contrib/operators/gcs_to_bq.py          |   8 +-
 airflow/contrib/operators/hipchat_operator.py   |   3 +-
 airflow/contrib/operators/mlengine_operator.py  |  30 +-
 .../operators/mlengine_prediction_summary.py    |   2 -
 airflow/contrib/operators/mysql_to_gcs.py       |   4 +-
 airflow/contrib/operators/sftp_operator.py      |   7 +-
 .../contrib/operators/spark_submit_operator.py  |   6 +-
 airflow/contrib/operators/ssh_operator.py       |   1 -
 airflow/contrib/operators/vertica_operator.py   |   5 +-
 airflow/contrib/operators/vertica_to_hive.py    |   5 +-
 airflow/contrib/sensors/bigquery_sensor.py      |   5 +-
 airflow/contrib/sensors/datadog_sensor.py       |   5 +-
 airflow/contrib/sensors/emr_base_sensor.py      |   9 +-
 airflow/contrib/sensors/emr_job_flow_sensor.py  |   6 +-
 airflow/contrib/sensors/emr_step_sensor.py      |   5 +-
 airflow/contrib/sensors/ftp_sensor.py           |   4 +-
 airflow/contrib/sensors/gcs_sensor.py           |   7 +-
 airflow/contrib/sensors/hdfs_sensors.py         |  12 +-
 airflow/contrib/sensors/jira_sensor.py          |  27 +-
 airflow/contrib/sensors/redis_key_sensor.py     |   4 -
 airflow/contrib/sensors/wasb_sensor.py          |  11 +-
 .../contrib/task_runner/cgroup_task_runner.py   |  49 +--
 airflow/executors/__init__.py                   |   8 +-
 airflow/executors/base_executor.py              |  18 +-
 airflow/executors/celery_executor.py            |  24 +-
 airflow/executors/dask_executor.py              |  10 +-
 airflow/executors/local_executor.py             |  11 +-
 airflow/executors/sequential_executor.py        |   4 +-
 airflow/hooks/S3_hook.py                        |  52 +--
 airflow/hooks/base_hook.py                      |   9 +-
 airflow/hooks/dbapi_hook.py                     |  33 +-
 airflow/hooks/druid_hook.py                     |  10 +-
 airflow/hooks/hive_hooks.py                     |  37 +-
 airflow/hooks/http_hook.py                      |   7 +-
 airflow/hooks/oracle_hook.py                    |   9 +-
 airflow/hooks/pig_hook.py                       |   6 +-
 airflow/hooks/presto_hook.py                    |   4 -
 airflow/hooks/webhdfs_hook.py                   |  17 +-
 airflow/hooks/zendesk_hook.py                   |  20 +-
 airflow/jobs.py                                 | 349 ++++++++++---------
 airflow/models.py                               | 211 +++++------
 airflow/operators/bash_operator.py              |  23 +-
 airflow/operators/check_operator.py             |  24 +-
 airflow/operators/dagrun_operator.py            |   6 +-
 airflow/operators/docker_operator.py            |  12 +-
 airflow/operators/generic_transfer.py           |  13 +-
 airflow/operators/hive_operator.py              |   4 +-
 airflow/operators/hive_stats_operator.py        |  10 +-
 airflow/operators/hive_to_druid.py              |  18 +-
 airflow/operators/hive_to_mysql.py              |  14 +-
 airflow/operators/hive_to_samba_operator.py     |   6 +-
 airflow/operators/http_operator.py              |   8 +-
 airflow/operators/jdbc_operator.py              |   7 +-
 airflow/operators/latest_only_operator.py       |  19 +-
 airflow/operators/mssql_operator.py             |   5 +-
 airflow/operators/mssql_to_hive.py              |   6 +-
 airflow/operators/mysql_operator.py             |   5 +-
 airflow/operators/mysql_to_hive.py              |   5 +-
 airflow/operators/oracle_operator.py            |   5 +-
 airflow/operators/pig_operator.py               |   4 +-
 airflow/operators/postgres_operator.py          |   5 +-
 airflow/operators/presto_to_mysql.py            |  12 +-
 airflow/operators/python_operator.py            |  24 +-
 airflow/operators/redshift_to_s3_operator.py    |  17 +-
 airflow/operators/s3_file_transform_operator.py |  30 +-
 airflow/operators/s3_to_hive_operator.py        |  39 +--
 airflow/operators/sensors.py                    |  49 +--
 airflow/operators/slack_operator.py             |   9 +-
 airflow/operators/sqlite_operator.py            |   5 +-
 airflow/plugins_manager.py                      |  11 +-
 airflow/security/kerberos.py                    |  25 +-
 airflow/settings.py                             |  17 +-
 airflow/task_runner/base_task_runner.py         |   9 +-
 airflow/utils/dag_processing.py                 |  55 +--
 airflow/utils/db.py                             |  11 +-
 airflow/utils/email.py                          |   8 +-
 airflow/utils/log/LoggingMixin.py               |  45 +++
 airflow/utils/log/file_task_handler.py          |  34 +-
 airflow/utils/log/gcs_task_handler.py           | 125 ++++++-
 airflow/utils/log/s3_task_handler.py            |  97 +++++-
 airflow/utils/logging.py                        | 252 -------------
 airflow/utils/timeout.py                        |  17 +-
 airflow/www/api/experimental/endpoints.py       |   6 +-
 airflow/www/app.py                              |  10 +-
 airflow/www/views.py                            |   9 +-
 setup.py                                        |  11 -
 tests/contrib/hooks/test_databricks_hook.py     |  15 +-
 .../contrib/operators/test_dataproc_operator.py |  59 ++--
 tests/core.py                                   |  16 +-
 tests/operators/sensors.py                      |  53 +--
 tests/utils/log/test_logging.py                 | 108 ++++++
 tests/utils/test_logging.py                     | 103 ------
 141 files changed, 1578 insertions(+), 1747 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/UPDATING.md
----------------------------------------------------------------------
diff --git a/UPDATING.md b/UPDATING.md
index 92ee4b4..cde7141 100644
--- a/UPDATING.md
+++ b/UPDATING.md
@@ -13,7 +13,9 @@ assists people when migrating to a new version.
   - No updates are required if you are using ftpHook, it will continue work as is.
 
 ### Logging update
-  Logs now are stored in the log folder as ``{dag_id}/{task_id}/{execution_date}/{try_number}.log``.
+Airflow's logging has been rewritten to uses Python’s builtin `logging` module to perform system logging. By extending classes with the existing `LoggingMixin`, all the logging will go through a central logger. The main benefit that this brings to us is the easy configuration of the logging through `default_airflow_logging.py` and the ability to use different handlers for logging.
+
+Logs now are stored in the log folder as `{dag_id}/{task_id}/{execution_date}/{try_number}.log`.
 
 ### New Features
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/__init__.py b/airflow/__init__.py
index 3daa6e2..8844eeb 100644
--- a/airflow/__init__.py
+++ b/airflow/__init__.py
@@ -21,9 +21,10 @@ in their PYTHONPATH. airflow_login should be based off the
 """
 from builtins import object
 from airflow import version
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 __version__ = version.version
 
-import logging
 import sys
 
 from airflow import configuration as conf
@@ -40,13 +41,15 @@ login = None
 
 
 def load_login():
+    log = LoggingMixin().logger
+
     auth_backend = 'airflow.default_login'
     try:
         if conf.getboolean('webserver', 'AUTHENTICATE'):
             auth_backend = conf.get('webserver', 'auth_backend')
     except conf.AirflowConfigException:
         if conf.getboolean('webserver', 'AUTHENTICATE'):
-            logging.warning(
+            log.warning(
                 "auth_backend not found in webserver config reverting to "
                 "*deprecated*  behavior of importing airflow_login")
             auth_backend = "airflow_login"
@@ -55,7 +58,7 @@ def load_login():
         global login
         login = import_module(auth_backend)
     except ImportError as err:
-        logging.critical(
+        log.critical(
             "Cannot import authentication module %s. "
             "Please correct your authentication backend or disable authentication: %s",
             auth_backend, err
@@ -76,7 +79,6 @@ from airflow import operators
 from airflow import hooks
 from airflow import executors
 from airflow import macros
-from airflow import contrib
 
 operators._integrate_plugins()
 hooks._integrate_plugins()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/api/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/api/__init__.py b/airflow/api/__init__.py
index ae47abf..39edbed 100644
--- a/airflow/api/__init__.py
+++ b/airflow/api/__init__.py
@@ -13,14 +13,16 @@
 # limitations under the License.
 from __future__ import print_function
 
-import logging
-
 from airflow.exceptions import AirflowException
 from airflow import configuration as conf
 from importlib import import_module
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 api_auth = None
 
+log = LoggingMixin().logger
+
 
 def load_auth():
     auth_backend = 'airflow.api.auth.backend.default'
@@ -33,6 +35,8 @@ def load_auth():
         global api_auth
         api_auth = import_module(auth_backend)
     except ImportError as err:
-        logging.critical("Cannot import {} for API authentication due to: {}"
-                         .format(auth_backend, err))
+        log.critical(
+            "Cannot import %s for API authentication due to: %s",
+            auth_backend, err
+        )
         raise AirflowException(err)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/api/auth/backend/kerberos_auth.py
----------------------------------------------------------------------
diff --git a/airflow/api/auth/backend/kerberos_auth.py b/airflow/api/auth/backend/kerberos_auth.py
index d1c3b70..73a5aa2 100644
--- a/airflow/api/auth/backend/kerberos_auth.py
+++ b/airflow/api/auth/backend/kerberos_auth.py
@@ -23,10 +23,12 @@
 # SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 from future.standard_library import install_aliases
+
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 install_aliases()
 
 import kerberos
-import logging
 import os
 
 from airflow import configuration as conf
@@ -45,6 +47,8 @@ client_auth = HTTPKerberosAuth(service='airflow')
 
 _SERVICE_NAME = None
 
+log = LoggingMixin().logger
+
 
 def init_app(app):
     global _SERVICE_NAME
@@ -52,7 +56,7 @@ def init_app(app):
     hostname = app.config.get('SERVER_NAME')
     if not hostname:
         hostname = getfqdn()
-    logging.info("Kerberos: hostname {}".format(hostname))
+    log.info("Kerberos: hostname %s", hostname)
 
     service = 'airflow'
 
@@ -62,12 +66,12 @@ def init_app(app):
         os.environ['KRB5_KTNAME'] = conf.get('kerberos', 'keytab')
 
     try:
-        logging.info("Kerberos init: {} {}".format(service, hostname))
+        log.info("Kerberos init: %s %s", service, hostname)
         principal = kerberos.getServerPrincipalDetails(service, hostname)
     except kerberos.KrbError as err:
-        logging.warning("Kerberos: {}".format(err))
+        log.warning("Kerberos: %s", err)
     else:
-        logging.info("Kerberos API: server is {}".format(principal))
+        log.info("Kerberos API: server is %s", principal)
 
 
 def _unauthorized():

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/api/common/experimental/get_task.py
----------------------------------------------------------------------
diff --git a/airflow/api/common/experimental/get_task.py b/airflow/api/common/experimental/get_task.py
index 39ab423..9023ad1 100644
--- a/airflow/api/common/experimental/get_task.py
+++ b/airflow/api/common/experimental/get_task.py
@@ -12,13 +12,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.exceptions import AirflowException
 from airflow.models import DagBag
 
-_log = logging.getLogger(__name__)
-
 
 def get_task(dag_id, task_id):
     """Return the task object identified by the given dag_id and task_id."""

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/api/common/experimental/get_task_instance.py
----------------------------------------------------------------------
diff --git a/airflow/api/common/experimental/get_task_instance.py b/airflow/api/common/experimental/get_task_instance.py
index 4c50731..7ab5e6e 100644
--- a/airflow/api/common/experimental/get_task_instance.py
+++ b/airflow/api/common/experimental/get_task_instance.py
@@ -12,13 +12,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.exceptions import AirflowException
 from airflow.models import DagBag
 
-_log = logging.getLogger(__name__)
-
 
 def get_task_instance(dag_id, task_id, execution_date):
     """Return the task object identified by the given dag_id and task_id."""

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/bin/airflow
----------------------------------------------------------------------
diff --git a/airflow/bin/airflow b/airflow/bin/airflow
index 0598596..2c0024d 100755
--- a/airflow/bin/airflow
+++ b/airflow/bin/airflow
@@ -12,7 +12,6 @@
 # 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
 import os
 from airflow import configuration
 from airflow.bin.cli import CLIFactory

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/bin/cli.py
----------------------------------------------------------------------
diff --git a/airflow/bin/cli.py b/airflow/bin/cli.py
index a0545c3..56f1855 100755
--- a/airflow/bin/cli.py
+++ b/airflow/bin/cli.py
@@ -53,6 +53,7 @@ from airflow.models import (DagModel, DagBag, TaskInstance,
 
 from airflow.ti_deps.dep_context import (DepContext, SCHEDULER_DEPS)
 from airflow.utils import db as db_utils
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.www.app import cached_app
 
 from sqlalchemy import func
@@ -63,6 +64,8 @@ api_module = import_module(conf.get('cli', 'api_client'))
 api_client = api_module.Client(api_base_url=conf.get('cli', 'endpoint_url'),
                                auth=api.api_auth.client_auth)
 
+log = LoggingMixin().logger
+
 
 def sigint_handler(sig, frame):
     sys.exit(0)
@@ -186,19 +189,21 @@ def trigger_dag(args):
     :param args:
     :return:
     """
+    log = LoggingMixin().logger
     try:
         message = api_client.trigger_dag(dag_id=args.dag_id,
                                          run_id=args.run_id,
                                          conf=args.conf,
                                          execution_date=args.exec_date)
     except IOError as err:
-        logging.error(err)
+        log.error(err)
         raise AirflowException(err)
-
-    logging.info(message)
+    log.info(message)
 
 
 def pool(args):
+    log = LoggingMixin().logger
+
     def _tabulate(pools):
         return "\n%s" % tabulate(pools, ['Pool', 'Slots', 'Description'],
                                  tablefmt="fancy_grid")
@@ -215,9 +220,9 @@ def pool(args):
         else:
             pools = api_client.get_pools()
     except (AirflowException, IOError) as err:
-        logging.error(err)
+        log.error(err)
     else:
-        logging.info(_tabulate(pools=pools))
+        log.info(_tabulate(pools=pools))
 
 
 def variables(args):
@@ -325,6 +330,8 @@ def run(args, dag=None):
     if dag:
         args.dag_id = dag.dag_id
 
+    log = LoggingMixin().logger
+
     # Load custom airflow config
     if args.cfg_path:
         with open(args.cfg_path, 'r') as conf_file:
@@ -343,7 +350,7 @@ def run(args, dag=None):
         dag = get_dag(args)
     elif not dag:
         session = settings.Session()
-        logging.info('Loading pickle id {args.pickle}'.format(args=args))
+        log.info('Loading pickle id {args.pickle}'.format(args=args))
         dag_pickle = session.query(
             DagPickle).filter(DagPickle.id == args.pickle).first()
         if not dag_pickle:
@@ -354,11 +361,11 @@ def run(args, dag=None):
     ti = TaskInstance(task, args.execution_date)
     ti.refresh_from_db()
 
-    logger = logging.getLogger('airflow.task')
+    log = logging.getLogger('airflow.task')
     if args.raw:
-        logger = logging.getLogger('airflow.task.raw')
+        log = logging.getLogger('airflow.task.raw')
 
-    for handler in logger.handlers:
+    for handler in log.handlers:
         try:
             handler.set_context(ti)
         except AttributeError:
@@ -367,7 +374,7 @@ def run(args, dag=None):
             pass
 
     hostname = socket.getfqdn()
-    logging.info("Running on host {}".format(hostname))
+    log.info("Running on host %s", hostname)
 
     if args.local:
         run_job = jobs.LocalTaskJob(
@@ -396,6 +403,7 @@ def run(args, dag=None):
                 session.add(pickle)
                 session.commit()
                 pickle_id = pickle.id
+                # TODO: This should be written to a log
                 print((
                           'Pickled dag {dag} '
                           'as pickle_id:{pickle_id}').format(**locals()))
@@ -427,7 +435,7 @@ def run(args, dag=None):
     # might subsequently read from the log to insert into S3 or
     # Google cloud storage. Explicitly close the handler is
     # needed in order to upload to remote storage services.
-    for handler in logger.handlers:
+    for handler in log.handlers:
         handler.flush()
         handler.close()
 
@@ -449,6 +457,7 @@ def task_failed_deps(args):
 
     dep_context = DepContext(deps=SCHEDULER_DEPS)
     failed_deps = list(ti.get_failed_dep_statuses(dep_context=dep_context))
+    # TODO, Do we want to print or log this
     if failed_deps:
         print("Task instance dependencies not met:")
         for dep in failed_deps:
@@ -605,8 +614,7 @@ def restart_workers(gunicorn_master_proc, num_workers_expected):
 
     def start_refresh(gunicorn_master_proc):
         batch_size = conf.getint('webserver', 'worker_refresh_batch_size')
-        logging.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()
 
@@ -628,14 +636,14 @@ def restart_workers(gunicorn_master_proc, num_workers_expected):
 
         # Whenever some workers are not ready, wait until all workers are ready
         if num_ready_workers_running < num_workers_running:
-            logging.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
-            logging.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)
@@ -646,7 +654,7 @@ def restart_workers(gunicorn_master_proc, num_workers_expected):
         # 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')
-            logging.debug(
+            log.debug(
                 '%s sleeping for %ss starting doing a refresh...',
                 state, refresh_interval
             )
@@ -655,7 +663,7 @@ def restart_workers(gunicorn_master_proc, num_workers_expected):
 
         else:
             # num_ready_workers_running == num_workers_running < num_workers_expected
-            logging.error((
+            log.error((
                 "%s some workers seem to have died and gunicorn"
                 "did not restart them as expected"
             ), state)
@@ -770,7 +778,7 @@ def webserver(args):
                             gunicorn_master_proc_pid = int(f.read())
                             break
                     except IOError:
-                        logging.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)
@@ -896,8 +904,6 @@ def resetdb(args):
     if args.yes or input(
         "This will drop existing tables if they exist. "
         "Proceed? (y/n)").upper() == "Y":
-        logging.basicConfig(level=settings.LOGGING_LEVEL,
-                            format=settings.SIMPLE_LOG_FORMAT)
         db_utils.resetdb()
     else:
         print("Bail.")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/configuration.py
----------------------------------------------------------------------
diff --git a/airflow/configuration.py b/airflow/configuration.py
index 460d975..db196f9 100644
--- a/airflow/configuration.py
+++ b/airflow/configuration.py
@@ -19,7 +19,6 @@ from __future__ import unicode_literals
 
 import copy
 import errno
-import logging
 import os
 import six
 import subprocess
@@ -28,6 +27,9 @@ import shlex
 import sys
 
 from future import standard_library
+
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 standard_library.install_aliases()
 
 from builtins import str
@@ -36,6 +38,8 @@ from six.moves import configparser
 
 from airflow.exceptions import AirflowConfigException
 
+log = LoggingMixin().logger
+
 # show Airflow's deprecation warnings
 warnings.filterwarnings(
     action='default', category=DeprecationWarning, module='airflow')
@@ -198,8 +202,9 @@ class AirflowConfigParser(ConfigParser):
             return option
 
         else:
-            logging.warning("section/key [{section}/{key}] not found "
-                            "in config".format(**locals()))
+            log.warning(
+                "section/key [{section}/{key}] not found in config".format(**locals())
+            )
 
             raise AirflowConfigException(
                 "section/key [{section}/{key}] not found "
@@ -366,20 +371,22 @@ else:
 TEMPLATE_START = (
     '# ----------------------- TEMPLATE BEGINS HERE -----------------------')
 if not os.path.isfile(TEST_CONFIG_FILE):
-    logging.info(
-        'Creating new Airflow config file for unit tests in: {}'.format(
-            TEST_CONFIG_FILE))
+    log.info(
+        'Creating new Airflow config file for unit tests in: %s', TEST_CONFIG_FILE
+    )
     with open(TEST_CONFIG_FILE, 'w') as f:
         cfg = parameterized_config(TEST_CONFIG)
         f.write(cfg.split(TEMPLATE_START)[-1].strip())
 if not os.path.isfile(AIRFLOW_CONFIG):
-    logging.info('Creating new Airflow config file in: {}'.format(
-        AIRFLOW_CONFIG))
+    log.info(
+        'Creating new Airflow config file in: %s',
+        AIRFLOW_CONFIG
+    )
     with open(AIRFLOW_CONFIG, 'w') as f:
         cfg = parameterized_config(DEFAULT_CONFIG)
         f.write(cfg.split(TEMPLATE_START)[-1].strip())
 
-logging.info("Reading the config from " + AIRFLOW_CONFIG)
+log.info("Reading the config from %s", AIRFLOW_CONFIG)
 
 conf = AirflowConfigParser()
 conf.read(AIRFLOW_CONFIG)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/auth/backends/github_enterprise_auth.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/auth/backends/github_enterprise_auth.py b/airflow/contrib/auth/backends/github_enterprise_auth.py
index 91126c7..459e9c9 100644
--- a/airflow/contrib/auth/backends/github_enterprise_auth.py
+++ b/airflow/contrib/auth/backends/github_enterprise_auth.py
@@ -11,8 +11,6 @@
 # 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
-
 import flask_login
 
 # Need to expose these downstream
@@ -29,8 +27,9 @@ from flask_oauthlib.client import OAuth
 
 from airflow import models, configuration, settings
 from airflow.configuration import AirflowConfigException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-_log = logging.getLogger(__name__)
+log = LoggingMixin().logger
 
 
 def get_config_param(param):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/auth/backends/google_auth.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/auth/backends/google_auth.py b/airflow/contrib/auth/backends/google_auth.py
index 70c8e13..f38f725 100644
--- a/airflow/contrib/auth/backends/google_auth.py
+++ b/airflow/contrib/auth/backends/google_auth.py
@@ -11,8 +11,6 @@
 # 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
-
 import flask_login
 
 # Need to expose these downstream
@@ -28,9 +26,9 @@ from flask import url_for, redirect, request
 from flask_oauthlib.client import OAuth
 
 from airflow import models, configuration, settings
-from airflow.configuration import AirflowConfigException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-_log = logging.getLogger(__name__)
+log = LoggingMixin().logger
 
 
 def get_config_param(param):
@@ -106,7 +104,7 @@ class GoogleAuthBackend(object):
                                     self.oauth_callback)
 
     def login(self, request):
-        _log.debug('Redirecting user to Google login')
+        log.debug('Redirecting user to Google login')
         return self.google_oauth.authorize(callback=url_for(
             'google_oauth_callback',
             _external=True,
@@ -142,7 +140,7 @@ class GoogleAuthBackend(object):
         return GoogleUser(user)
 
     def oauth_callback(self):
-        _log.debug('Google OAuth callback called')
+        log.debug('Google OAuth callback called')
 
         next_url = request.args.get('next') or url_for('admin.index')
 
@@ -162,7 +160,6 @@ class GoogleAuthBackend(object):
                 return redirect(url_for('airflow.noaccess'))
 
         except AuthenticationError:
-            _log.exception('')
             return redirect(url_for('airflow.noaccess'))
 
         session = settings.Session()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/auth/backends/kerberos_auth.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/auth/backends/kerberos_auth.py b/airflow/contrib/auth/backends/kerberos_auth.py
index e381059..ffb711f 100644
--- a/airflow/contrib/auth/backends/kerberos_auth.py
+++ b/airflow/contrib/auth/backends/kerberos_auth.py
@@ -29,8 +29,7 @@ from flask import url_for, redirect
 from airflow import settings
 from airflow import models
 from airflow import configuration
-
-import logging
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 login_manager = flask_login.LoginManager()
 login_manager.login_view = 'airflow.login'  # Calls login() below
@@ -41,7 +40,7 @@ class AuthenticationError(Exception):
     pass
 
 
-class KerberosUser(models.User):
+class KerberosUser(models.User, LoggingMixin):
     def __init__(self, user):
         self.user = user
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/auth/backends/ldap_auth.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/auth/backends/ldap_auth.py b/airflow/contrib/auth/backends/ldap_auth.py
index 341f710..8ce0875 100644
--- a/airflow/contrib/auth/backends/ldap_auth.py
+++ b/airflow/contrib/auth/backends/ldap_auth.py
@@ -30,16 +30,16 @@ from airflow import models
 from airflow import configuration
 from airflow.configuration import AirflowConfigException
 
-import logging
-
 import traceback
 import re
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 login_manager = flask_login.LoginManager()
 login_manager.login_view = 'airflow.login'  # Calls login() below
 login_manager.login_message = None
 
-LOG = logging.getLogger(__name__)
+log = LoggingMixin().logger
 
 
 class AuthenticationError(Exception):
@@ -64,7 +64,7 @@ def get_ldap_connection(dn=None, password=None):
     conn = Connection(server, native(dn), native(password))
 
     if not conn.bind():
-        LOG.error("Cannot bind to ldap server: %s ", conn.last_error)
+        log.error("Cannot bind to ldap server: %s ", conn.last_error)
         raise AuthenticationError("Cannot bind to ldap server")
 
     return conn
@@ -74,7 +74,7 @@ def group_contains_user(conn, search_base, group_filter, user_name_attr, usernam
     search_filter = '(&({0}))'.format(group_filter)
     if not conn.search(native(search_base), native(search_filter),
                        attributes=[native(user_name_attr)]):
-        LOG.warning("Unable to find group for %s %s", search_base, search_filter)
+        log.warning("Unable to find group for %s %s", search_base, search_filter)
     else:
         for resp in conn.response:
             if (
@@ -95,11 +95,11 @@ def groups_user(conn, search_base, user_filter, user_name_att, username):
         memberof_attr = "memberOf"
     res = conn.search(native(search_base), native(search_filter), attributes=[native(memberof_attr)])
     if not res:
-        LOG.info("Cannot find user %s", username)
+        log.info("Cannot find user %s", username)
         raise AuthenticationError("Invalid username or password")
 
     if conn.response and memberof_attr not in conn.response[0]["attributes"]:
-        LOG.warning("""Missing attribute "%s" when looked-up in Ldap database.
+        log.warning("""Missing attribute "%s" when looked-up in Ldap database.
         The user does not seem to be a member of a group and therefore won't see any dag
         if the option filter_by_owner=True and owner_mode=ldapgroup are set""", memberof_attr)
         return []
@@ -111,7 +111,7 @@ def groups_user(conn, search_base, user_filter, user_name_att, username):
     try:
         groups_list = [regex.search(i).group(1) for i in user_groups]
     except IndexError:
-        LOG.warning("Parsing error when retrieving the user's group(s)."
+        log.warning("Parsing error when retrieving the user's group(s)."
                     " Check if the user belongs to at least one group"
                     " or if the user's groups name do not contain special characters")
 
@@ -134,7 +134,7 @@ class LdapUser(models.User):
                                                  user.username)
         except AirflowConfigException:
             self.superuser = True
-            LOG.debug("Missing configuration for superuser settings.  Skipping.")
+            log.debug("Missing configuration for superuser settings.  Skipping.")
 
         try:
             self.data_profiler = group_contains_user(conn,
@@ -144,7 +144,7 @@ class LdapUser(models.User):
                                                      user.username)
         except AirflowConfigException:
             self.data_profiler = True
-            LOG.debug("Missing configuration for dataprofiler settings. Skipping")
+            log.debug("Missing configuration for dataprofiler settings. Skipping")
 
         # Load the ldap group(s) a user belongs to
         try:
@@ -154,7 +154,7 @@ class LdapUser(models.User):
                                            configuration.get("ldap", "user_name_attr"),
                                            user.username)
         except AirflowConfigException:
-            LOG.debug("Missing configuration for ldap settings. Skipping")
+            log.debug("Missing configuration for ldap settings. Skipping")
 
     @staticmethod
     def try_login(username, password):
@@ -185,7 +185,7 @@ class LdapUser(models.User):
 
         # todo: use list or result?
         if not res:
-            LOG.info("Cannot find user %s", username)
+            log.info("Cannot find user %s", username)
             raise AuthenticationError("Invalid username or password")
 
         entry = conn.response[0]
@@ -200,14 +200,14 @@ class LdapUser(models.User):
         try:
             conn = get_ldap_connection(entry['dn'], password)
         except KeyError as e:
-            LOG.error("""
+            log.error("""
             Unable to parse LDAP structure. If you're using Active Directory and not specifying an OU, you must set search_scope=SUBTREE in airflow.cfg.
             %s
             """ % traceback.format_exc())
             raise LdapException("Could not parse LDAP structure. Try setting search_scope in airflow.cfg, or check logs")
 
         if not conn:
-            LOG.info("Password incorrect for user %s", username)
+            log.info("Password incorrect for user %s", username)
             raise AuthenticationError("Invalid username or password")
 
     def is_active(self):
@@ -237,7 +237,7 @@ class LdapUser(models.User):
 
 @login_manager.user_loader
 def load_user(userid):
-    LOG.debug("Loading user %s", userid)
+    log.debug("Loading user %s", userid)
     if not userid or userid == 'None':
         return None
 
@@ -270,7 +270,7 @@ def login(self, request):
 
     try:
         LdapUser.try_login(username, password)
-        LOG.info("User %s successfully authenticated", username)
+        log.info("User %s successfully authenticated", username)
 
         session = settings.Session()
         user = session.query(models.User).filter(

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/auth/backends/password_auth.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/auth/backends/password_auth.py b/airflow/contrib/auth/backends/password_auth.py
index 000b986..3ad2a8b 100644
--- a/airflow/contrib/auth/backends/password_auth.py
+++ b/airflow/contrib/auth/backends/password_auth.py
@@ -32,15 +32,13 @@ from sqlalchemy.ext.hybrid import hybrid_property
 
 from airflow import settings
 from airflow import models
-from airflow import configuration
-
-import logging
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 login_manager = flask_login.LoginManager()
 login_manager.login_view = 'airflow.login'  # Calls login() below
 login_manager.login_message = None
 
-LOG = logging.getLogger(__name__)
+log = LoggingMixin().logger
 PY3 = version_info[0] == 3
 
 
@@ -94,7 +92,7 @@ class PasswordUser(models.User):
 
 @login_manager.user_loader
 def load_user(userid):
-    LOG.debug("Loading user %s", userid)
+    log.debug("Loading user %s", userid)
     if not userid or userid == 'None':
         return None
 
@@ -137,7 +135,7 @@ def login(self, request):
         if not user.authenticate(password):
             session.close()
             raise AuthenticationError()
-        LOG.info("User %s successfully authenticated", username)
+        log.info("User %s successfully authenticated", username)
 
         flask_login.login_user(user)
         session.commit()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/executors/mesos_executor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/executors/mesos_executor.py b/airflow/contrib/executors/mesos_executor.py
index 49788fc..19d72ed 100644
--- a/airflow/contrib/executors/mesos_executor.py
+++ b/airflow/contrib/executors/mesos_executor.py
@@ -13,9 +13,12 @@
 # limitations under the License.
 
 from future import standard_library
+
+from airflow.utils.log.LoggingMixin import LoggingMixin
+from airflow.www.utils import LoginMixin
+
 standard_library.install_aliases()
 from builtins import str
-import logging
 from queue import Queue
 
 import mesos.interface
@@ -41,7 +44,7 @@ def get_framework_name():
 
 # AirflowMesosScheduler, implements Mesos Scheduler interface
 # To schedule airflow jobs on mesos
-class AirflowMesosScheduler(mesos.interface.Scheduler):
+class AirflowMesosScheduler(mesos.interface.Scheduler, LoggingMixin):
     """
     Airflow Mesos scheduler implements mesos scheduler interface
     to schedule airflow tasks on mesos.
@@ -49,7 +52,6 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
     'airflow run <dag_id> <task_instance_id> <start_date> --local -p=<pickle>'
     to run on a mesos slave.
     """
-
     def __init__(self,
                  task_queue,
                  result_queue,
@@ -63,7 +65,7 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
         self.task_key_map = {}
 
     def registered(self, driver, frameworkId, masterInfo):
-        logging.info("AirflowScheduler registered to mesos with framework ID %s", frameworkId.value)
+        self.logger.info("AirflowScheduler registered to Mesos with framework ID %s", frameworkId.value)
 
         if configuration.getboolean('mesos', 'CHECKPOINT') and configuration.get('mesos', 'FAILOVER_TIMEOUT'):
             # Import here to work around a circular import error
@@ -84,25 +86,25 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
             Session.remove()
 
     def reregistered(self, driver, masterInfo):
-        logging.info("AirflowScheduler re-registered to mesos")
+        self.logger.info("AirflowScheduler re-registered to mesos")
 
     def disconnected(self, driver):
-        logging.info("AirflowScheduler disconnected from mesos")
+        self.logger.info("AirflowScheduler disconnected from mesos")
 
     def offerRescinded(self, driver, offerId):
-        logging.info("AirflowScheduler offer %s rescinded", str(offerId))
+        self.logger.info("AirflowScheduler offer %s rescinded", str(offerId))
 
     def frameworkMessage(self, driver, executorId, slaveId, message):
-        logging.info("AirflowScheduler received framework message %s", message)
+        self.logger.info("AirflowScheduler received framework message %s", message)
 
     def executorLost(self, driver, executorId, slaveId, status):
-        logging.warning("AirflowScheduler executor %s lost", str(executorId))
+        self.logger.warning("AirflowScheduler executor %s lost", str(executorId))
 
     def slaveLost(self, driver, slaveId):
-        logging.warning("AirflowScheduler slave %s lost", str(slaveId))
+        self.logger.warning("AirflowScheduler slave %s lost", str(slaveId))
 
     def error(self, driver, message):
-        logging.error("AirflowScheduler driver aborted %s", message)
+        self.logger.error("AirflowScheduler driver aborted %s", message)
         raise AirflowException("AirflowScheduler driver aborted %s" % message)
 
     def resourceOffers(self, driver, offers):
@@ -116,7 +118,7 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
                 elif resource.name == "mem":
                     offerMem += resource.scalar.value
 
-            logging.info("Received offer %s with cpus: %s and mem: %s", offer.id.value, offerCpus, offerMem)
+            self.logger.info("Received offer %s with cpus: %s and mem: %s", offer.id.value, offerCpus, offerMem)
 
             remainingCpus = offerCpus
             remainingMem = offerMem
@@ -129,7 +131,7 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
                 self.task_counter += 1
                 self.task_key_map[str(tid)] = key
 
-                logging.info("Launching task %d using offer %s", tid, offer.id.value)
+                self.logger.info("Launching task %d using offer %s", tid, offer.id.value)
 
                 task = mesos_pb2.TaskInfo()
                 task.task_id.value = str(tid)
@@ -159,15 +161,17 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
             driver.launchTasks(offer.id, tasks)
 
     def statusUpdate(self, driver, update):
-        logging.info("Task %s is in state %s, data %s",
-                     update.task_id.value, mesos_pb2.TaskState.Name(update.state), str(update.data))
+        self.logger.info(
+            "Task %s is in state %s, data %s",
+            update.task_id.value, mesos_pb2.TaskState.Name(update.state), str(update.data)
+        )
 
         try:
             key = self.task_key_map[update.task_id.value]
         except KeyError:
             # The map may not contain an item if the framework re-registered after a failover.
             # Discard these tasks.
-            logging.warning("Unrecognised task key %s" % update.task_id.value)
+            self.logger.warning("Unrecognised task key %s", update.task_id.value)
             return
 
         if update.state == mesos_pb2.TASK_FINISHED:
@@ -181,7 +185,7 @@ class AirflowMesosScheduler(mesos.interface.Scheduler):
             self.task_queue.task_done()
 
 
-class MesosExecutor(BaseExecutor):
+class MesosExecutor(BaseExecutor, LoginMixin):
     """
     MesosExecutor allows distributing the execution of task
     instances to multiple mesos workers.
@@ -192,7 +196,6 @@ class MesosExecutor(BaseExecutor):
     elastic distributed systems to easily be built and run effectively.
     See http://mesos.apache.org/
     """
-
     def start(self):
         self.task_queue = Queue()
         self.result_queue = Queue()
@@ -200,7 +203,7 @@ class MesosExecutor(BaseExecutor):
         framework.user = ''
 
         if not configuration.get('mesos', 'MASTER'):
-            logging.error("Expecting mesos master URL for mesos executor")
+            self.logger.error("Expecting mesos master URL for mesos executor")
             raise AirflowException("mesos.master not provided for mesos executor")
 
         master = configuration.get('mesos', 'MASTER')
@@ -236,17 +239,19 @@ class MesosExecutor(BaseExecutor):
         else:
             framework.checkpoint = False
 
-        logging.info('MesosFramework master : %s, name : %s, cpu : %s, mem : %s, checkpoint : %s',
-            master, framework.name, str(task_cpu), str(task_memory), str(framework.checkpoint))
+        self.logger.info(
+            'MesosFramework master : %s, name : %s, cpu : %s, mem : %s, checkpoint : %s',
+            master, framework.name, str(task_cpu), str(task_memory), str(framework.checkpoint)
+        )
 
         implicit_acknowledgements = 1
 
         if configuration.getboolean('mesos', 'AUTHENTICATE'):
             if not configuration.get('mesos', 'DEFAULT_PRINCIPAL'):
-                logging.error("Expecting authentication principal in the environment")
+                self.logger.error("Expecting authentication principal in the environment")
                 raise AirflowException("mesos.default_principal not provided in authenticated mode")
             if not configuration.get('mesos', 'DEFAULT_SECRET'):
-                logging.error("Expecting authentication secret in the environment")
+                self.logger.error("Expecting authentication secret in the environment")
                 raise AirflowException("mesos.default_secret not provided in authenticated mode")
 
             credential = mesos_pb2.Credential()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/bigquery_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/bigquery_hook.py b/airflow/contrib/hooks/bigquery_hook.py
index d2ce2b0..497fa28 100644
--- a/airflow/contrib/hooks/bigquery_hook.py
+++ b/airflow/contrib/hooks/bigquery_hook.py
@@ -18,7 +18,6 @@ This module contains a BigQuery Hook, as well as a very basic PEP 249
 implementation for BigQuery.
 """
 
-import logging
 import time
 
 from apiclient.discovery import build, HttpError
@@ -33,11 +32,10 @@ from past.builtins import basestring
 
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
 from airflow.hooks.dbapi_hook import DbApiHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-logging.getLogger("bigquery").setLevel(logging.INFO)
 
-
-class BigQueryHook(GoogleCloudBaseHook, DbApiHook):
+class BigQueryHook(GoogleCloudBaseHook, DbApiHook, LoggingMixin):
     """
     Interact with BigQuery. This hook uses the Google Cloud Platform
     connection.
@@ -178,13 +176,12 @@ class BigQueryConnection(object):
             "BigQueryConnection does not have transactions")
 
 
-class BigQueryBaseCursor(object):
+class BigQueryBaseCursor(LoggingMixin):
     """
     The BigQuery base cursor contains helper methods to execute queries against
     BigQuery. The methods can be used directly by operators, in cases where a
     PEP 249 cursor isn't needed.
     """
-
     def __init__(self, service, project_id):
         self.service = service
         self.project_id = project_id
@@ -290,10 +287,12 @@ class BigQueryBaseCursor(object):
         :param print_header: Whether to print a header for a CSV file extract.
         :type print_header: boolean
         """
+
         source_project, source_dataset, source_table = \
             _split_tablename(table_input=source_project_dataset_table,
                              default_project_id=self.project_id,
                              var_name='source_project_dataset_table')
+
         configuration = {
             'extract': {
                 'sourceTable': {
@@ -500,7 +499,7 @@ class BigQueryBaseCursor(object):
                     "'WRITE_APPEND' or 'WRITE_TRUNCATE'."
                 )
             else:
-                logging.info(
+                self.logger.info(
                     "Adding experimental "
                     "'schemaUpdateOptions': {0}".format(schema_update_options)
                 )
@@ -577,12 +576,12 @@ class BigQueryBaseCursor(object):
                             )
                         )
                 else:
-                    logging.info('Waiting for job to complete : %s, %s', self.project_id, job_id)
+                    self.logger.info('Waiting for job to complete : %s, %s', self.project_id, job_id)
                     time.sleep(5)
 
             except HttpError as err:
                 if err.resp.status in [500, 503]:
-                    logging.info('%s: Retryable error, waiting for job to complete: %s', err.resp.status, job_id)
+                    self.logger.info('%s: Retryable error, waiting for job to complete: %s', err.resp.status, job_id)
                     time.sleep(5)
                 else:
                     raise Exception(
@@ -661,14 +660,14 @@ class BigQueryBaseCursor(object):
                         datasetId=deletion_dataset,
                         tableId=deletion_table) \
                 .execute()
-            logging.info('Deleted table %s:%s.%s.',
+            self.logger.info('Deleted table %s:%s.%s.',
                          deletion_project, deletion_dataset, deletion_table)
         except HttpError:
             if not ignore_if_missing:
                 raise Exception(
                     'Table deletion failed. Table does not exist.')
             else:
-                logging.info('Table does not exist. Skipping.')
+                self.logger.info('Table does not exist. Skipping.')
 
 
     def run_table_upsert(self, dataset_id, table_resource, project_id=None):
@@ -695,8 +694,10 @@ class BigQueryBaseCursor(object):
             for table in tables_list_resp.get('tables', []):
                 if table['tableReference']['tableId'] == table_id:
                     # found the table, do update
-                    logging.info('table %s:%s.%s exists, updating.',
-                                 project_id, dataset_id, table_id)
+                    self.logger.info(
+                        'Table %s:%s.%s exists, updating.',
+                        project_id, dataset_id, table_id
+                    )
                     return self.service.tables().update(projectId=project_id,
                                                         datasetId=dataset_id,
                                                         tableId=table_id,
@@ -711,8 +712,10 @@ class BigQueryBaseCursor(object):
             # If there is no next page, then the table doesn't exist.
             else:
                 # do insert
-                logging.info('table %s:%s.%s does not exist. creating.',
-                             project_id, dataset_id, table_id)
+                self.logger.info(
+                    'Table %s:%s.%s does not exist. creating.',
+                    project_id, dataset_id, table_id
+                )
                 return self.service.tables().insert(projectId=project_id,
                                                     datasetId=dataset_id,
                                                     body=table_resource).execute()
@@ -756,18 +759,20 @@ class BigQueryBaseCursor(object):
                                 'tableId': view_table}}
         # check to see if the view we want to add already exists.
         if view_access not in access:
-            logging.info('granting table %s:%s.%s authorized view access to %s:%s dataset.',
-                         view_project, view_dataset, view_table,
-                         source_project, source_dataset)
+            self.logger.info(
+                'Granting table %s:%s.%s authorized view access to %s:%s dataset.',
+                view_project, view_dataset, view_table, source_project, source_dataset
+            )
             access.append(view_access)
             return self.service.datasets().patch(projectId=source_project,
                                                  datasetId=source_dataset,
                                                  body={'access': access}).execute()
         else:
             # if view is already in access, do nothing.
-            logging.info('table %s:%s.%s already has authorized view access to %s:%s dataset.',
-                         view_project, view_dataset, view_table,
-                         source_project, source_dataset)
+            self.logger.info(
+                'Table %s:%s.%s already has authorized view access to %s:%s dataset.',
+                view_project, view_dataset, view_table, source_project, source_dataset
+            )
             return source_dataset_resource
 
 
@@ -1027,10 +1032,12 @@ def _split_tablename(table_input, default_project_id, var_name=None):
 
     if project_id is None:
         if var_name is not None:
-            logging.info(
-                'project not included in {var}: '
-                '{input}; using project "{project}"'.format(
-                    var=var_name, input=table_input, project=default_project_id))
+            log = LoggingMixin().logger
+            log.info(
+                'Project not included in {var}: {input}; using project "{project}"'.format(
+                    var=var_name, input=table_input, project=default_project_id
+                )
+            )
         project_id = default_project_id
 
     return project_id, dataset_id, table_id

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/cloudant_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/cloudant_hook.py b/airflow/contrib/hooks/cloudant_hook.py
index 6cea26f..d9db08d 100644
--- a/airflow/contrib/hooks/cloudant_hook.py
+++ b/airflow/contrib/hooks/cloudant_hook.py
@@ -15,10 +15,10 @@
 from past.builtins import unicode
 
 import cloudant
-import logging
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
 class CloudantHook(BaseHook):
@@ -35,9 +35,11 @@ class CloudantHook(BaseHook):
         def _str(s):
             # cloudant-python doesn't support unicode.
             if isinstance(s, unicode):
-                logging.debug(('cloudant-python does not support unicode. '
-                               'Encoding %s as ascii using "ignore".'),
-                              s)
+                log = LoggingMixin().logger
+                log.debug(
+                    'cloudant-python does not support unicode. Encoding %s as ascii using "ignore".',
+                    s
+                )
                 return s.encode('ascii', 'ignore')
 
             return s

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/databricks_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/databricks_hook.py b/airflow/contrib/hooks/databricks_hook.py
index 18e20c4..7b20433 100644
--- a/airflow/contrib/hooks/databricks_hook.py
+++ b/airflow/contrib/hooks/databricks_hook.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
 import requests
 
 from airflow import __version__
@@ -22,6 +20,7 @@ from airflow.hooks.base_hook import BaseHook
 from requests import exceptions as requests_exceptions
 from requests.auth import AuthBase
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 try:
     from urllib import parse as urlparse
@@ -35,7 +34,7 @@ CANCEL_RUN_ENDPOINT = ('POST', 'api/2.0/jobs/runs/cancel')
 USER_AGENT_HEADER = {'user-agent': 'airflow-{v}'.format(v=__version__)}
 
 
-class DatabricksHook(BaseHook):
+class DatabricksHook(BaseHook, LoggingMixin):
     """
     Interact with Databricks.
     """
@@ -101,10 +100,10 @@ class DatabricksHook(BaseHook):
             host=self._parse_host(self.databricks_conn.host),
             endpoint=endpoint)
         if 'token' in self.databricks_conn.extra_dejson:
-            logging.info('Using token auth.')
+            self.logger.info('Using token auth.')
             auth = _TokenAuth(self.databricks_conn.extra_dejson['token'])
         else:
-            logging.info('Using basic auth.')
+            self.logger.info('Using basic auth.')
             auth = (self.databricks_conn.login, self.databricks_conn.password)
         if method == 'GET':
             request_func = requests.get
@@ -130,8 +129,10 @@ class DatabricksHook(BaseHook):
                         response.content, response.status_code))
             except (requests_exceptions.ConnectionError,
                     requests_exceptions.Timeout) as e:
-                logging.error(('Attempt {0} API Request to Databricks failed ' +
-                              'with reason: {1}').format(attempt_num, e))
+                self.logger.error(
+                    'Attempt %s API Request to Databricks failed with reason: %s',
+                    attempt_num, e
+                )
         raise AirflowException(('API requests to Databricks failed {} times. ' +
                                'Giving up.').format(self.retry_limit))
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/datadog_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/datadog_hook.py b/airflow/contrib/hooks/datadog_hook.py
index 2125701..0f5af00 100644
--- a/airflow/contrib/hooks/datadog_hook.py
+++ b/airflow/contrib/hooks/datadog_hook.py
@@ -13,14 +13,14 @@
 # limitations under the License.
 
 import time
-import logging
-
 from airflow.hooks.base_hook import BaseHook
 from airflow.exceptions import AirflowException
 from datadog import initialize, api
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 
-class DatadogHook(BaseHook):
+class DatadogHook(BaseHook, LoggingMixin):
     """
     Uses datadog API to send metrics of practically anything measurable,
     so it's possible to track # of db records inserted/deleted, records read
@@ -32,7 +32,6 @@ class DatadogHook(BaseHook):
     :param datadog_conn_id: The connection to datadog, containing metadata for api keys.
     :param datadog_conn_id: string
     """
-
     def __init__(self, datadog_conn_id='datadog_default'):
         conn = self.get_connection(datadog_conn_id)
         self.api_key = conn.extra_dejson.get('api_key', None)
@@ -48,7 +47,7 @@ class DatadogHook(BaseHook):
         if self.app_key is None:
             raise AirflowException("app_key must be specified in the Datadog connection details")
 
-        logging.info("Setting up api keys for datadog")
+        self.logger.info("Setting up api keys for Datadog")
         options = {
             'api_key': self.api_key,
             'app_key': self.app_key
@@ -57,8 +56,8 @@ class DatadogHook(BaseHook):
 
     def validate_response(self, response):
         if response['status'] != 'ok':
-            logging.error("Data dog returned: " + response)
-            raise AirflowException("Error status received from datadog")
+            self.logger.error("Datadog returned: %s", response)
+            raise AirflowException("Error status received from Datadog")
 
     def send_metric(self, metric_name, datapoint, tags=None):
         """

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/datastore_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/datastore_hook.py b/airflow/contrib/hooks/datastore_hook.py
index 7a4386a..2ff1600 100644
--- a/airflow/contrib/hooks/datastore_hook.py
+++ b/airflow/contrib/hooks/datastore_hook.py
@@ -15,7 +15,6 @@
 
 import json
 import time
-import logging
 from apiclient.discovery import build
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
 
@@ -137,7 +136,7 @@ class DatastoreHook(GoogleCloudBaseHook):
             result = self.get_operation(name)
             state = result['metadata']['common']['state']
             if state == 'PROCESSING':
-                logging.info('Operation is processing. Re-polling state in {} seconds'
+                self.logger.info('Operation is processing. Re-polling state in {} seconds'
                         .format(polling_interval_in_seconds))
                 time.sleep(polling_interval_in_seconds)
             else:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/ftp_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/ftp_hook.py b/airflow/contrib/hooks/ftp_hook.py
index 148811f..a6b3181 100644
--- a/airflow/contrib/hooks/ftp_hook.py
+++ b/airflow/contrib/hooks/ftp_hook.py
@@ -15,11 +15,12 @@
 
 import datetime
 import ftplib
-import logging
 import os.path
 from airflow.hooks.base_hook import BaseHook
 from past.builtins import basestring
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 
 def mlsd(conn, path="", facts=None):
     """
@@ -54,7 +55,7 @@ def mlsd(conn, path="", facts=None):
         yield (name, entry)
 
 
-class FTPHook(BaseHook):
+class FTPHook(BaseHook, LoggingMixin):
     """
     Interact with FTP.
 
@@ -166,10 +167,9 @@ class FTPHook(BaseHook):
 
         remote_path, remote_file_name = os.path.split(remote_full_path)
         conn.cwd(remote_path)
-        logging.info('Retrieving file from FTP: {}'.format(remote_full_path))
+        self.logger.info('Retrieving file from FTP: %s', remote_full_path)
         conn.retrbinary('RETR %s' % remote_file_name, output_handle.write)
-        logging.info('Finished retrieving file from FTP: {}'.format(
-            remote_full_path))
+        self.logger.info('Finished retrieving file from FTP: %s', remote_full_path)
 
         if is_path:
             output_handle.close()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/gcp_api_base_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcp_api_base_hook.py b/airflow/contrib/hooks/gcp_api_base_hook.py
index 48c5979..7476c90 100644
--- a/airflow/contrib/hooks/gcp_api_base_hook.py
+++ b/airflow/contrib/hooks/gcp_api_base_hook.py
@@ -12,18 +12,16 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
-import json
-
 import httplib2
 from oauth2client.client import GoogleCredentials
 from oauth2client.service_account import ServiceAccountCredentials
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-class GoogleCloudBaseHook(BaseHook):
+
+class GoogleCloudBaseHook(BaseHook, LoggingMixin):
     """
     A base hook for Google cloud-related hooks. Google cloud has a shared REST
     API client that is built in the same way no matter which service you use.
@@ -43,7 +41,6 @@ class GoogleCloudBaseHook(BaseHook):
 
     Legacy P12 key files are not supported.
     """
-
     def __init__(self, conn_id, delegate_to=None):
         """
         :param conn_id: The connection ID to use when fetching connection info.
@@ -69,7 +66,7 @@ class GoogleCloudBaseHook(BaseHook):
             kwargs['sub'] = self.delegate_to
 
         if not key_path:
-            logging.info('Getting connection using `gcloud auth` user, since no key file '
+            self.logger.info('Getting connection using `gcloud auth` user, since no key file '
                          'is defined for hook.')
             credentials = GoogleCredentials.get_application_default()
         else:
@@ -77,7 +74,7 @@ class GoogleCloudBaseHook(BaseHook):
                 raise AirflowException('Scope should be defined when using a key file.')
             scopes = [s.strip() for s in scope.split(',')]
             if key_path.endswith('.json'):
-                logging.info('Getting connection using a JSON key file.')
+                self.logger.info('Getting connection using a JSON key file.')
                 credentials = ServiceAccountCredentials\
                     .from_json_keyfile_name(key_path, scopes)
             elif key_path.endswith('.p12'):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/gcp_dataflow_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcp_dataflow_hook.py b/airflow/contrib/hooks/gcp_dataflow_hook.py
index fc73288..66dfb07 100644
--- a/airflow/contrib/hooks/gcp_dataflow_hook.py
+++ b/airflow/contrib/hooks/gcp_dataflow_hook.py
@@ -11,8 +11,6 @@
 # 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
 import select
 import subprocess
 import time
@@ -21,10 +19,10 @@ import uuid
 from apiclient.discovery import build
 
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class _DataflowJob(object):
-
+class _DataflowJob(LoggingMixin):
     def __init__(self, dataflow, project_number, name):
         self._dataflow = dataflow
         self._project_number = project_number
@@ -49,11 +47,15 @@ class _DataflowJob(object):
             job = self._dataflow.projects().jobs().get(projectId=self._project_number,
                                                        jobId=self._job_id).execute()
         if 'currentState' in job:
-            logging.info('Google Cloud DataFlow job %s is %s', job['name'],
-                         job['currentState'])
+            self.logger.info(
+                'Google Cloud DataFlow job %s is %s',
+                job['name'], job['currentState']
+            )
         else:
-            logging.info('Google Cloud DataFlow with job_id %s has name %s', self._job_id,
-                         job['name'])
+            self.logger.info(
+                'Google Cloud DataFlow with job_id %s has name %s',
+                self._job_id, job['name']
+            )
         return job
 
     def wait_for_done(self):
@@ -70,7 +72,7 @@ class _DataflowJob(object):
                 elif 'JOB_STATE_RUNNING' == self._job['currentState']:
                     time.sleep(10)
                 else:
-                    logging.debug(str(self._job))
+                    self.logger.debug(str(self._job))
                     raise Exception(
                         "Google Cloud Dataflow job {} was unknown state: {}".format(
                             self._job['name'], self._job['currentState']))
@@ -83,8 +85,7 @@ class _DataflowJob(object):
         return self._job
 
 
-class _Dataflow(object):
-
+class _Dataflow(LoggingMixin):
     def __init__(self, cmd):
         self._proc = subprocess.Popen(cmd, shell=False, stdout=subprocess.PIPE,
                                       stderr=subprocess.PIPE)
@@ -105,15 +106,15 @@ class _Dataflow(object):
 
     def wait_for_done(self):
         reads = [self._proc.stderr.fileno(), self._proc.stdout.fileno()]
-        logging.info("Start waiting for DataFlow process to complete.")
+        self.logger.info("Start waiting for DataFlow process to complete.")
         while self._proc.poll() is None:
             ret = select.select(reads, [], [], 5)
             if ret is not None:
                 for fd in ret[0]:
                     line = self._line(fd)
-                    logging.debug(line[:-1])
+                    self.logger.debug(line[:-1])
             else:
-                logging.info("Waiting for DataFlow process to complete.")
+                self.logger.info("Waiting for DataFlow process to complete.")
         if self._proc.returncode is not 0:
             raise Exception("DataFlow failed with return code {}".format(
                 self._proc.returncode))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/gcp_dataproc_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcp_dataproc_hook.py b/airflow/contrib/hooks/gcp_dataproc_hook.py
index c1d8993..3a1336e 100644
--- a/airflow/contrib/hooks/gcp_dataproc_hook.py
+++ b/airflow/contrib/hooks/gcp_dataproc_hook.py
@@ -12,16 +12,16 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-import logging
 import time
 import uuid
 
 from apiclient.discovery import build
 
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class _DataProcJob:
+class _DataProcJob(LoggingMixin):
     def __init__(self, dataproc_api, project_id, job):
         self.dataproc_api = dataproc_api
         self.project_id = project_id
@@ -30,8 +30,10 @@ class _DataProcJob:
             region='global',
             body=job).execute()
         self.job_id = self.job['reference']['jobId']
-        logging.info('DataProc job %s is %s', self.job_id,
-                     str(self.job['status']['state']))
+        self.logger.info(
+            'DataProc job %s is %s',
+            self.job_id, str(self.job['status']['state'])
+        )
 
     def wait_for_done(self):
         while True:
@@ -41,21 +43,23 @@ class _DataProcJob:
                 jobId=self.job_id).execute()
             if 'ERROR' == self.job['status']['state']:
                 print(str(self.job))
-                logging.error('DataProc job %s has errors', self.job_id)
-                logging.error(self.job['status']['details'])
-                logging.debug(str(self.job))
+                self.logger.error('DataProc job %s has errors', self.job_id)
+                self.logger.error(self.job['status']['details'])
+                self.logger.debug(str(self.job))
                 return False
             if 'CANCELLED' == self.job['status']['state']:
                 print(str(self.job))
-                logging.warning('DataProc job %s is cancelled', self.job_id)
+                self.logger.warning('DataProc job %s is cancelled', self.job_id)
                 if 'details' in self.job['status']:
-                    logging.warning(self.job['status']['details'])
-                logging.debug(str(self.job))
+                    self.logger.warning(self.job['status']['details'])
+                self.logger.debug(str(self.job))
                 return False
             if 'DONE' == self.job['status']['state']:
                 return True
-            logging.debug('DataProc job %s is %s', self.job_id,
-                          str(self.job['status']['state']))
+            self.logger.debug(
+                'DataProc job %s is %s',
+                self.job_id, str(self.job['status']['state'])
+            )
             time.sleep(5)
 
     def raise_error(self, message=None):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/gcp_mlengine_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcp_mlengine_hook.py b/airflow/contrib/hooks/gcp_mlengine_hook.py
index 47d9700..35f31a7 100644
--- a/airflow/contrib/hooks/gcp_mlengine_hook.py
+++ b/airflow/contrib/hooks/gcp_mlengine_hook.py
@@ -13,44 +13,40 @@
 # 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
 import random
 import time
-from airflow import settings
-from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
-from apiclient.discovery import build
 from apiclient import errors
+from apiclient.discovery import build
 from oauth2client.client import GoogleCredentials
 
-logging.getLogger('GoogleCloudMLEngine').setLevel(settings.LOGGING_LEVEL)
+from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
 def _poll_with_exponential_delay(request, max_n, is_done_func, is_error_func):
+    log = LoggingMixin().logger
 
     for i in range(0, max_n):
         try:
             response = request.execute()
             if is_error_func(response):
                 raise ValueError(
-                    'The response contained an error: {}'.format(response))
+                    'The response contained an error: {}'.format(response)
+                )
             elif is_done_func(response):
-                logging.info('Operation is done: {}'.format(response))
+                log.info('Operation is done: %s', response)
                 return response
             else:
                 time.sleep((2**i) + (random.randint(0, 1000) / 1000))
         except errors.HttpError as e:
             if e.resp.status != 429:
-                logging.info(
-                    'Something went wrong. Not retrying: {}'.format(e))
+                log.info('Something went wrong. Not retrying: %s', format(e))
                 raise
             else:
                 time.sleep((2**i) + (random.randint(0, 1000) / 1000))
 
 
 class MLEngineHook(GoogleCloudBaseHook):
-
     def __init__(self, gcp_conn_id='google_cloud_default', delegate_to=None):
         super(MLEngineHook, self).__init__(gcp_conn_id, delegate_to)
         self._mlengine = self.get_conn()
@@ -107,17 +103,20 @@ class MLEngineHook(GoogleCloudBaseHook):
                 if use_existing_job_fn is not None:
                     existing_job = self._get_job(project_id, job_id)
                     if not use_existing_job_fn(existing_job):
-                        logging.error(
-                            'Job with job_id {} already exist, but it does '
-                            'not match our expectation: {}'.format(
-                                job_id, existing_job))
+                        self.logger.error(
+                            'Job with job_id %s already exist, but it does '
+                            'not match our expectation: %s',
+                            job_id, existing_job
+                        )
                         raise
-                logging.info(
-                    'Job with job_id {} already exist. Will waiting for it to '
-                    'finish'.format(job_id))
+                self.logger.info(
+                    'Job with job_id %s already exist. Will waiting for it to finish',
+                    job_id
+                )
             else:
-                logging.error('Failed to create MLEngine job: {}'.format(e))
+                self.logger.error('Failed to create MLEngine job: {}'.format(e))
                 raise
+
         return self._wait_for_job_done(project_id, job_id)
 
     def _get_job(self, project_id, job_id):
@@ -140,7 +139,7 @@ class MLEngineHook(GoogleCloudBaseHook):
                     # polling after 30 seconds when quota failure occurs
                     time.sleep(30)
                 else:
-                    logging.error('Failed to get MLEngine job: {}'.format(e))
+                    self.logger.error('Failed to get MLEngine job: {}'.format(e))
                     raise
 
     def _wait_for_job_done(self, project_id, job_id, interval=30):
@@ -192,11 +191,10 @@ class MLEngineHook(GoogleCloudBaseHook):
 
         try:
             response = request.execute()
-            logging.info(
-                'Successfully set version: {} to default'.format(response))
+            self.logger.info('Successfully set version: %s to default', response)
             return response
         except errors.HttpError as e:
-            logging.error('Something went wrong: {}'.format(e))
+            self.logger.error('Something went wrong: %s', e)
             raise
 
     def list_versions(self, project_id, model_name):
@@ -264,6 +262,6 @@ class MLEngineHook(GoogleCloudBaseHook):
             return request.execute()
         except errors.HttpError as e:
             if e.resp.status == 404:
-                logging.error('Model was not found: {}'.format(e))
+                self.logger.error('Model was not found: %s', e)
                 return None
             raise

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/gcs_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/gcs_hook.py b/airflow/contrib/hooks/gcs_hook.py
index b5f3edc..eb17c3b 100644
--- a/airflow/contrib/hooks/gcs_hook.py
+++ b/airflow/contrib/hooks/gcs_hook.py
@@ -12,17 +12,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
-
 from apiclient.discovery import build
 from apiclient.http import MediaFileUpload
 from googleapiclient import errors
 
 from airflow.contrib.hooks.gcp_api_base_hook import GoogleCloudBaseHook
 
-logging.getLogger("google_cloud_storage").setLevel(logging.INFO)
-
 
 class GoogleCloudStorageHook(GoogleCloudBaseHook):
     """
@@ -187,8 +182,7 @@ class GoogleCloudStorageHook(GoogleCloudBaseHook):
                     ts = ts.replace(tzinfo=dateutil.tz.tzutc())
 
                 updated = dateutil.parser.parse(response['updated'])
-                logging.log(logging.INFO, "Verify object date: " + str(updated)
-                            + " > " + str(ts))
+                self.logger.info("Verify object date: %s > %s", updated, ts)
 
                 if updated > ts:
                     return True
@@ -253,7 +247,7 @@ class GoogleCloudStorageHook(GoogleCloudBaseHook):
             ).execute()
 
             if 'items' not in response:
-                logging.info("No items found for prefix:{}".format(prefix))
+                self.logger.info("No items found for prefix: %s", prefix)
                 break
 
             for item in response['items']:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/jira_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/jira_hook.py b/airflow/contrib/hooks/jira_hook.py
index 148101b..8702608 100644
--- a/airflow/contrib/hooks/jira_hook.py
+++ b/airflow/contrib/hooks/jira_hook.py
@@ -11,24 +11,21 @@
 # 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 jira import JIRA
 from jira.exceptions import JIRAError
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class JiraHook(BaseHook):
+class JiraHook(BaseHook, LoggingMixin):
     """
     Jira interaction hook, a Wrapper around JIRA Python SDK.
 
     :param jira_conn_id: reference to a pre-defined Jira Connection
     :type jira_conn_id: string
     """
-
     def __init__(self,
                  jira_conn_id='jira_default'):
         super(JiraHook, self).__init__(jira_conn_id)
@@ -38,7 +35,7 @@ class JiraHook(BaseHook):
 
     def get_conn(self):
         if not self.client:
-            logging.debug('creating jira client for conn_id: {0}'.format(self.jira_conn_id))
+            self.logger.debug('Creating Jira client for conn_id: %s', self.jira_conn_id)
 
             get_server_info = True
             validate = True

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/qubole_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/qubole_hook.py b/airflow/contrib/hooks/qubole_hook.py
index c51a757..1a5e7ec 100755
--- a/airflow/contrib/hooks/qubole_hook.py
+++ b/airflow/contrib/hooks/qubole_hook.py
@@ -16,12 +16,12 @@
 import os
 import time
 import datetime
-import logging
 import six
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
 from airflow import configuration
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.utils.state import State
 
 from qds_sdk.qubole import Qubole
@@ -68,7 +68,7 @@ COMMAND_ARGS = {
 }
 
 
-class QuboleHook(BaseHook):
+class QuboleHook(BaseHook, LoggingMixin):
     def __init__(self, *args, **kwargs):
         conn = self.get_connection(kwargs['qubole_conn_id'])
         Qubole.configure(api_token=conn.password, api_url=conn.host)
@@ -84,31 +84,33 @@ class QuboleHook(BaseHook):
         cmd_id = ti.xcom_pull(key='qbol_cmd_id', task_ids=ti.task_id)
 
         if cmd_id is not None:
-            logger = logging.getLogger("QuboleHook")
             cmd = Command.find(cmd_id)
             if cmd is not None:
+                log = LoggingMixin().logger
                 if cmd.status == 'done':
-                    logger.info('Command ID: %s has been succeeded, hence marking this '
+                    log.info('Command ID: %s has been succeeded, hence marking this '
                                 'TI as Success.', cmd_id)
                     ti.state = State.SUCCESS
                 elif cmd.status == 'running':
-                    logger.info('Cancelling the Qubole Command Id: %s', cmd_id)
+                    log.info('Cancelling the Qubole Command Id: %s', cmd_id)
                     cmd.cancel()
 
     def execute(self, context):
         args = self.cls.parse(self.create_cmd_args(context))
         self.cmd = self.cls.create(**args)
         context['task_instance'].xcom_push(key='qbol_cmd_id', value=self.cmd.id)
-        logging.info("Qubole command created with Id: %s and Status: %s",
-                     self.cmd.id, self.cmd.status)
+        self.logger.info(
+            "Qubole command created with Id: %s and Status: %s",
+            self.cmd.id, self.cmd.status
+        )
 
         while not Command.is_done(self.cmd.status):
             time.sleep(Qubole.poll_interval)
             self.cmd = self.cls.find(self.cmd.id)
-            logging.info("Command Id: %s and Status: %s", self.cmd.id, self.cmd.status)
+            self.logger.info("Command Id: %s and Status: %s", self.cmd.id, self.cmd.status)
 
         if 'fetch_logs' in self.kwargs and self.kwargs['fetch_logs'] is True:
-            logging.info("Logs for Command Id: %s \n%s", self.cmd.id, self.cmd.get_log())
+            self.logger.info("Logs for Command Id: %s \n%s", self.cmd.id, self.cmd.get_log())
 
         if self.cmd.status != 'done':
             raise AirflowException('Command Id: {0} failed with Status: {1}'.format(
@@ -124,7 +126,7 @@ class QuboleHook(BaseHook):
             cmd_id = ti.xcom_pull(key="qbol_cmd_id", task_ids=ti.task_id)
             self.cmd = self.cls.find(cmd_id)
         if self.cls and self.cmd:
-            logging.info('Sending KILL signal to Qubole Command Id: %s', self.cmd.id)
+            self.logger.info('Sending KILL signal to Qubole Command Id: %s', self.cmd.id)
             self.cmd.cancel()
 
     def get_results(self, ti=None, fp=None, inline=True, delim=None, fetch=True):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/redis_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/redis_hook.py b/airflow/contrib/hooks/redis_hook.py
index 936eff8..a8999d6 100644
--- a/airflow/contrib/hooks/redis_hook.py
+++ b/airflow/contrib/hooks/redis_hook.py
@@ -15,16 +15,14 @@
 """
 RedisHook module
 """
-
-import logging
-
 from redis import StrictRedis
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class RedisHook(BaseHook):
+class RedisHook(BaseHook, LoggingMixin):
     """
     Hook to interact with Redis database
     """
@@ -42,7 +40,7 @@ class RedisHook(BaseHook):
         self.port = int(conn.port)
         self.password = conn.password
         self.db = int(conn.extra_dejson.get('db', 0))
-        self.logger = logging.getLogger(__name__)
+
         self.logger.debug(
             '''Connection "{conn}":
             \thost: {host}
@@ -62,11 +60,9 @@ class RedisHook(BaseHook):
         """
         if not self.client:
             self.logger.debug(
-                'generating Redis client for conn_id "{conn}" on '
-                '{host}:{port}:{db}'.format(conn=self.redis_conn_id,
-                                            host=self.host,
-                                            port=self.port,
-                                            db=self.db))
+                'generating Redis client for conn_id "%s" on %s:%s:%s',
+                self.redis_conn_id, self.host, self.port, self.db
+            )
             try:
                 self.client = StrictRedis(
                     host=self.host,


[4/5] incubator-airflow git commit: [AIRFLOW-1582] Improve logging within Airflow

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/salesforce_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/salesforce_hook.py b/airflow/contrib/hooks/salesforce_hook.py
index 67d1605..f2b5fef 100644
--- a/airflow/contrib/hooks/salesforce_hook.py
+++ b/airflow/contrib/hooks/salesforce_hook.py
@@ -24,14 +24,15 @@ NOTE:   this hook also relies on the simple_salesforce package:
 from simple_salesforce import Salesforce
 from airflow.hooks.base_hook import BaseHook
 
-import logging
 import json
 
 import pandas as pd
 import time
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-class SalesforceHook(BaseHook):
+
+class SalesforceHook(BaseHook, LoggingMixin):
     def __init__(
             self,
             conn_id,
@@ -91,13 +92,12 @@ class SalesforceHook(BaseHook):
         """
         self.sign_in()
 
-        logging.info("Querying for all objects")
+        self.logger.info("Querying for all objects")
         query = self.sf.query_all(query)
 
-        logging.info(
-            "Received results: Total size: {0}; Done: {1}".format(
-                query['totalSize'], query['done']
-            )
+        self.logger.info(
+            "Received results: Total size: %s; Done: %s",
+            query['totalSize'], query['done']
         )
 
         query = json.loads(json.dumps(query))
@@ -144,11 +144,9 @@ class SalesforceHook(BaseHook):
         field_string = self._build_field_list(fields)
 
         query = "SELECT {0} FROM {1}".format(field_string, obj)
-        logging.info(
-            "Making query to salesforce: {0}".format(
-                query if len(query) < 30
-                else " ... ".join([query[:15], query[-15:]])
-            )
+        self.logger.info(
+            "Making query to Salesforce: %s",
+            query if len(query) < 30 else " ... ".join([query[:15], query[-15:]])
         )
         return self.make_query(query)
 
@@ -171,8 +169,9 @@ class SalesforceHook(BaseHook):
         try:
             col = pd.to_datetime(col)
         except ValueError:
-            logging.warning(
-                "Could not convert field to timestamps: {0}".format(col.name)
+            log = LoggingMixin().logger
+            log.warning(
+                "Could not convert field to timestamps: %s", col.name
             )
             return col
 
@@ -266,7 +265,7 @@ class SalesforceHook(BaseHook):
             # for each returned record
             object_name = query_results[0]['attributes']['type']
 
-            logging.info("Coercing timestamps for: {0}".format(object_name))
+            self.logger.info("Coercing timestamps for: %s", object_name)
 
             schema = self.describe_object(object_name)
 
@@ -300,7 +299,7 @@ class SalesforceHook(BaseHook):
             # there are also a ton of newline objects
             # that mess up our ability to write to csv
             # we remove these newlines so that the output is a valid CSV format
-            logging.info("Cleaning data and writing to CSV")
+            self.logger.info("Cleaning data and writing to CSV")
             possible_strings = df.columns[df.dtypes == "object"]
             df[possible_strings] = df[possible_strings].apply(
                 lambda x: x.str.replace("\r\n", "")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/spark_sql_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/spark_sql_hook.py b/airflow/contrib/hooks/spark_sql_hook.py
index d7bef7b..aa16130 100644
--- a/airflow/contrib/hooks/spark_sql_hook.py
+++ b/airflow/contrib/hooks/spark_sql_hook.py
@@ -12,16 +12,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-import logging
 import subprocess
 
 from airflow.hooks.base_hook import BaseHook
 from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-log = logging.getLogger(__name__)
 
-
-class SparkSqlHook(BaseHook):
+class SparkSqlHook(BaseHook, LoggingMixin):
     """
     This hook is a wrapper around the spark-sql binary. It requires that the
     "spark-sql" binary is in the PATH.
@@ -123,7 +121,7 @@ class SparkSqlHook(BaseHook):
             connection_cmd += ["--queue", self._yarn_queue]
 
         connection_cmd += cmd
-        logging.debug("Spark-Sql cmd: {}".format(connection_cmd))
+        self.logger.debug("Spark-Sql cmd: %s", connection_cmd)
 
         return connection_cmd
 
@@ -153,5 +151,5 @@ class SparkSqlHook(BaseHook):
 
     def kill(self):
         if self._sp and self._sp.poll() is None:
-            logging.info("Killing the Spark-Sql job")
+            self.logger.info("Killing the Spark-Sql job")
             self._sp.kill()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/spark_submit_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/spark_submit_hook.py b/airflow/contrib/hooks/spark_submit_hook.py
index a667753..bdd1efe 100644
--- a/airflow/contrib/hooks/spark_submit_hook.py
+++ b/airflow/contrib/hooks/spark_submit_hook.py
@@ -12,18 +12,16 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-import logging
 import os
 import subprocess
 import re
 
 from airflow.hooks.base_hook import BaseHook
 from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-log = logging.getLogger(__name__)
 
-
-class SparkSubmitHook(BaseHook):
+class SparkSubmitHook(BaseHook, LoggingMixin):
     """
     This hook is a wrapper around the spark-submit binary to kick off a spark-submit job.
     It requires that the "spark-submit" binary is in the PATH or the spark_home to be
@@ -63,7 +61,6 @@ class SparkSubmitHook(BaseHook):
     :param verbose: Whether to pass the verbose flag to spark-submit process for debugging
     :type verbose: bool
     """
-
     def __init__(self,
                  conf=None,
                  conn_id='spark_default',
@@ -126,10 +123,9 @@ class SparkSubmitHook(BaseHook):
             conn_data['spark_home'] = extra.get('spark-home', None)
             conn_data['spark_binary'] = extra.get('spark-binary', 'spark-submit')
         except AirflowException:
-            logging.debug(
-                "Could not load connection string {}, defaulting to {}".format(
-                    self._conn_id, conn_data['master']
-                )
+            self.logger.debug(
+                "Could not load connection string %s, defaulting to %s",
+                self._conn_id, conn_data['master']
             )
 
         return conn_data
@@ -196,7 +192,7 @@ class SparkSubmitHook(BaseHook):
         if self._application_args:
             connection_cmd += self._application_args
 
-        logging.debug("Spark-Submit cmd: {}".format(connection_cmd))
+        self.logger.debug("Spark-Submit cmd: %s", connection_cmd)
 
         return connection_cmd
 
@@ -243,15 +239,15 @@ class SparkSubmitHook(BaseHook):
                     self._yarn_application_id = match.groups()[0]
 
             # Pass to logging
-            logging.info(line)
+            self.logger.info(line)
 
     def on_kill(self):
         if self._sp and self._sp.poll() is None:
-            logging.info('Sending kill signal to {}'.format(self._connection['spark_binary']))
+            self.logger.info('Sending kill signal to %s', self._connection['spark_binary'])
             self._sp.kill()
 
             if self._yarn_application_id:
-                logging.info('Killing application on YARN')
+                self.logger.info('Killing application on YARN')
                 kill_cmd = "yarn application -kill {0}".format(self._yarn_application_id).split()
                 yarn_kill = subprocess.Popen(kill_cmd, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
-                logging.info("YARN killed with return code: {0}".format(yarn_kill.wait()))
+                self.logger.info("YARN killed with return code: %s", yarn_kill.wait())

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/sqoop_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/sqoop_hook.py b/airflow/contrib/hooks/sqoop_hook.py
index 7fbb6c5..0584df4 100644
--- a/airflow/contrib/hooks/sqoop_hook.py
+++ b/airflow/contrib/hooks/sqoop_hook.py
@@ -16,17 +16,14 @@
 """
 This module contains a sqoop 1.x hook
 """
-
-import logging
 import subprocess
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
-
-log = logging.getLogger(__name__)
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class SqoopHook(BaseHook):
+class SqoopHook(BaseHook, LoggingMixin):
     """
     This hook is a wrapper around the sqoop 1 binary. To be able to use the hook
     it is required that "sqoop" is in the PATH.
@@ -79,7 +76,7 @@ class SqoopHook(BaseHook):
             password_index = cmd.index('--password')
             cmd[password_index + 1] = 'MASKED'
         except ValueError:
-            logging.debug("No password in sqoop cmd")
+            self.logger.debug("No password in sqoop cmd")
         return cmd
 
     def Popen(self, cmd, **kwargs):
@@ -90,21 +87,21 @@ class SqoopHook(BaseHook):
         :param kwargs: extra arguments to Popen (see subprocess.Popen)
         :return: handle to subprocess
         """
-        logging.info("Executing command: {}".format(' '.join(cmd)))
+        self.logger.info("Executing command: %s", ' '.join(cmd))
         sp = subprocess.Popen(cmd,
                               stdout=subprocess.PIPE,
                               stderr=subprocess.STDOUT,
                               **kwargs)
 
         for line in iter(sp.stdout):
-            logging.info(line.strip())
+            self.logger.info(line.strip())
 
         sp.wait()
 
-        logging.info("Command exited with return code {0}".format(sp.returncode))
+        self.logger.info("Command exited with return code %s", sp.returncode)
 
         if sp.returncode:
-            raise AirflowException("Sqoop command failed: {}".format(' '.join(cmd)))
+            raise AirflowException("Sqoop command failed: %s", ' '.join(cmd))
 
     def _prepare_command(self, export=False):
         if export:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/hooks/ssh_hook.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/hooks/ssh_hook.py b/airflow/contrib/hooks/ssh_hook.py
index f1e25a6..3fe9146 100755
--- a/airflow/contrib/hooks/ssh_hook.py
+++ b/airflow/contrib/hooks/ssh_hook.py
@@ -16,7 +16,6 @@
 # limitations under the License.
 
 import getpass
-import logging
 import os
 
 import paramiko
@@ -24,9 +23,10 @@ import paramiko
 from contextlib import contextmanager
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 
-class SSHHook(BaseHook):
+class SSHHook(BaseHook, LoggingMixin):
     """
     Hook for ssh remote execution using Paramiko.
     ref: https://github.com/paramiko/paramiko
@@ -70,7 +70,7 @@ class SSHHook(BaseHook):
 
     def get_conn(self):
         if not self.client:
-            logging.debug('creating ssh client for conn_id: {0}'.format(self.ssh_conn_id))
+            self.logger.debug('Creating SSH client for conn_id: %s', self.ssh_conn_id)
             if self.ssh_conn_id is not None:
                 conn = self.get_connection(self.ssh_conn_id)
                 if self.username is None:
@@ -98,9 +98,11 @@ class SSHHook(BaseHook):
 
             # Auto detecting username values from system
             if not self.username:
-                logging.debug("username to ssh to host: {0} is not specified, using "
-                             "system's default provided by getpass.getuser()"
-                             .format(self.remote_host, self.ssh_conn_id))
+                self.logger.debug(
+                    "username to ssh to host: %s is not specified for connection id"
+                    " %s. Using system's default provided by getpass.getuser()",
+                    self.remote_host, self.ssh_conn_id
+                )
                 self.username = getpass.getuser()
 
             host_proxy = None
@@ -140,14 +142,20 @@ class SSHHook(BaseHook):
 
                 self.client = client
             except paramiko.AuthenticationException as auth_error:
-                logging.error("Auth failed while connecting to host: {0}, error: {1}"
-                              .format(self.remote_host, auth_error))
+                self.logger.error(
+                    "Auth failed while connecting to host: %s, error: %s",
+                    self.remote_host, auth_error
+                )
             except paramiko.SSHException as ssh_error:
-                logging.error("Failed connecting to host: {0}, error: {1}"
-                              .format(self.remote_host, ssh_error))
+                self.logger.error(
+                    "Failed connecting to host: %s, error: %s",
+                    self.remote_host, ssh_error
+                )
             except Exception as error:
-                logging.error("Error connecting to host: {0}, error: {1}"
-                              .format(self.remote_host, error))
+                self.logger.error(
+                    "Error connecting to host: %s, error: %s",
+                    self.remote_host, error
+                )
         return self.client
 
     @contextmanager
@@ -183,7 +191,7 @@ class SSHHook(BaseHook):
                           ]
 
         ssh_cmd += ssh_tunnel_cmd
-        logging.debug("creating tunnel with cmd: {0}".format(ssh_cmd))
+        self.logger.debug("Creating tunnel with cmd: %s", ssh_cmd)
 
         proc = subprocess.Popen(ssh_cmd,
                                 stdin=subprocess.PIPE,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/bigquery_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/bigquery_operator.py b/airflow/contrib/operators/bigquery_operator.py
index 3b804a8..37e4a97 100644
--- a/airflow/contrib/operators/bigquery_operator.py
+++ b/airflow/contrib/operators/bigquery_operator.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -89,7 +87,7 @@ class BigQueryOperator(BaseOperator):
         self.query_params = query_params
 
     def execute(self, context):
-        logging.info('Executing: %s', self.bql)
+        self.logger.info('Executing: %s', self.bql)
         hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,
                             delegate_to=self.delegate_to)
         conn = hook.get_conn()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/bigquery_table_delete_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/bigquery_table_delete_operator.py b/airflow/contrib/operators/bigquery_table_delete_operator.py
index 643f5ac..21de7cc 100644
--- a/airflow/contrib/operators/bigquery_table_delete_operator.py
+++ b/airflow/contrib/operators/bigquery_table_delete_operator.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -55,7 +53,7 @@ class BigQueryTableDeleteOperator(BaseOperator):
         self.ignore_if_missing = ignore_if_missing
 
     def execute(self, context):
-        logging.info('Deleting: %s', self.deletion_dataset_table)
+        self.logger.info('Deleting: %s', self.deletion_dataset_table)
         hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,
                             delegate_to=self.delegate_to)
         conn = hook.get_conn()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/bigquery_to_bigquery.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/bigquery_to_bigquery.py b/airflow/contrib/operators/bigquery_to_bigquery.py
index 6f4843c..8e21270 100644
--- a/airflow/contrib/operators/bigquery_to_bigquery.py
+++ b/airflow/contrib/operators/bigquery_to_bigquery.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -70,8 +68,10 @@ class BigQueryToBigQueryOperator(BaseOperator):
         self.delegate_to = delegate_to
 
     def execute(self, context):
-        logging.info('Executing copy of %s into: %s', self.source_project_dataset_tables,
-                     self.destination_project_dataset_table)
+        self.logger.info(
+            'Executing copy of %s into: %s',
+            self.source_project_dataset_tables, self.destination_project_dataset_table
+        )
         hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,
                             delegate_to=self.delegate_to)
         conn = hook.get_conn()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/bigquery_to_gcs.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/bigquery_to_gcs.py b/airflow/contrib/operators/bigquery_to_gcs.py
index aaff462..23a2029 100644
--- a/airflow/contrib/operators/bigquery_to_gcs.py
+++ b/airflow/contrib/operators/bigquery_to_gcs.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -81,7 +79,7 @@ class BigQueryToCloudStorageOperator(BaseOperator):
         self.delegate_to = delegate_to
 
     def execute(self, context):
-        logging.info('Executing extract of %s into: %s',
+        self.logger.info('Executing extract of %s into: %s',
                      self.source_project_dataset_table,
                      self.destination_cloud_storage_uris)
         hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/databricks_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/databricks_operator.py b/airflow/contrib/operators/databricks_operator.py
index 1aa1441..8773357 100644
--- a/airflow/contrib/operators/databricks_operator.py
+++ b/airflow/contrib/operators/databricks_operator.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 #
 
-import logging
 import six
 import time
 
@@ -21,8 +20,6 @@ from airflow.exceptions import AirflowException
 from airflow.contrib.hooks.databricks_hook import DatabricksHook
 from airflow.models import BaseOperator
 
-LINE_BREAK = ('-' * 80)
-
 
 class DatabricksSubmitRunOperator(BaseOperator):
     """
@@ -217,7 +214,7 @@ class DatabricksSubmitRunOperator(BaseOperator):
             raise AirflowException(msg)
 
     def _log_run_page_url(self, url):
-        logging.info('View run status, Spark UI, and logs at {}'.format(url))
+        self.logger.info('View run status, Spark UI, and logs at %s', url)
 
     def get_hook(self):
         return DatabricksHook(
@@ -228,16 +225,13 @@ class DatabricksSubmitRunOperator(BaseOperator):
         hook = self.get_hook()
         self.run_id = hook.submit_run(self.json)
         run_page_url = hook.get_run_page_url(self.run_id)
-        logging.info(LINE_BREAK)
-        logging.info('Run submitted with run_id: {}'.format(self.run_id))
+        self.logger.info('Run submitted with run_id: %s', self.run_id)
         self._log_run_page_url(run_page_url)
-        logging.info(LINE_BREAK)
         while True:
             run_state = hook.get_run_state(self.run_id)
             if run_state.is_terminal:
                 if run_state.is_successful:
-                    logging.info('{} completed successfully.'.format(
-                        self.task_id))
+                    self.logger.info('%s completed successfully.', self.task_id)
                     self._log_run_page_url(run_page_url)
                     return
                 else:
@@ -246,16 +240,15 @@ class DatabricksSubmitRunOperator(BaseOperator):
                         s=run_state)
                     raise AirflowException(error_message)
             else:
-                logging.info('{t} in run state: {s}'.format(t=self.task_id,
-                                                            s=run_state))
+                self.logger.info('%s in run state: %s', self.task_id, run_state)
                 self._log_run_page_url(run_page_url)
-                logging.info('Sleeping for {} seconds.'.format(
-                    self.polling_period_seconds))
+                self.logger.info('Sleeping for %s seconds.', self.polling_period_seconds)
                 time.sleep(self.polling_period_seconds)
 
     def on_kill(self):
         hook = self.get_hook()
         hook.cancel_run(self.run_id)
-        logging.info('Task: {t} with run_id: {r} was requested to be cancelled.'.format(
-            t=self.task_id,
-            r=self.run_id))
+        self.logger.info(
+            'Task: %s with run_id: %s was requested to be cancelled.',
+            self.task_id, self.run_id
+        )

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/dataproc_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/dataproc_operator.py b/airflow/contrib/operators/dataproc_operator.py
index c0ff6a7..3c22b60 100644
--- a/airflow/contrib/operators/dataproc_operator.py
+++ b/airflow/contrib/operators/dataproc_operator.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 #
 
-import logging
 import time
 
 from airflow.contrib.hooks.gcp_dataproc_hook import DataProcHook
@@ -178,13 +177,14 @@ class DataprocClusterCreateOperator(BaseOperator):
         while True:
             state = self._get_cluster_state(service)
             if state is None:
-                logging.info("No state for cluster '%s'", self.cluster_name)
+                self.logger.info("No state for cluster '%s'", self.cluster_name)
                 time.sleep(15)
             else:
-                logging.info("State for cluster '%s' is %s", self.cluster_name, state)
+                self.logger.info("State for cluster '%s' is %s", self.cluster_name, state)
                 if self._cluster_ready(state, service):
-                    logging.info("Cluster '%s' successfully created",
-                                 self.cluster_name)
+                    self.logger.info(
+                        "Cluster '%s' successfully created", self.cluster_name
+                    )
                     return
                 time.sleep(15)
 
@@ -264,7 +264,7 @@ class DataprocClusterCreateOperator(BaseOperator):
         return cluster_data
 
     def execute(self, context):
-        logging.info('Creating cluster: {}'.format(self.cluster_name))
+        self.logger.info('Creating cluster: %s', self.cluster_name)
         hook = DataProcHook(
             gcp_conn_id=self.google_cloud_conn_id,
             delegate_to=self.delegate_to
@@ -272,9 +272,10 @@ class DataprocClusterCreateOperator(BaseOperator):
         service = hook.get_conn()
 
         if self._get_cluster(service):
-            logging.info('Cluster {} already exists... Checking status...'.format(
-                            self.cluster_name
-                        ))
+            self.logger.info(
+                'Cluster %s already exists... Checking status...',
+                self.cluster_name
+            )
             self._wait_for_done(service)
             return True
 
@@ -289,9 +290,10 @@ class DataprocClusterCreateOperator(BaseOperator):
             # probably two cluster start commands at the same time
             time.sleep(10)
             if self._get_cluster(service):
-                logging.info('Cluster {} already exists... Checking status...'.format(
-                             self.cluster_name
-                             ))
+                self.logger.info(
+                    'Cluster {} already exists... Checking status...',
+                    self.cluster_name
+                 )
                 self._wait_for_done(service)
                 return True
             else:
@@ -356,7 +358,7 @@ class DataprocClusterDeleteOperator(BaseOperator):
             time.sleep(15)
 
     def execute(self, context):
-        logging.info('Deleting cluster: {}'.format(self.cluster_name))
+        self.logger.info('Deleting cluster: %s', self.cluster_name)
         hook = DataProcHook(
             gcp_conn_id=self.google_cloud_conn_id,
             delegate_to=self.delegate_to
@@ -369,7 +371,7 @@ class DataprocClusterDeleteOperator(BaseOperator):
             clusterName=self.cluster_name
         ).execute()
         operation_name = response['name']
-        logging.info("Cluster delete operation name: {}".format(operation_name))
+        self.logger.info("Cluster delete operation name: %s", operation_name)
         self._wait_for_done(service, operation_name)
 
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/datastore_export_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/datastore_export_operator.py b/airflow/contrib/operators/datastore_export_operator.py
index 1980dfe..76415e1 100644
--- a/airflow/contrib/operators/datastore_export_operator.py
+++ b/airflow/contrib/operators/datastore_export_operator.py
@@ -12,13 +12,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
 from airflow.contrib.hooks.datastore_hook import DatastoreHook
 from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 
+
 class DatastoreExportOperator(BaseOperator):
     """
     Export entities from Google Cloud Datastore to Cloud Storage
@@ -79,7 +78,7 @@ class DatastoreExportOperator(BaseOperator):
         self.xcom_push = xcom_push
 
     def execute(self, context):
-        logging.info('Exporting data to Cloud Storage bucket ' + self.bucket)
+        self.logger.info('Exporting data to Cloud Storage bucket ' + self.bucket)
 
         if self.overwrite_existing and self.namespace:
             gcs_hook = GoogleCloudStorageHook(self.cloud_storage_conn_id)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/datastore_import_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/datastore_import_operator.py b/airflow/contrib/operators/datastore_import_operator.py
index 3427ba5..74bd940 100644
--- a/airflow/contrib/operators/datastore_import_operator.py
+++ b/airflow/contrib/operators/datastore_import_operator.py
@@ -12,13 +12,11 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
-
 from airflow.contrib.hooks.datastore_hook import DatastoreHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 
+
 class DatastoreImportOperator(BaseOperator):
     """
     Import entities from Cloud Storage to Google Cloud Datastore
@@ -74,7 +72,7 @@ class DatastoreImportOperator(BaseOperator):
         self.xcom_push = xcom_push
 
     def execute(self, context):
-        logging.info('Importing data from Cloud Storage bucket ' + self.bucket)
+        self.logger.info('Importing data from Cloud Storage bucket %s', self.bucket)
         ds_hook = DatastoreHook(self.datastore_conn_id, self.delegate_to)
         result = ds_hook.import_from_storage_bucket(bucket=self.bucket,
                                                     file=self.file,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/ecs_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/ecs_operator.py b/airflow/contrib/operators/ecs_operator.py
index 11f8c94..0c75eaa 100644
--- a/airflow/contrib/operators/ecs_operator.py
+++ b/airflow/contrib/operators/ecs_operator.py
@@ -12,7 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 import sys
-import logging
 
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
@@ -57,12 +56,11 @@ class ECSOperator(BaseOperator):
         self.hook = self.get_hook()
 
     def execute(self, context):
-
-        logging.info('Running ECS Task - Task definition: {} - on cluster {}'.format(
-            self.task_definition,
-            self.cluster
-        ))
-        logging.info('ECSOperator overrides: {}'.format(self.overrides))
+        self.logger.info(
+            'Running ECS Task - Task definition: %s - on cluster %s',
+            self.task_definition,self.cluster
+        )
+        self.logger.info('ECSOperator overrides: %s', self.overrides)
 
         self.client = self.hook.get_client_type(
             'ecs',
@@ -77,15 +75,15 @@ class ECSOperator(BaseOperator):
         )
 
         failures = response['failures']
-        if (len(failures) > 0):
+        if len(failures) > 0:
             raise AirflowException(response)
-        logging.info('ECS Task started: {}'.format(response))
+        self.logger.info('ECS Task started: %s', response)
 
         self.arn = response['tasks'][0]['taskArn']
         self._wait_for_task_ended()
 
         self._check_success_task()
-        logging.info('ECS Task has been successfully executed: {}'.format(response))
+        self.logger.info('ECS Task has been successfully executed: %s', response)
 
     def _wait_for_task_ended(self):
         waiter = self.client.get_waiter('tasks_stopped')
@@ -100,9 +98,9 @@ class ECSOperator(BaseOperator):
             cluster=self.cluster,
             tasks=[self.arn]
         )
-        logging.info('ECS Task stopped, check status: {}'.format(response))
+        self.logger.info('ECS Task stopped, check status: %s', response)
 
-        if (len(response.get('failures', [])) > 0):
+        if len(response.get('failures', [])) > 0:
             raise AirflowException(response)
 
         for task in response['tasks']:
@@ -126,4 +124,4 @@ class ECSOperator(BaseOperator):
             cluster=self.cluster,
             task=self.arn,
             reason='Task killed by the user')
-        logging.info(response)
+        self.logger.info(response)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/emr_add_steps_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/emr_add_steps_operator.py b/airflow/contrib/operators/emr_add_steps_operator.py
index 84ef2d0..dbf764e 100644
--- a/airflow/contrib/operators/emr_add_steps_operator.py
+++ b/airflow/contrib/operators/emr_add_steps_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.models import BaseOperator
 from airflow.utils import apply_defaults
 from airflow.exceptions import AirflowException
@@ -51,11 +48,11 @@ class EmrAddStepsOperator(BaseOperator):
     def execute(self, context):
         emr = EmrHook(aws_conn_id=self.aws_conn_id).get_conn()
 
-        logging.info('Adding steps to %s', self.job_flow_id)
+        self.logger.info('Adding steps to %s', self.job_flow_id)
         response = emr.add_job_flow_steps(JobFlowId=self.job_flow_id, Steps=self.steps)
 
         if not response['ResponseMetadata']['HTTPStatusCode'] == 200:
             raise AirflowException('Adding steps failed: %s' % response)
         else:
-            logging.info('Steps %s added to JobFlow', response['StepIds'])
+            self.logger.info('Steps %s added to JobFlow', response['StepIds'])
             return response['StepIds']

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/emr_create_job_flow_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/emr_create_job_flow_operator.py b/airflow/contrib/operators/emr_create_job_flow_operator.py
index 37d885d..4e40b17 100644
--- a/airflow/contrib/operators/emr_create_job_flow_operator.py
+++ b/airflow/contrib/operators/emr_create_job_flow_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.emr_hook import EmrHook
 from airflow.models import BaseOperator
 from airflow.utils import apply_defaults
@@ -53,11 +50,14 @@ class EmrCreateJobFlowOperator(BaseOperator):
     def execute(self, context):
         emr = EmrHook(aws_conn_id=self.aws_conn_id, emr_conn_id=self.emr_conn_id)
 
-        logging.info('Creating JobFlow')
+        self.logger.info(
+            'Creating JobFlow using aws-conn-id: %s, emr-conn-id: %s',
+            self.aws_conn_id, self.emr_conn_id
+        )
         response = emr.create_job_flow(self.job_flow_overrides)
 
         if not response['ResponseMetadata']['HTTPStatusCode'] == 200:
             raise AirflowException('JobFlow creation failed: %s' % response)
         else:
-            logging.info('JobFlow with id %s created', response['JobFlowId'])
+            self.logger.info('JobFlow with id %s created', response['JobFlowId'])
             return response['JobFlowId']

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/emr_terminate_job_flow_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/emr_terminate_job_flow_operator.py b/airflow/contrib/operators/emr_terminate_job_flow_operator.py
index 1b57276..df641ad 100644
--- a/airflow/contrib/operators/emr_terminate_job_flow_operator.py
+++ b/airflow/contrib/operators/emr_terminate_job_flow_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.models import BaseOperator
 from airflow.utils import apply_defaults
 from airflow.exceptions import AirflowException
@@ -46,10 +43,10 @@ class EmrTerminateJobFlowOperator(BaseOperator):
     def execute(self, context):
         emr = EmrHook(aws_conn_id=self.aws_conn_id).get_conn()
 
-        logging.info('Terminating JobFlow %s', self.job_flow_id)
+        self.logger.info('Terminating JobFlow %s', self.job_flow_id)
         response = emr.terminate_job_flows(JobFlowIds=[self.job_flow_id])
 
         if not response['ResponseMetadata']['HTTPStatusCode'] == 200:
             raise AirflowException('JobFlow termination failed: %s' % response)
         else:
-            logging.info('JobFlow with id %s terminated', self.job_flow_id)
+            self.logger.info('JobFlow with id %s terminated', self.job_flow_id)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/file_to_wasb.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/file_to_wasb.py b/airflow/contrib/operators/file_to_wasb.py
index 32e6b29..4519e1e 100644
--- a/airflow/contrib/operators/file_to_wasb.py
+++ b/airflow/contrib/operators/file_to_wasb.py
@@ -12,9 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
-
 from airflow.contrib.hooks.wasb_hook import WasbHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -23,7 +20,7 @@ from airflow.utils.decorators import apply_defaults
 class FileToWasbOperator(BaseOperator):
     """
     Uploads a file to Azure Blob Storage.
-    
+
     :param file_path: Path to the file to load.
     :type file_path: str
     :param container_name: Name of the container.
@@ -54,8 +51,7 @@ class FileToWasbOperator(BaseOperator):
     def execute(self, context):
         """Upload a file to Azure Blob Storage."""
         hook = WasbHook(wasb_conn_id=self.wasb_conn_id)
-        logging.info(
-            'Uploading {self.file_path} to wasb://{self.container_name} as '
-            '{self.blob_name}'.format(**locals()))
-        hook.load_file(self.file_path, self.container_name, self.blob_name,
-                       **self.load_options)
+        self.logger.info(
+            'Uploading {self.file_path} to wasb://{self.container_name} as {self.blob_name}'.format(**locals())
+        )
+        hook.load_file(self.file_path, self.container_name, self.blob_name, **self.load_options)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/fs_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/fs_operator.py b/airflow/contrib/operators/fs_operator.py
index 2596487..ca7d716 100644
--- a/airflow/contrib/operators/fs_operator.py
+++ b/airflow/contrib/operators/fs_operator.py
@@ -14,7 +14,6 @@
 #
 
 from os import walk
-import logging
 
 from airflow.operators.sensors import BaseSensorOperator
 from airflow.contrib.hooks.fs_hook import FSHook
@@ -49,8 +48,7 @@ class FileSensor(BaseSensorOperator):
         hook = FSHook(self.fs_conn_id)
         basepath = hook.get_path()
         full_path = "/".join([basepath, self.filepath])
-        logging.info(
-            'Poking for file {full_path} '.format(**locals()))
+        self.logger.info('Poking for file {full_path}'.format(**locals()))
         try:
             files = [f for f in walk(full_path)]
         except:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/gcs_download_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/gcs_download_operator.py b/airflow/contrib/operators/gcs_download_operator.py
index c17f774..27e85b7 100644
--- a/airflow/contrib/operators/gcs_download_operator.py
+++ b/airflow/contrib/operators/gcs_download_operator.py
@@ -12,7 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
 import sys
 
 from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
@@ -48,7 +47,6 @@ class GoogleCloudStorageDownloadOperator(BaseOperator):
     template_fields = ('bucket', 'object', 'filename', 'store_to_xcom_key',)
     ui_color = '#f0eee4'
 
-    @apply_defaults
     def __init__(self,
                  bucket,
                  object,
@@ -67,7 +65,7 @@ class GoogleCloudStorageDownloadOperator(BaseOperator):
         self.delegate_to = delegate_to
 
     def execute(self, context):
-        logging.info('Executing download: %s, %s, %s', self.bucket, self.object, self.filename)
+        self.logger.info('Executing download: %s, %s, %s', self.bucket, self.object, self.filename)
         hook = GoogleCloudStorageHook(google_cloud_storage_conn_id=self.google_cloud_storage_conn_id,
                                       delegate_to=self.delegate_to)
         file_bytes = hook.download(self.bucket, self.object, self.filename)
@@ -76,4 +74,4 @@ class GoogleCloudStorageDownloadOperator(BaseOperator):
                 context['ti'].xcom_push(key=self.store_to_xcom_key, value=file_bytes)
             else:
                 raise RuntimeError('The size of the downloaded file is too large to push to XCom!')
-        logging.info(file_bytes)
+        self.logger.info(file_bytes)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/gcs_to_bq.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/gcs_to_bq.py b/airflow/contrib/operators/gcs_to_bq.py
index 39f0a48..01f53cc 100644
--- a/airflow/contrib/operators/gcs_to_bq.py
+++ b/airflow/contrib/operators/gcs_to_bq.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 import json
-import logging
 
 from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
@@ -190,7 +189,8 @@ class GoogleCloudStorageToBigQueryOperator(BaseOperator):
                 self.destination_project_dataset_table))
             row = cursor.fetchone()
             max_id = row[0] if row[0] else 0
-            logging.info('Loaded BQ data with max {}.{}={}'.format(
-                self.destination_project_dataset_table,
-                self.max_id_key, max_id))
+            self.logger.info(
+                'Loaded BQ data with max %s.%s=%s',
+                self.destination_project_dataset_table, self.max_id_key, max_id
+            )
             return max_id

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/hipchat_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/hipchat_operator.py b/airflow/contrib/operators/hipchat_operator.py
index aeb37d9..19c6d76 100644
--- a/airflow/contrib/operators/hipchat_operator.py
+++ b/airflow/contrib/operators/hipchat_operator.py
@@ -17,7 +17,6 @@ from builtins import str
 from airflow.utils.decorators import apply_defaults
 from airflow.models import BaseOperator
 from airflow.exceptions import AirflowException
-import logging
 import requests
 import json
 
@@ -67,7 +66,7 @@ class HipChatAPIOperator(BaseOperator):
                                         'Authorization': 'Bearer %s' % self.token},
                                     data=self.body)
         if response.status_code >= 400:
-            logging.error('HipChat API call failed: %s %s',
+            self.logger.error('HipChat API call failed: %s %s',
                           response.status_code, response.reason)
             raise AirflowException('HipChat API call failed: %s %s' %
                                    (response.status_code, response.reason))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/mlengine_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/mlengine_operator.py b/airflow/contrib/operators/mlengine_operator.py
index 7476825..fdbfede 100644
--- a/airflow/contrib/operators/mlengine_operator.py
+++ b/airflow/contrib/operators/mlengine_operator.py
@@ -13,8 +13,6 @@
 # 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
 import re
 
 from airflow import settings
@@ -24,8 +22,9 @@ from airflow.operators import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from apiclient import errors
 
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
-logging.getLogger('GoogleCloudMLEngine').setLevel(settings.LOGGING_LEVEL)
+log = LoggingMixin().logger
 
 
 def _create_prediction_input(project_id,
@@ -52,7 +51,6 @@ def _create_prediction_input(project_id,
     Raises:
         ValueError: if a unique model/version origin cannot be determined.
     """
-
     prediction_input = {
         'dataFormat': data_format,
         'inputPaths': input_paths,
@@ -62,9 +60,9 @@ def _create_prediction_input(project_id,
 
     if uri:
         if model_name or version_name:
-            logging.error(
-                'Ambiguous model origin: Both uri and model/version name are '
-                'provided.')
+            log.error(
+                'Ambiguous model origin: Both uri and model/version name are provided.'
+            )
             raise ValueError('Ambiguous model origin.')
         prediction_input['uri'] = uri
     elif model_name:
@@ -75,7 +73,7 @@ def _create_prediction_input(project_id,
             prediction_input['versionName'] = \
                 origin_name + '/versions/{}'.format(version_name)
     else:
-        logging.error(
+        log.error(
             'Missing model origin: Batch prediction expects a model, '
             'a model & version combination, or a URI to savedModel.')
         raise ValueError('Missing model origin.')
@@ -227,9 +225,10 @@ class MLEngineBatchPredictionOperator(BaseOperator):
                 model_name, version_name, uri, max_worker_count,
                 runtime_version)
         except ValueError as e:
-            logging.error(
-                'Cannot create batch prediction job request due to: {}'
-                .format(str(e)))
+            self.logger.error(
+                'Cannot create batch prediction job request due to: %s',
+                e
+            )
             raise
 
         self.prediction_job_request = {
@@ -252,7 +251,7 @@ class MLEngineBatchPredictionOperator(BaseOperator):
             raise
 
         if finished_prediction_job['state'] != 'SUCCEEDED':
-            logging.error(
+            self.logger.error(
                 'Batch prediction job failed: %s',
                 str(finished_prediction_job))
             raise RuntimeError(finished_prediction_job['errorMessage'])
@@ -539,9 +538,8 @@ class MLEngineTrainingOperator(BaseOperator):
         }
 
         if self._mode == 'DRY_RUN':
-            logging.info('In dry_run mode.')
-            logging.info(
-                'MLEngine Training job request is: {}'.format(training_request))
+            self.logger.info('In dry_run mode.')
+            self.logger.info('MLEngine Training job request is: {}'.format(training_request))
             return
 
         hook = MLEngineHook(
@@ -559,6 +557,6 @@ class MLEngineTrainingOperator(BaseOperator):
             raise
 
         if finished_training_job['state'] != 'SUCCEEDED':
-            logging.error('MLEngine training job failed: {}'.format(
+            self.logger.error('MLEngine training job failed: {}'.format(
                 str(finished_training_job)))
             raise RuntimeError(finished_training_job['errorMessage'])

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/mlengine_prediction_summary.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/mlengine_prediction_summary.py b/airflow/contrib/operators/mlengine_prediction_summary.py
index 1f4d540..17fc2c0 100644
--- a/airflow/contrib/operators/mlengine_prediction_summary.py
+++ b/airflow/contrib/operators/mlengine_prediction_summary.py
@@ -95,7 +95,6 @@ from __future__ import print_function
 import argparse
 import base64
 import json
-import logging
 import os
 
 import apache_beam as beam
@@ -173,5 +172,4 @@ def run(argv=None):
 
 
 if __name__ == "__main__":
-    logging.getLogger().setLevel(logging.INFO)
     run()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/mysql_to_gcs.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/mysql_to_gcs.py b/airflow/contrib/operators/mysql_to_gcs.py
index 374567b..7e83bce 100644
--- a/airflow/contrib/operators/mysql_to_gcs.py
+++ b/airflow/contrib/operators/mysql_to_gcs.py
@@ -13,14 +13,12 @@
 # limitations under the License.
 
 import json
-import logging
 import time
 
 from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
 from airflow.hooks.mysql_hook import MySqlHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
-from collections import OrderedDict
 from datetime import date, datetime
 from decimal import Decimal
 from MySQLdb.constants import FIELD_TYPE
@@ -170,7 +168,7 @@ class MySqlToGoogleCloudStorageOperator(BaseOperator):
                 'mode': field_mode,
             })
 
-        logging.info('Using schema for %s: %s', self.schema_filename, schema)
+        self.logger.info('Using schema for %s: %s', self.schema_filename, schema)
         tmp_schema_file_handle = NamedTemporaryFile(delete=True)
         json.dump(schema, tmp_schema_file_handle)
         return {self.schema_filename: tmp_schema_file_handle}

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/sftp_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/sftp_operator.py b/airflow/contrib/operators/sftp_operator.py
index b9f07d5..5abfc51 100644
--- a/airflow/contrib/operators/sftp_operator.py
+++ b/airflow/contrib/operators/sftp_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.ssh_hook import SSHHook
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
@@ -84,12 +81,12 @@ class SFTPOperator(BaseOperator):
             if self.operation.lower() == SFTPOperation.GET:
                 file_msg = "from {0} to {1}".format(self.remote_filepath,
                                                     self.local_filepath)
-                logging.debug("Starting to transfer {0}".format(file_msg))
+                self.logger.debug("Starting to transfer %s", file_msg)
                 sftp_client.get(self.remote_filepath, self.local_filepath)
             else:
                 file_msg = "from {0} to {1}".format(self.local_filepath,
                                                     self.remote_filepath)
-                logging.debug("Starting to transfer file {0}".format(file_msg))
+                self.logger.debug("Starting to transfer file %s", file_msg)
                 sftp_client.put(self.local_filepath, self.remote_filepath)
 
         except Exception as e:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/spark_submit_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/spark_submit_operator.py b/airflow/contrib/operators/spark_submit_operator.py
index ca628e9..2aed4c6 100644
--- a/airflow/contrib/operators/spark_submit_operator.py
+++ b/airflow/contrib/operators/spark_submit_operator.py
@@ -12,14 +12,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-import logging
-
 from airflow.contrib.hooks.spark_submit_hook import SparkSubmitHook
 from airflow.models import BaseOperator
-from airflow.utils.decorators import apply_defaults
 from airflow.settings import WEB_COLORS
-
-log = logging.getLogger(__name__)
+from airflow.utils.decorators import apply_defaults
 
 
 class SparkSubmitOperator(BaseOperator):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/ssh_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/ssh_operator.py b/airflow/contrib/operators/ssh_operator.py
index 2e03b96..897cd1a 100644
--- a/airflow/contrib/operators/ssh_operator.py
+++ b/airflow/contrib/operators/ssh_operator.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 from base64 import b64encode
-import logging
 
 from airflow import configuration
 from airflow.contrib.hooks.ssh_hook import SSHHook

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/vertica_operator.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/vertica_operator.py b/airflow/contrib/operators/vertica_operator.py
index 9266563..fc9cf3b 100644
--- a/airflow/contrib/operators/vertica_operator.py
+++ b/airflow/contrib/operators/vertica_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.vertica_hook import VerticaHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -42,6 +39,6 @@ class VerticaOperator(BaseOperator):
         self.sql = sql
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         hook = VerticaHook(vertica_conn_id=self.vertica_conn_id)
         hook.run(self.sql)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/operators/vertica_to_hive.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/operators/vertica_to_hive.py b/airflow/contrib/operators/vertica_to_hive.py
index 31ce110..35ff3c6 100644
--- a/airflow/contrib/operators/vertica_to_hive.py
+++ b/airflow/contrib/operators/vertica_to_hive.py
@@ -15,7 +15,6 @@
 from builtins import chr
 from collections import OrderedDict
 import unicodecsv as csv
-import logging
 from tempfile import NamedTemporaryFile
 
 from airflow.hooks.hive_hooks import HiveCliHook
@@ -104,7 +103,7 @@ class VerticaToHiveTransfer(BaseOperator):
         hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id)
         vertica = VerticaHook(vertica_conn_id=self.vertica_conn_id)
 
-        logging.info("Dumping Vertica query results to local file")
+        self.logger.info("Dumping Vertica query results to local file")
         conn = vertica.get_conn()
         cursor = conn.cursor()
         cursor.execute(self.sql)
@@ -120,7 +119,7 @@ class VerticaToHiveTransfer(BaseOperator):
             f.flush()
             cursor.close()
             conn.close()
-            logging.info("Loading file into Hive")
+            self.logger.info("Loading file into Hive")
             hive.load_file(
                 f.name,
                 self.hive_table,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/bigquery_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/bigquery_sensor.py b/airflow/contrib/sensors/bigquery_sensor.py
index 8a8ca62..630cebe 100644
--- a/airflow/contrib/sensors/bigquery_sensor.py
+++ b/airflow/contrib/sensors/bigquery_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.operators.sensors import BaseSensorOperator
 from airflow.contrib.hooks.bigquery_hook import BigQueryHook
 from airflow.utils.decorators import apply_defaults
@@ -62,7 +59,7 @@ class BigQueryTableSensor(BaseSensorOperator):
 
     def poke(self, context):
         table_uri = '{0}:{1}.{2}'.format(self.project_id, self.dataset_id, self.table_id)
-        logging.info('Sensor checks existence of table: %s', table_uri)
+        self.logger.info('Sensor checks existence of table: %s', table_uri)
         hook = BigQueryHook(
             bigquery_conn_id=self.bigquery_conn_id,
             delegate_to=self.delegate_to)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/datadog_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/datadog_sensor.py b/airflow/contrib/sensors/datadog_sensor.py
index d8660f7..4ee45f9 100644
--- a/airflow/contrib/sensors/datadog_sensor.py
+++ b/airflow/contrib/sensors/datadog_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.operators.sensors import BaseSensorOperator
 from airflow.contrib.hooks.datadog_hook import DatadogHook
 from airflow.utils import apply_defaults
@@ -70,7 +67,7 @@ class DatadogSensor(BaseSensorOperator):
             tags=self.tags)
 
         if isinstance(response, dict) and response.get('status', 'ok') != 'ok':
-            logging.error("Unexpected datadog result: %s" % (response))
+            self.logger.error("Unexpected Datadog result: %s", response)
             raise AirflowException("Datadog returned unexpected result")
 
         if self.response_check:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/emr_base_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/emr_base_sensor.py b/airflow/contrib/sensors/emr_base_sensor.py
index 5526604..034fcb6 100644
--- a/airflow/contrib/sensors/emr_base_sensor.py
+++ b/airflow/contrib/sensors/emr_base_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.operators.sensors import BaseSensorOperator
 from airflow.utils import apply_defaults
 from airflow.exceptions import AirflowException
@@ -23,7 +20,7 @@ class EmrBaseSensor(BaseSensorOperator):
     """
     Contains general sensor behavior for EMR.
     Subclasses should implement get_emr_response() and state_from_response() methods.
-    Subclasses should also implment NON_TERMINAL_STATES and FAILED_STATE constants.
+    Subclasses should also implement NON_TERMINAL_STATES and FAILED_STATE constants.
     """
     ui_color = '#66c3ff'
 
@@ -39,11 +36,11 @@ class EmrBaseSensor(BaseSensorOperator):
         response = self.get_emr_response()
 
         if not response['ResponseMetadata']['HTTPStatusCode'] == 200:
-            logging.info('Bad HTTP response: %s' % response)
+            self.logger.info('Bad HTTP response: %s', response)
             return False
 
         state = self.state_from_response(response)
-        logging.info('Job flow currently %s' % state)
+        self.logger.info('Job flow currently %s', state)
 
         if state in self.NON_TERMINAL_STATES:
             return False

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/emr_job_flow_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/emr_job_flow_sensor.py b/airflow/contrib/sensors/emr_job_flow_sensor.py
index 662b3b8..e5610a1 100644
--- a/airflow/contrib/sensors/emr_job_flow_sensor.py
+++ b/airflow/contrib/sensors/emr_job_flow_sensor.py
@@ -11,10 +11,6 @@
 # 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 airflow.contrib.hooks.emr_hook import EmrHook
 from airflow.contrib.sensors.emr_base_sensor import EmrBaseSensor
 from airflow.utils import apply_defaults
@@ -45,7 +41,7 @@ class EmrJobFlowSensor(EmrBaseSensor):
     def get_emr_response(self):
         emr = EmrHook(aws_conn_id=self.aws_conn_id).get_conn()
 
-        logging.info('Poking cluster %s' % self.job_flow_id)
+        self.logger.info('Poking cluster %s', self.job_flow_id)
         return emr.describe_cluster(ClusterId=self.job_flow_id)
 
     def state_from_response(self, response):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/emr_step_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/emr_step_sensor.py b/airflow/contrib/sensors/emr_step_sensor.py
index 4cc6bc4..e131d77 100644
--- a/airflow/contrib/sensors/emr_step_sensor.py
+++ b/airflow/contrib/sensors/emr_step_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.emr_hook import EmrHook
 from airflow.contrib.sensors.emr_base_sensor import EmrBaseSensor
 from airflow.utils import apply_defaults
@@ -48,7 +45,7 @@ class EmrStepSensor(EmrBaseSensor):
     def get_emr_response(self):
         emr = EmrHook(aws_conn_id=self.aws_conn_id).get_conn()
 
-        logging.info('Poking step {0} on cluster {1}'.format(self.step_id, self.job_flow_id))
+        self.logger.info('Poking step %s on cluster %s', self.step_id, self.job_flow_id)
         return emr.describe_step(ClusterId=self.job_flow_id, StepId=self.step_id)
 
     def state_from_response(self, response):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/ftp_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/ftp_sensor.py b/airflow/contrib/sensors/ftp_sensor.py
index 4a9428b..2e604e9 100644
--- a/airflow/contrib/sensors/ftp_sensor.py
+++ b/airflow/contrib/sensors/ftp_sensor.py
@@ -11,9 +11,7 @@
 # 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 ftplib
-import logging
 
 from airflow.contrib.hooks.ftp_hook import FTPHook, FTPSHook
 from airflow.operators.sensors import BaseSensorOperator
@@ -44,7 +42,7 @@ class FTPSensor(BaseSensorOperator):
 
     def poke(self, context):
         with self._create_hook() as hook:
-            logging.info('Poking for %s', self.path)
+            self.logger.info('Poking for %s', self.path)
             try:
                 hook.get_mod_time(self.path)
             except ftplib.error_perm as e:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/gcs_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/gcs_sensor.py b/airflow/contrib/sensors/gcs_sensor.py
index c9d741b..800c1bd 100644
--- a/airflow/contrib/sensors/gcs_sensor.py
+++ b/airflow/contrib/sensors/gcs_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
 from airflow.operators.sensors import BaseSensorOperator
 from airflow.utils.decorators import apply_defaults
@@ -57,7 +54,7 @@ class GoogleCloudStorageObjectSensor(BaseSensorOperator):
         self.delegate_to = delegate_to
 
     def poke(self, context):
-        logging.info('Sensor checks existence of : %s, %s', self.bucket, self.object)
+        self.logger.info('Sensor checks existence of : %s, %s', self.bucket, self.object)
         hook = GoogleCloudStorageHook(
             google_cloud_storage_conn_id=self.google_cloud_conn_id,
             delegate_to=self.delegate_to)
@@ -119,7 +116,7 @@ class GoogleCloudStorageObjectUpdatedSensor(BaseSensorOperator):
         self.delegate_to = delegate_to
 
     def poke(self, context):
-        logging.info('Sensor checks existence of : %s, %s', self.bucket, self.object)
+        self.logger.info('Sensor checks existence of : %s, %s', self.bucket, self.object)
         hook = GoogleCloudStorageHook(
             google_cloud_storage_conn_id=self.google_cloud_conn_id,
             delegate_to=self.delegate_to)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/hdfs_sensors.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/hdfs_sensors.py b/airflow/contrib/sensors/hdfs_sensors.py
index 4e9bb9b..11e8b07 100644
--- a/airflow/contrib/sensors/hdfs_sensors.py
+++ b/airflow/contrib/sensors/hdfs_sensors.py
@@ -12,7 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 from airflow.operators.sensors import HdfsSensor
-import logging
 
 
 class HdfsSensorRegex(HdfsSensor):
@@ -29,9 +28,9 @@ class HdfsSensorRegex(HdfsSensor):
         :return: Bool depending on the search criteria
         """
         sb = self.hook(self.hdfs_conn_id).get_conn()
-        logging.getLogger("snakebite").setLevel(logging.WARNING)
-        logging.info(
-            'Poking for {self.filepath} to be a directory with files matching {self.regex.pattern}'.format(**locals()))
+        self.logger.info(
+            'Poking for {self.filepath} to be a directory with files matching {self.regex.pattern}'.format(**locals())
+        )
         result = [f for f in sb.ls([self.filepath], include_toplevel=False) if
                   f['file_type'] == 'f' and self.regex.match(f['path'].replace('%s/' % self.filepath, ''))]
         result = self.filter_for_ignored_ext(result, self.ignored_ext, self.ignore_copying)
@@ -53,15 +52,14 @@ class HdfsSensorFolder(HdfsSensor):
         :return: Bool depending on the search criteria
         """
         sb = self.hook(self.hdfs_conn_id).get_conn()
-        logging.getLogger("snakebite").setLevel(logging.WARNING)
         result = [f for f in sb.ls([self.filepath], include_toplevel=True)]
         result = self.filter_for_ignored_ext(result, self.ignored_ext, self.ignore_copying)
         result = self.filter_for_filesize(result, self.file_size)
         if self.be_empty:
-            logging.info('Poking for filepath {self.filepath} to a empty directory'.format(**locals()))
+            self.logger.info('Poking for filepath {self.filepath} to a empty directory'.format(**locals()))
             return len(result) == 1 and result[0]['path'] == self.filepath
         else:
-            logging.info('Poking for filepath {self.filepath} to a non empty directory'.format(**locals()))
+            self.logger.info('Poking for filepath {self.filepath} to a non empty directory'.format(**locals()))
             result.pop(0)
             return bool(result) and result[0]['file_type'] == 'f'
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/jira_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/jira_sensor.py b/airflow/contrib/sensors/jira_sensor.py
index 708caad..4cbc676 100644
--- a/airflow/contrib/sensors/jira_sensor.py
+++ b/airflow/contrib/sensors/jira_sensor.py
@@ -11,9 +11,6 @@
 # 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 jira.resources import Resource
 
 from airflow.contrib.operators.jira_operator import JIRAError
@@ -100,8 +97,7 @@ class JiraTicketSensor(JiraSensor):
                                                *args, **kwargs)
 
     def poke(self, context):
-        logging.info('Jira Sensor checking for change in ticket : {0}'
-                     .format(self.ticket_id))
+        self.logger.info('Jira Sensor checking for change in ticket: %s', self.ticket_id)
 
         self.jira_operator.method_name = "issue"
         self.jira_operator.jira_method_args = {
@@ -127,20 +123,19 @@ class JiraTicketSensor(JiraSensor):
                             and getattr(field_value, 'name'):
                         result = self.expected_value.lower() == field_value.name.lower()
                     else:
-                        logging.warning("not implemented checker for issue field {0} "
-                                        "which is neither string nor list nor "
-                                        "jira Resource".format(self.field))
+                        self.logger.warning(
+                            "Not implemented checker for issue field %s which "
+                            "is neither string nor list nor Jira Resource",
+                            self.field
+                        )
 
         except JIRAError as jira_error:
-            logging.error("jira error while checking with expected value: {0}"
-                          .format(jira_error))
+            self.logger.error("Jira error while checking with expected value: %s", jira_error)
         except Exception as e:
-            logging.error("error while checking with expected value {0}, error: {1}"
-                          .format(self.expected_value, e))
+            self.logger.error("Error while checking with expected value %s:", self.expected_value)
+            self.logger.exception(e)
         if result is True:
-            logging.info("issue field {0} has expected value {1}, returning success"
-                         .format(self.field, self.expected_value))
+            self.logger.info("Issue field %s has expected value %s, returning success", self.field, self.expected_value)
         else:
-            logging.info("issue field {0} dont have expected value {1} yet."
-                         .format(self.field, self.expected_value))
+            self.logger.info("Issue field %s don't have expected value %s yet.", self.field, self.expected_value)
         return result

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/redis_key_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/redis_key_sensor.py b/airflow/contrib/sensors/redis_key_sensor.py
index 4cab407..220d766 100644
--- a/airflow/contrib/sensors/redis_key_sensor.py
+++ b/airflow/contrib/sensors/redis_key_sensor.py
@@ -11,9 +11,6 @@
 # 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 airflow.contrib.hooks.redis_hook import RedisHook
 from airflow.operators.sensors import BaseSensorOperator
 from airflow.utils.decorators import apply_defaults
@@ -37,7 +34,6 @@ class RedisKeySensor(BaseSensorOperator):
         :type redis_conn_id: string
         """
         super(RedisKeySensor, self).__init__(*args, **kwargs)
-        self.logger = logging.getLogger(__name__)
         self.redis_conn_id = redis_conn_id
         self.key = key
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/sensors/wasb_sensor.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/sensors/wasb_sensor.py b/airflow/contrib/sensors/wasb_sensor.py
index 3f3d56c..1a54e12 100644
--- a/airflow/contrib/sensors/wasb_sensor.py
+++ b/airflow/contrib/sensors/wasb_sensor.py
@@ -12,9 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
-import logging
-
 from airflow.contrib.hooks.wasb_hook import WasbHook
 from airflow.operators.sensors import BaseSensorOperator
 from airflow.utils.decorators import apply_defaults
@@ -23,7 +20,7 @@ from airflow.utils.decorators import apply_defaults
 class WasbBlobSensor(BaseSensorOperator):
     """
     Waits for a blob to arrive on Azure Blob Storage.
-    
+
     :param container_name: Name of the container.
     :type container_name: str
     :param blob_name: Name of the blob.
@@ -50,7 +47,7 @@ class WasbBlobSensor(BaseSensorOperator):
         self.check_options = check_options
 
     def poke(self, context):
-        logging.info(
+        self.logger.info(
             'Poking for blob: {self.blob_name}\n'
             'in wasb://{self.container_name}'.format(**locals())
         )
@@ -62,7 +59,7 @@ class WasbBlobSensor(BaseSensorOperator):
 class WasbPrefixSensor(BaseSensorOperator):
     """
     Waits for blobs matching a prefix to arrive on Azure Blob Storage.
-    
+
     :param container_name: Name of the container.
     :type container_name: str
     :param prefix: Prefix of the blob.
@@ -88,7 +85,7 @@ class WasbPrefixSensor(BaseSensorOperator):
         self.check_options = check_options
 
     def poke(self, context):
-        logging.info(
+        self.logger.info(
             'Poking for prefix: {self.prefix}\n'
             'in wasb://{self.container_name}'.format(**locals())
         )

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/contrib/task_runner/cgroup_task_runner.py
----------------------------------------------------------------------
diff --git a/airflow/contrib/task_runner/cgroup_task_runner.py b/airflow/contrib/task_runner/cgroup_task_runner.py
index 11c45c1..5d2518d 100644
--- a/airflow/contrib/task_runner/cgroup_task_runner.py
+++ b/airflow/contrib/task_runner/cgroup_task_runner.py
@@ -14,7 +14,6 @@
 
 import datetime
 import getpass
-import subprocess
 import os
 import uuid
 
@@ -73,13 +72,13 @@ class CgroupTaskRunner(BaseTaskRunner):
         for path_element in path_split:
             name_to_node = {x.name: x for x in node.children}
             if path_element not in name_to_node:
-                self.logger.debug("Creating cgroup {} in {}"
-                                  .format(path_element, node.path))
+                self.logger.debug("Creating cgroup %s in %s", path_element, node.path)
                 node = node.create_cgroup(path_element)
             else:
-                self.logger.debug("Not creating cgroup {} in {} "
-                                  "since it already exists"
-                                  .format(path_element, node.path))
+                self.logger.debug(
+                    "Not creating cgroup %s in %s since it already exists",
+                    path_element, node.path
+                )
                 node = name_to_node[path_element]
         return node
 
@@ -95,24 +94,23 @@ class CgroupTaskRunner(BaseTaskRunner):
         for path_element in path_split:
             name_to_node = {x.name: x for x in node.children}
             if path_element not in name_to_node:
-                self.logger.warning("Cgroup does not exist: {}"
-                                    .format(path))
+                self.logger.warning("Cgroup does not exist: %s", path)
                 return
             else:
                 node = name_to_node[path_element]
         # node is now the leaf node
         parent = node.parent
-        self.logger.debug("Deleting cgroup {}/{}".format(parent, node.name))
+        self.logger.debug("Deleting cgroup %s/%s", parent, node.name)
         parent.delete_cgroup(node.name)
 
     def start(self):
         # Use bash if it's already in a cgroup
         cgroups = self._get_cgroup_names()
         if cgroups["cpu"] != "/" or cgroups["memory"] != "/":
-            self.logger.debug("Already running in a cgroup (cpu: {} memory: {} so "
-                              "not creating another one"
-                              .format(cgroups.get("cpu"),
-                                      cgroups.get("memory")))
+            self.logger.debug(
+                "Already running in a cgroup (cpu: %s memory: %s) so not creating another one",
+                cgroups.get("cpu"), cgroups.get("memory")
+            )
             self.process = self.run_command(['bash', '-c'], join_args=True)
             return
 
@@ -135,21 +133,27 @@ class CgroupTaskRunner(BaseTaskRunner):
         mem_cgroup_node = self._create_cgroup(self.mem_cgroup_name)
         self._created_mem_cgroup = True
         if self._mem_mb_limit > 0:
-            self.logger.debug("Setting {} with {} MB of memory"
-                              .format(self.mem_cgroup_name, self._mem_mb_limit))
+            self.logger.debug(
+                "Setting %s with %s MB of memory",
+                self.mem_cgroup_name, self._mem_mb_limit
+            )
             mem_cgroup_node.controller.limit_in_bytes = self._mem_mb_limit * 1024 * 1024
 
         # Create the CPU cgroup
         cpu_cgroup_node = self._create_cgroup(self.cpu_cgroup_name)
         self._created_cpu_cgroup = True
         if self._cpu_shares > 0:
-            self.logger.debug("Setting {} with {} CPU shares"
-                              .format(self.cpu_cgroup_name, self._cpu_shares))
+            self.logger.debug(
+                "Setting %s with %s CPU shares",
+                self.cpu_cgroup_name, self._cpu_shares
+            )
             cpu_cgroup_node.controller.shares = self._cpu_shares
 
         # Start the process w/ cgroups
-        self.logger.debug("Starting task process with cgroups cpu,memory:{}"
-                          .format(cgroup_name))
+        self.logger.debug(
+            "Starting task process with cgroups cpu,memory: %s",
+            cgroup_name
+        )
         self.process = self.run_command(
             ['cgexec', '-g', 'cpu,memory:{}'.format(cgroup_name)]
         )
@@ -165,10 +169,9 @@ class CgroupTaskRunner(BaseTaskRunner):
         # we might want to revisit that approach at some other point.
         if return_code == 137:
             self.logger.warning("Task failed with return code of 137. This may indicate "
-                                "that it was killed due to excessive memory usage. "
-                                "Please consider optimizing your task or using the "
-                                "resources argument to reserve more memory for your "
-                                "task")
+                              "that it was killed due to excessive memory usage. "
+                              "Please consider optimizing your task or using the "
+                              "resources argument to reserve more memory for your task")
         return return_code
 
     def terminate(self):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/__init__.py
----------------------------------------------------------------------
diff --git a/airflow/executors/__init__.py b/airflow/executors/__init__.py
index a8cb087..7812f96 100644
--- a/airflow/executors/__init__.py
+++ b/airflow/executors/__init__.py
@@ -11,8 +11,6 @@
 # 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
 import sys
 
 from airflow import configuration
@@ -21,6 +19,7 @@ from airflow.executors.local_executor import LocalExecutor
 from airflow.executors.sequential_executor import SequentialExecutor
 
 from airflow.exceptions import AirflowException
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 DEFAULT_EXECUTOR = None
 
@@ -42,14 +41,15 @@ def GetDefaultExecutor():
 
     DEFAULT_EXECUTOR = _get_executor(executor_name)
 
-    logging.info("Using executor " + executor_name)
+    log = LoggingMixin().logger
+    log.info("Using executor %s", executor_name)
 
     return DEFAULT_EXECUTOR
 
 
 def _get_executor(executor_name):
     """
-    Creates a new instance of the named executor. In case the executor name is not know in airflow, 
+    Creates a new instance of the named executor. In case the executor name is not know in airflow,
     look for it in the plugins
     """
     if executor_name == 'LocalExecutor':

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/base_executor.py
----------------------------------------------------------------------
diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py
index 7a4065e..1197958 100644
--- a/airflow/executors/base_executor.py
+++ b/airflow/executors/base_executor.py
@@ -11,12 +11,11 @@
 # 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.
-
 from builtins import range
 
 from airflow import configuration
+from airflow.utils.log.LoggingMixin import LoggingMixin
 from airflow.utils.state import State
-from airflow.utils.logging import LoggingMixin
 
 PARALLELISM = configuration.getint('core', 'PARALLELISM')
 
@@ -47,7 +46,7 @@ class BaseExecutor(LoggingMixin):
     def queue_command(self, task_instance, command, priority=1, queue=None):
         key = task_instance.key
         if key not in self.queued_tasks and key not in self.running:
-            self.logger.info("Adding to queue: {}".format(command))
+            self.logger.info("Adding to queue: %s", command)
             self.queued_tasks[key] = (command, priority, queue, task_instance)
 
     def queue_task_instance(
@@ -100,9 +99,9 @@ class BaseExecutor(LoggingMixin):
         else:
             open_slots = self.parallelism - len(self.running)
 
-        self.logger.debug("{} running task instances".format(len(self.running)))
-        self.logger.debug("{} in queue".format(len(self.queued_tasks)))
-        self.logger.debug("{} open slots".format(open_slots))
+        self.logger.debug("%s running task instances", len(self.running))
+        self.logger.debug("%s in queue", len(self.queued_tasks))
+        self.logger.debug("%s open slots", open_slots)
 
         sorted_queue = sorted(
             [(k, v) for k, v in self.queued_tasks.items()],
@@ -124,11 +123,12 @@ class BaseExecutor(LoggingMixin):
                 self.execute_async(key, command=command, queue=queue)
             else:
                 self.logger.debug(
-                    'Task is already running, not sending to '
-                    'executor: {}'.format(key))
+                    'Task is already running, not sending to executor: %s',
+                    key
+                )
 
         # Calling child class sync method
-        self.logger.debug("Calling the {} sync method".format(self.__class__))
+        self.logger.debug("Calling the %s sync method", self.__class__)
         self.sync()
 
     def change_state(self, key, state):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/celery_executor.py
----------------------------------------------------------------------
diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py
index 17c343b..39c895c 100644
--- a/airflow/executors/celery_executor.py
+++ b/airflow/executors/celery_executor.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 from builtins import object
-import logging
 import subprocess
 import ssl
 import time
@@ -25,6 +24,7 @@ from celery import states as celery_states
 from airflow.exceptions import AirflowConfigException, AirflowException
 from airflow.executors.base_executor import BaseExecutor
 from airflow import configuration
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 PARALLELISM = configuration.get('core', 'PARALLELISM')
 
@@ -53,7 +53,8 @@ class CeleryConfig(object):
     try:
         celery_ssl_active = configuration.getboolean('celery', 'CELERY_SSL_ACTIVE')
     except AirflowConfigException as e:
-        logging.warning("Celery Executor will run without SSL")
+        log = LoggingMixin().logger
+        log.warning("Celery Executor will run without SSL")
 
     try:
         if celery_ssl_active:
@@ -75,11 +76,12 @@ app = Celery(
 
 @app.task
 def execute_command(command):
-    logging.info("Executing command in Celery " + command)
+    log = LoggingMixin().logger
+    log.info("Executing command in Celery: %s", command)
     try:
         subprocess.check_call(command, shell=True)
     except subprocess.CalledProcessError as e:
-        logging.error(e)
+        log.error(e)
         raise AirflowException('Celery command failed')
 
 
@@ -92,22 +94,18 @@ class CeleryExecutor(BaseExecutor):
     vast amounts of messages, while providing operations with the tools
     required to maintain such a system.
     """
-
     def start(self):
         self.tasks = {}
         self.last_state = {}
 
     def execute_async(self, key, command, queue=DEFAULT_QUEUE):
-        self.logger.info( "[celery] queuing {key} through celery, "
-                       "queue={queue}".format(**locals()))
+        self.logger.info("[celery] queuing {key} through celery, queue={queue}".format(**locals()))
         self.tasks[key] = execute_command.apply_async(
             args=[command], queue=queue)
         self.last_state[key] = celery_states.PENDING
 
     def sync(self):
-
-        self.logger.debug(
-            "Inquiring about {} celery task(s)".format(len(self.tasks)))
+        self.logger.debug("Inquiring about %s celery task(s)", len(self.tasks))
         for key, async in list(self.tasks.items()):
             try:
                 state = async.state
@@ -125,11 +123,11 @@ class CeleryExecutor(BaseExecutor):
                         del self.tasks[key]
                         del self.last_state[key]
                     else:
-                        self.logger.info("Unexpected state: " + async.state)
+                        self.logger.info("Unexpected state: %s", async.state)
                     self.last_state[key] = async.state
             except Exception as e:
-                logging.error("Error syncing the celery executor, ignoring "
-                              "it:\n{}\n".format(e, traceback.format_exc()))
+                self.logger.error("Error syncing the celery executor, ignoring it:")
+                self.logger.exception(e)
 
     def end(self, synchronous=False):
         if synchronous:

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/executors/dask_executor.py
----------------------------------------------------------------------
diff --git a/airflow/executors/dask_executor.py b/airflow/executors/dask_executor.py
index d65830a..8a56506 100644
--- a/airflow/executors/dask_executor.py
+++ b/airflow/executors/dask_executor.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 import distributed
-
 import subprocess
 import warnings
 
@@ -41,8 +40,8 @@ class DaskExecutor(BaseExecutor):
     def execute_async(self, key, command, queue=None):
         if queue is not None:
             warnings.warn(
-                'DaskExecutor does not support queues. All tasks will be run '
-                'in the same cluster')
+                'DaskExecutor does not support queues. All tasks will be run in the same cluster'
+            )
 
         def airflow_run():
             return subprocess.check_call(command, shell=True)
@@ -54,12 +53,11 @@ class DaskExecutor(BaseExecutor):
         if future.done():
             key = self.futures[future]
             if future.exception():
+                self.logger.error("Failed to execute task: %s", repr(future.exception()))
                 self.fail(key)
-                self.logger.error("Failed to execute task: {}".format(
-                    repr(future.exception())))
             elif future.cancelled():
-                self.fail(key)
                 self.logger.error("Failed to execute task")
+                self.fail(key)
             else:
                 self.success(key)
             self.futures.pop(future)



[2/5] incubator-airflow git commit: [AIRFLOW-1582] Improve logging within Airflow

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/models.py
----------------------------------------------------------------------
diff --git a/airflow/models.py b/airflow/models.py
index 3078f4e..39e65e8 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -35,7 +35,6 @@ import inspect
 import zipfile
 import jinja2
 import json
-import logging
 import os
 import pickle
 import re
@@ -75,11 +74,11 @@ from airflow.utils.decorators import apply_defaults
 from airflow.utils.email import send_email
 from airflow.utils.helpers import (
     as_tuple, is_container, is_in, validate_key, pprinttable)
-from airflow.utils.logging import LoggingMixin
 from airflow.utils.operator_resources import Resources
 from airflow.utils.state import State
 from airflow.utils.timeout import timeout
 from airflow.utils.trigger_rule import TriggerRule
+from airflow.utils.log.LoggingMixin import LoggingMixin
 
 Base = declarative_base()
 ID_LEN = 250
@@ -87,7 +86,6 @@ XCOM_RETURN_KEY = 'return_value'
 
 Stats = settings.Stats
 
-
 def get_fernet():
     """
     Deferred load of Fernet key.
@@ -180,6 +178,7 @@ class DagBag(BaseDagBag, LoggingMixin):
         by the scheduler job only
     :type sync_to_db: bool
     """
+
     def __init__(
             self,
             dag_folder=None,
@@ -190,7 +189,7 @@ class DagBag(BaseDagBag, LoggingMixin):
         if executor is None:
             executor = GetDefaultExecutor()
         dag_folder = dag_folder or settings.DAGS_FOLDER
-        self.logger.info("Filling up the DagBag from {}".format(dag_folder))
+        self.logger.info("Filling up the DagBag from %s", dag_folder)
         self.dag_folder = dag_folder
         self.dags = {}
         # the file's last modified timestamp when we last read it
@@ -263,7 +262,7 @@ class DagBag(BaseDagBag, LoggingMixin):
                 return found_dags
 
         except Exception as e:
-            logging.exception(e)
+            self.logger.exception(e)
             return found_dags
 
         mods = []
@@ -275,7 +274,7 @@ class DagBag(BaseDagBag, LoggingMixin):
                         self.file_last_changed[filepath] = file_last_changed_on_disk
                         return found_dags
 
-            self.logger.debug("Importing {}".format(filepath))
+            self.logger.debug("Importing %s", filepath)
             org_mod_name, _ = os.path.splitext(os.path.split(filepath)[-1])
             mod_name = ('unusual_prefix_' +
                         hashlib.sha1(filepath.encode('utf-8')).hexdigest() +
@@ -289,7 +288,7 @@ class DagBag(BaseDagBag, LoggingMixin):
                     m = imp.load_source(mod_name, filepath)
                     mods.append(m)
                 except Exception as e:
-                    self.logger.exception("Failed to import: " + filepath)
+                    self.logger.exception("Failed to import: %s", filepath)
                     self.import_errors[filepath] = str(e)
                     self.file_last_changed[filepath] = file_last_changed_on_disk
 
@@ -300,13 +299,10 @@ class DagBag(BaseDagBag, LoggingMixin):
                 mod_name, ext = os.path.splitext(mod.filename)
                 if not head and (ext == '.py' or ext == '.pyc'):
                     if mod_name == '__init__':
-                        self.logger.warning("Found __init__.{0} at root of {1}".
-                                            format(ext, filepath))
-
+                        self.logger.warning("Found __init__.%s at root of %s", ext, filepath)
                     if safe_mode:
                         with zip_file.open(mod.filename) as zf:
-                            self.logger.debug("Reading {} from {}".
-                                              format(mod.filename, filepath))
+                            self.logger.debug("Reading %s from %s", mod.filename, filepath)
                             content = zf.read()
                             if not all([s in content for s in (b'DAG', b'airflow')]):
                                 self.file_last_changed[filepath] = (
@@ -322,7 +318,7 @@ class DagBag(BaseDagBag, LoggingMixin):
                         m = importlib.import_module(mod_name)
                         mods.append(m)
                     except Exception as e:
-                        self.logger.exception("Failed to import: " + filepath)
+                        self.logger.exception("Failed to import: %s", filepath)
                         self.import_errors[filepath] = str(e)
                         self.file_last_changed[filepath] = file_last_changed_on_disk
 
@@ -347,11 +343,9 @@ class DagBag(BaseDagBag, LoggingMixin):
         from airflow.jobs import LocalTaskJob as LJ
         self.logger.info("Finding 'running' jobs without a recent heartbeat")
         TI = TaskInstance
-        secs = (
-            configuration.getint('scheduler', 'scheduler_zombie_task_threshold'))
+        secs = configuration.getint('scheduler', 'scheduler_zombie_task_threshold')
         limit_dttm = datetime.now() - timedelta(seconds=secs)
-        self.logger.info(
-            "Failing jobs without heartbeat after {}".format(limit_dttm))
+        self.logger.info("Failing jobs without heartbeat after %s", limit_dttm)
 
         tis = (
             session.query(TI)
@@ -371,9 +365,8 @@ class DagBag(BaseDagBag, LoggingMixin):
                 if ti.task_id in dag.task_ids:
                     task = dag.get_task(ti.task_id)
                     ti.task = task
-                    ti.handle_failure("{} killed as zombie".format(ti))
-                    self.logger.info(
-                        'Marked zombie job {} as failed'.format(ti))
+                    ti.handle_failure("{} killed as zombie".format(str(ti)))
+                    self.logger.info('Marked zombie job %s as failed', ti)
                     Stats.incr('zombies_killed')
         session.commit()
 
@@ -451,7 +444,7 @@ class DagBag(BaseDagBag, LoggingMixin):
                                 str([dag.dag_id for dag in found_dags]),
                             ))
                     except Exception as e:
-                        logging.warning(e)
+                        self.logger.warning(e)
         Stats.gauge(
             'collect_dags', (datetime.now() - start_dttm).total_seconds(), 1)
         Stats.gauge(
@@ -619,7 +612,7 @@ class Connection(Base):
                 self.is_encrypted = True
             except AirflowException:
                 self.logger.exception("Failed to load fernet while encrypting value, "
-                                      "using non-encrypted value.")
+                                    "using non-encrypted value.")
                 self._password = value
                 self.is_encrypted = False
 
@@ -648,7 +641,7 @@ class Connection(Base):
                 self.is_extra_encrypted = True
             except AirflowException:
                 self.logger.exception("Failed to load fernet while encrypting value, "
-                                      "using non-encrypted value.")
+                                    "using non-encrypted value.")
                 self._extra = value
                 self.is_extra_encrypted = False
 
@@ -718,8 +711,8 @@ class Connection(Base):
             try:
                 obj = json.loads(self.extra)
             except Exception as e:
-                logging.exception(e)
-                logging.error("Failed parsing the json for conn_id %s", self.conn_id)
+                self.logger.exception(e)
+                self.logger.error("Failed parsing the json for conn_id %s", self.conn_id)
 
         return obj
 
@@ -750,7 +743,7 @@ class DagPickle(Base):
         self.pickle = dag
 
 
-class TaskInstance(Base):
+class TaskInstance(Base, LoggingMixin):
     """
     Task instances store the state of a task instance. This table is the
     authority and single source of truth around what tasks have run and the
@@ -764,7 +757,6 @@ class TaskInstance(Base):
     even while multiple schedulers may be firing task instances.
     """
 
-
     __tablename__ = "task_instance"
 
     task_id = Column(String(ID_LEN), primary_key=True)
@@ -1014,7 +1006,7 @@ class TaskInstance(Base):
         """
         Forces the task instance's state to FAILED in the database.
         """
-        logging.error("Recording the task instance as FAILED")
+        self.logger.error("Recording the task instance as FAILED")
         self.state = State.FAILED
         session.merge(self)
         session.commit()
@@ -1165,14 +1157,16 @@ class TaskInstance(Base):
                 session=session):
             failed = True
             if verbose:
-                logging.info("Dependencies not met for {}, dependency '{}' FAILED: {}"
-                             .format(self, dep_status.dep_name, dep_status.reason))
+                self.logger.info(
+                    "Dependencies not met for %s, dependency '%s' FAILED: %s",
+                    self, dep_status.dep_name, dep_status.reason
+                )
 
         if failed:
             return False
 
         if verbose:
-            logging.info("Dependencies all met for {}".format(self))
+            self.logger.info("Dependencies all met for %s", self)
 
         return True
 
@@ -1188,11 +1182,10 @@ class TaskInstance(Base):
                     session,
                     dep_context):
 
-                logging.debug("{} dependency '{}' PASSED: {}, {}"
-                              .format(self,
-                                      dep_status.dep_name,
-                                      dep_status.passed,
-                                      dep_status.reason))
+                self.logger.debug(
+                    "%s dependency '%s' PASSED: %s, %s",
+                    self, dep_status.dep_name, dep_status.passed, dep_status.reason
+                )
 
                 if not dep_status.passed:
                     yield dep_status
@@ -1335,6 +1328,7 @@ class TaskInstance(Base):
             session.commit()
             return False
 
+        #TODO: Logging needs cleanup, not clear what is being printed
         hr = "\n" + ("-" * 80) + "\n"  # Line break
 
         # For reporting purposes, we report based on 1-indexed,
@@ -1365,11 +1359,10 @@ class TaskInstance(Base):
                    "runtime. Attempt {attempt} of {total}. State set to NONE.").format(
                 attempt=self.try_number + 1,
                 total=self.max_tries + 1)
-            logging.warning(hr + msg + hr)
+            self.logger.warning(hr + msg + hr)
 
             self.queued_dttm = datetime.now()
-            msg = "Queuing into pool {}".format(self.pool)
-            logging.info(msg)
+            self.logger.info("Queuing into pool %s", self.pool)
             session.merge(self)
             session.commit()
             return False
@@ -1378,12 +1371,12 @@ class TaskInstance(Base):
         # the current worker process was blocked on refresh_from_db
         if self.state == State.RUNNING:
             msg = "Task Instance already running {}".format(self)
-            logging.warning(msg)
+            self.logger.warning(msg)
             session.commit()
             return False
 
         # print status message
-        logging.info(hr + msg + hr)
+        self.logger.info(hr + msg + hr)
         self.try_number += 1
 
         if not test_mode:
@@ -1401,10 +1394,10 @@ class TaskInstance(Base):
         if verbose:
             if mark_success:
                 msg = "Marking success for {} on {}".format(self.task, self.execution_date)
-                logging.info(msg)
+                self.logger.info(msg)
             else:
                 msg = "Executing {} on {}".format(self.task, self.execution_date)
-                logging.info(msg)
+                self.logger.info(msg)
         return True
 
     @provide_session
@@ -1445,8 +1438,8 @@ class TaskInstance(Base):
                 self.task = task_copy
 
                 def signal_handler(signum, frame):
-                    '''Setting kill signal handler'''
-                    logging.error("Killing subprocess")
+                    """Setting kill signal handler"""
+                    self.logger.error("Killing subprocess")
                     task_copy.on_kill()
                     raise AirflowException("Task received SIGTERM signal")
                 signal.signal(signal.SIGTERM, signal_handler)
@@ -1525,8 +1518,8 @@ class TaskInstance(Base):
             if task.on_success_callback:
                 task.on_success_callback(context)
         except Exception as e3:
-            logging.error("Failed when executing success callback")
-            logging.exception(e3)
+            self.logger.error("Failed when executing success callback")
+            self.logger.exception(e3)
 
         session.commit()
 
@@ -1571,7 +1564,7 @@ class TaskInstance(Base):
         task_copy.dry_run()
 
     def handle_failure(self, error, test_mode=False, context=None):
-        logging.exception(error)
+        self.logger.exception(error)
         task = self.task
         session = settings.Session()
         self.end_date = datetime.now()
@@ -1592,21 +1585,20 @@ class TaskInstance(Base):
             # next task instance try_number exceeds the max_tries.
             if task.retries and self.try_number <= self.max_tries:
                 self.state = State.UP_FOR_RETRY
-                logging.info('Marking task as UP_FOR_RETRY')
+                self.logger.info('Marking task as UP_FOR_RETRY')
                 if task.email_on_retry and task.email:
                     self.email_alert(error, is_retry=True)
             else:
                 self.state = State.FAILED
                 if task.retries:
-                    logging.info('All retries failed; marking task as FAILED')
+                    self.logger.info('All retries failed; marking task as FAILED')
                 else:
-                    logging.info('Marking task as FAILED.')
+                    self.logger.info('Marking task as FAILED.')
                 if task.email_on_failure and task.email:
                     self.email_alert(error, is_retry=False)
         except Exception as e2:
-            logging.error(
-                'Failed to send email to: ' + str(task.email))
-            logging.exception(e2)
+            self.logger.error('Failed to send email to: %s', task.email)
+            self.logger.exception(e2)
 
         # Handling callbacks pessimistically
         try:
@@ -1615,13 +1607,13 @@ class TaskInstance(Base):
             if self.state == State.FAILED and task.on_failure_callback:
                 task.on_failure_callback(context)
         except Exception as e3:
-            logging.error("Failed at executing callback")
-            logging.exception(e3)
+            self.logger.error("Failed at executing callback")
+            self.logger.exception(e3)
 
         if not test_mode:
             session.merge(self)
         session.commit()
-        logging.error(str(error))
+        self.logger.error(str(error))
 
     @provide_session
     def get_template_context(self, session=None):
@@ -1939,7 +1931,7 @@ class SkipMixin(object):
         else:
             assert execution_date is not None, "Execution date is None and no dag run"
 
-            logging.warning("No DAG RUN present this should not happen")
+            self.logger.warning("No DAG RUN present this should not happen")
             # this is defensive against dag runs that are not complete
             for task in tasks:
                 ti = TaskInstance(task, execution_date=execution_date)
@@ -1953,7 +1945,7 @@ class SkipMixin(object):
 
 
 @functools.total_ordering
-class BaseOperator(object):
+class BaseOperator(LoggingMixin):
     """
     Abstract base class for all operators. Since operators create objects that
     become node in the dag, BaseOperator contains many recursive methods for
@@ -2134,8 +2126,7 @@ class BaseOperator(object):
         self.email_on_failure = email_on_failure
         self.start_date = start_date
         if start_date and not isinstance(start_date, datetime):
-            logging.warning(
-                "start_date for {} isn't datetime.datetime".format(self))
+            self.logger.warning("start_date for %s isn't datetime.datetime", self)
         self.end_date = end_date
         if not TriggerRule.is_valid(trigger_rule):
             raise AirflowException(
@@ -2151,10 +2142,12 @@ class BaseOperator(object):
             self.depends_on_past = True
 
         if schedule_interval:
-            logging.warning(
+            self.logger.warning(
                 "schedule_interval is used for {}, though it has "
                 "been deprecated as a task parameter, you need to "
-                "specify it as a DAG parameter instead".format(self))
+                "specify it as a DAG parameter instead",
+                self
+            )
         self._schedule_interval = schedule_interval
         self.retries = retries
         self.queue = queue
@@ -2167,7 +2160,7 @@ class BaseOperator(object):
         if isinstance(retry_delay, timedelta):
             self.retry_delay = retry_delay
         else:
-            logging.debug("retry_delay isn't timedelta object, assuming secs")
+            self.logger.debug("Retry_delay isn't timedelta object, assuming secs")
             self.retry_delay = timedelta(seconds=retry_delay)
         self.retry_exponential_backoff = retry_exponential_backoff
         self.max_retry_delay = max_retry_delay
@@ -2467,7 +2460,7 @@ class BaseOperator(object):
                 try:
                     setattr(self, attr, env.loader.get_source(env, content)[0])
                 except Exception as e:
-                    logging.exception(e)
+                    self.logger.exception(e)
         self.prepare_template()
 
     @property
@@ -2586,12 +2579,12 @@ class BaseOperator(object):
                 ignore_ti_state=ignore_ti_state)
 
     def dry_run(self):
-        logging.info('Dry run')
+        self.logger.info('Dry run')
         for attr in self.template_fields:
             content = getattr(self, attr)
             if content and isinstance(content, six.string_types):
-                logging.info('Rendering template for {0}'.format(attr))
-                logging.info(content)
+                self.logger.info('Rendering template for %s', attr)
+                self.logger.info(content)
 
     def get_direct_relatives(self, upstream=False):
         """
@@ -2753,7 +2746,7 @@ class DagModel(Base):
 
 
 @functools.total_ordering
-class DAG(BaseDag, LoggingMixin):
+class DAG(BaseDag):
     """
     A dag (directed acyclic graph) is a collection of tasks with directional
     dependencies. A dag also has a schedule, a start end an end date
@@ -3529,7 +3522,8 @@ class DAG(BaseDag, LoggingMixin):
             d['pickle_len'] = len(pickled)
             d['pickling_duration'] = "{}".format(datetime.now() - dttm)
         except Exception as e:
-            logging.debug(e)
+            log = LoggingMixin().logger
+            log.debug(e)
             d['is_picklable'] = False
             d['stacktrace'] = traceback.format_exc()
         return d
@@ -3757,12 +3751,13 @@ class DAG(BaseDag, LoggingMixin):
         :type sync_time: datetime
         :return: None
         """
+
         orm_dag = session.query(
             DagModel).filter(DagModel.dag_id == dag.dag_id).first()
         if not orm_dag:
             orm_dag = DagModel(dag_id=dag.dag_id)
-            logging.info("Creating ORM DAG for %s",
-                         dag.dag_id)
+            log = LoggingMixin().logger
+            log.info("Creating ORM DAG for %s", dag.dag_id)
         orm_dag.fileloc = dag.fileloc
         orm_dag.is_subdag = dag.is_subdag
         orm_dag.owners = owner
@@ -3805,13 +3800,14 @@ class DAG(BaseDag, LoggingMixin):
         :type expiration_date: datetime
         :return: None
         """
+        logger = LoggingMixin().logger
         for dag in session.query(
                 DagModel).filter(DagModel.last_scheduler_run < expiration_date,
                                  DagModel.is_active).all():
-            logging.info("Deactivating DAG ID %s since it was last touched "
-                         "by the scheduler at %s",
-                         dag.dag_id,
-                         dag.last_scheduler_run.isoformat())
+            logger.info(
+                "Deactivating DAG ID %s since it was last touched by the scheduler at %s",
+                dag.dag_id, dag.last_scheduler_run.isoformat()
+            )
             dag.is_active = False
             session.merge(dag)
             session.commit()
@@ -3901,7 +3897,7 @@ class KnownEvent(Base):
         return self.label
 
 
-class Variable(Base):
+class Variable(Base, LoggingMixin):
     __tablename__ = "variable"
 
     id = Column(Integer, primary_key=True)
@@ -3937,8 +3933,9 @@ class Variable(Base):
                 self._val = fernet.encrypt(bytes(value, 'utf-8')).decode()
                 self.is_encrypted = True
             except AirflowException:
-                self.logger.exception("Failed to load fernet while encrypting value, "
-                                      "using non-encrypted value.")
+                self.logger.exception(
+                    "Failed to load fernet while encrypting value, using non-encrypted value."
+                )
                 self._val = value
                 self.is_encrypted = False
 
@@ -4005,7 +4002,7 @@ class Variable(Base):
         session.flush()
 
 
-class XCom(Base):
+class XCom(Base, LoggingMixin):
     """
     Base class for XCom objects.
     """
@@ -4061,10 +4058,11 @@ class XCom(Base):
             try:
                 value = json.dumps(value).encode('UTF-8')
             except ValueError:
-                logging.error("Could not serialize the XCOM value into JSON. "
-                              "If you are using pickles instead of JSON "
-                              "for XCOM, then you need to enable pickle "
-                              "support for XCOM in your airflow config.")
+                log = LoggingMixin().logger
+                log.error("Could not serialize the XCOM value into JSON. "
+                          "If you are using pickles instead of JSON "
+                          "for XCOM, then you need to enable pickle "
+                          "support for XCOM in your airflow config.")
                 raise
 
         # remove any duplicate XComs
@@ -4131,10 +4129,11 @@ class XCom(Base):
                 try:
                     return json.loads(result.value.decode('UTF-8'))
                 except ValueError:
-                    logging.error("Could not serialize the XCOM value into JSON. "
-                                  "If you are using pickles instead of JSON "
-                                  "for XCOM, then you need to enable pickle "
-                                  "support for XCOM in your airflow config.")
+                    log = LoggingMixin().logger
+                    log.error("Could not serialize the XCOM value into JSON. "
+                              "If you are using pickles instead of JSON "
+                              "for XCOM, then you need to enable pickle "
+                              "support for XCOM in your airflow config.")
                     raise
 
     @classmethod
@@ -4180,10 +4179,11 @@ class XCom(Base):
                 try:
                     result.value = json.loads(result.value.decode('UTF-8'))
                 except ValueError:
-                    logging.error("Could not serialize the XCOM value into JSON. "
-                                    "If you are using pickles instead of JSON "
-                                    "for XCOM, then you need to enable pickle "
-                                    "support for XCOM in your airflow config.")
+                    log = LoggingMixin().logger
+                    log.error("Could not serialize the XCOM value into JSON. "
+                              "If you are using pickles instead of JSON "
+                              "for XCOM, then you need to enable pickle "
+                              "support for XCOM in your airflow config.")
                     raise
         return results
 
@@ -4235,8 +4235,9 @@ class DagStat(Base):
             session.commit()
         except Exception as e:
             session.rollback()
-            logging.warning("Could not update dag stats for {}".format(dag_id))
-            logging.exception(e)
+            log = LoggingMixin().logger
+            log.warning("Could not update dag stats for %s", dag_id)
+            log.exception(e)
 
     @staticmethod
     @provide_session
@@ -4287,8 +4288,9 @@ class DagStat(Base):
             session.commit()
         except Exception as e:
             session.rollback()
-            logging.warning("Could not update dag stat table")
-            logging.exception(e)
+            log = LoggingMixin().logger
+            log.warning("Could not update dag stat table")
+            log.exception(e)
 
     @staticmethod
     @provide_session
@@ -4310,11 +4312,12 @@ class DagStat(Base):
                     session.commit()
                 except Exception as e:
                     session.rollback()
-                    logging.warning("Could not create stat record")
-                    logging.exception(e)
+                    log = LoggingMixin().logger
+                    log.warning("Could not create stat record")
+                    log.exception(e)
 
 
-class DagRun(Base):
+class DagRun(Base, LoggingMixin):
     """
     DagRun describes an instance of a Dag. It can be created
     by the scheduler (for regular runs) or by an external trigger
@@ -4527,8 +4530,7 @@ class DagRun(Base):
 
         tis = self.get_task_instances(session=session)
 
-        logging.info("Updating state for {} considering {} task(s)"
-                     .format(self, len(tis)))
+        self.logger.info("Updating state for %s considering %s task(s)", self, len(tis))
 
         for ti in list(tis):
             # skip in db?
@@ -4574,19 +4576,18 @@ class DagRun(Base):
             # if all roots finished and at least on failed, the run failed
             if (not unfinished_tasks and
                     any(r.state in (State.FAILED, State.UPSTREAM_FAILED) for r in roots)):
-                logging.info('Marking run {} failed'.format(self))
+                self.logger.info('Marking run %s failed', self)
                 self.state = State.FAILED
 
             # if all roots succeeded and no unfinished tasks, the run succeeded
             elif not unfinished_tasks and all(r.state in (State.SUCCESS, State.SKIPPED)
                                               for r in roots):
-                logging.info('Marking run {} successful'.format(self))
+                self.logger.info('Marking run %s successful', self)
                 self.state = State.SUCCESS
 
             # if *all tasks* are deadlocked, the run failed
             elif unfinished_tasks and none_depends_on_past and no_dependencies_met:
-                logging.info(
-                    'Deadlock; marking run {} failed'.format(self))
+                self.logger.info('Deadlock; marking run %s failed', self)
                 self.state = State.FAILED
 
             # finally, if the roots aren't done, the dag is still running

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/bash_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/bash_operator.py b/airflow/operators/bash_operator.py
index 3146cd6..63321fb 100644
--- a/airflow/operators/bash_operator.py
+++ b/airflow/operators/bash_operator.py
@@ -16,7 +16,6 @@
 from builtins import bytes
 import os
 import signal
-import logging
 from subprocess import Popen, STDOUT, PIPE
 from tempfile import gettempdir, NamedTemporaryFile
 
@@ -68,7 +67,7 @@ class BashOperator(BaseOperator):
         which will be cleaned afterwards
         """
         bash_command = self.bash_command
-        logging.info("tmp dir root location: \n" + gettempdir())
+        self.logger.info("Tmp dir root location: \n %s", gettempdir())
         with TemporaryDirectory(prefix='airflowtmp') as tmp_dir:
             with NamedTemporaryFile(dir=tmp_dir, prefix=self.task_id) as f:
 
@@ -76,9 +75,11 @@ class BashOperator(BaseOperator):
                 f.flush()
                 fname = f.name
                 script_location = tmp_dir + "/" + fname
-                logging.info("Temporary script "
-                             "location :{0}".format(script_location))
-                logging.info("Running command: " + bash_command)
+                self.logger.info(
+                    "Temporary script location: %s",
+                    script_location
+                )
+                self.logger.info("Running command: %s", bash_command)
                 sp = Popen(
                     ['bash', fname],
                     stdout=PIPE, stderr=STDOUT,
@@ -87,14 +88,16 @@ class BashOperator(BaseOperator):
 
                 self.sp = sp
 
-                logging.info("Output:")
+                self.logger.info("Output:")
                 line = ''
                 for line in iter(sp.stdout.readline, b''):
                     line = line.decode(self.output_encoding).strip()
-                    logging.info(line)
+                    self.logger.info(line)
                 sp.wait()
-                logging.info("Command exited with "
-                             "return code {0}".format(sp.returncode))
+                self.logger.info(
+                    "Command exited with return code %s",
+                    sp.returncode
+                )
 
                 if sp.returncode:
                     raise AirflowException("Bash command failed")
@@ -103,6 +106,6 @@ class BashOperator(BaseOperator):
             return line
 
     def on_kill(self):
-        logging.info('Sending SIGTERM signal to bash process group')
+        self.logger.info('Sending SIGTERM signal to bash process group')
         os.killpg(os.getpgid(self.sp.pid), signal.SIGTERM)
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/check_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/check_operator.py b/airflow/operators/check_operator.py
index 1cf50da..f263a2c 100644
--- a/airflow/operators/check_operator.py
+++ b/airflow/operators/check_operator.py
@@ -14,7 +14,6 @@
 
 from builtins import zip
 from builtins import str
-import logging
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.base_hook import BaseHook
@@ -72,15 +71,15 @@ class CheckOperator(BaseOperator):
         self.sql = sql
 
     def execute(self, context=None):
-        logging.info('Executing SQL check: ' + self.sql)
+        self.logger.info('Executing SQL check: %s', self.sql)
         records = self.get_db_hook().get_first(self.sql)
-        logging.info("Record: " + str(records))
+        self.logger.info('Record: %s', records)
         if not records:
             raise AirflowException("The query returned None")
         elif not all([bool(r) for r in records]):
             exceptstr = "Test failed.\nQuery:\n{q}\nResults:\n{r!s}"
             raise AirflowException(exceptstr.format(q=self.sql, r=records))
-        logging.info("Success.")
+        self.logger.info("Success.")
 
     def get_db_hook(self):
         return BaseHook.get_hook(conn_id=self.conn_id)
@@ -135,7 +134,7 @@ class ValueCheckOperator(BaseOperator):
         self.has_tolerance = self.tol is not None
 
     def execute(self, context=None):
-        logging.info('Executing SQL check: ' + self.sql)
+        self.logger.info('Executing SQL check: %s', self.sql)
         records = self.get_db_hook().get_first(self.sql)
         if not records:
             raise AirflowException("The query returned None")
@@ -209,9 +208,9 @@ class IntervalCheckOperator(BaseOperator):
 
     def execute(self, context=None):
         hook = self.get_db_hook()
-        logging.info('Executing SQL check: ' + self.sql2)
+        self.logger.info('Executing SQL check: %s', self.sql2)
         row2 = hook.get_first(self.sql2)
-        logging.info('Executing SQL check: ' + self.sql1)
+        self.logger.info('Executing SQL check: %s', self.sql1)
         row1 = hook.get_first(self.sql1)
         if not row2:
             raise AirflowException("The query {q} returned None".format(q=self.sql2))
@@ -231,19 +230,20 @@ class IntervalCheckOperator(BaseOperator):
             else:
                 ratio = float(max(current[m], reference[m])) / \
                     min(current[m], reference[m])
-            logging.info(rlog.format(m, ratio, self.metrics_thresholds[m]))
+            self.logger.info(rlog.format(m, ratio, self.metrics_thresholds[m]))
             ratios[m] = ratio
             test_results[m] = ratio < self.metrics_thresholds[m]
         if not all(test_results.values()):
             failed_tests = [it[0] for it in test_results.items() if not it[1]]
             j = len(failed_tests)
             n = len(self.metrics_sorted)
-            logging.warning(countstr.format(**locals()))
+            self.logger.warning(countstr.format(**locals()))
             for k in failed_tests:
-                logging.warning(fstr.format(k=k, r=ratios[k],
-                                tr=self.metrics_thresholds[k]))
+                self.logger.warning(
+                    fstr.format(k=k, r=ratios[k], tr=self.metrics_thresholds[k])
+                )
             raise AirflowException(estr.format(", ".join(failed_tests)))
-        logging.info("All tests have passed")
+        self.logger.info("All tests have passed")
 
     def get_db_hook(self):
         return BaseHook.get_hook(conn_id=self.conn_id)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/dagrun_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/dagrun_operator.py b/airflow/operators/dagrun_operator.py
index c3ffa1a..bd2862b 100644
--- a/airflow/operators/dagrun_operator.py
+++ b/airflow/operators/dagrun_operator.py
@@ -13,13 +13,11 @@
 # limitations under the License.
 
 from datetime import datetime
-import logging
 
 from airflow.models import BaseOperator, DagBag
 from airflow.utils.decorators import apply_defaults
 from airflow.utils.state import State
 from airflow import settings
-from airflow import configuration as conf
 
 
 class DagRunOrder(object):
@@ -71,9 +69,9 @@ class TriggerDagRunOperator(BaseOperator):
                 state=State.RUNNING,
                 conf=dro.payload,
                 external_trigger=True)
-            logging.info("Creating DagRun {}".format(dr))
+            self.logger.info("Creating DagRun %s", dr)
             session.add(dr)
             session.commit()
             session.close()
         else:
-            logging.info("Criteria not met, moving on")
+            self.logger.info("Criteria not met, moving on")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/docker_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/docker_operator.py b/airflow/operators/docker_operator.py
index ddcc97b..8a333d6 100644
--- a/airflow/operators/docker_operator.py
+++ b/airflow/operators/docker_operator.py
@@ -13,7 +13,7 @@
 # limitations under the License.
 
 import json
-import logging
+
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -134,7 +134,7 @@ class DockerOperator(BaseOperator):
         self.container = None
 
     def execute(self, context):
-        logging.info('Starting docker container from image ' + self.image)
+        self.logger.info('Starting docker container from image %s', self.image)
 
         tls_config = None
         if self.tls_ca_cert and self.tls_client_cert and self.tls_client_key:
@@ -155,10 +155,10 @@ class DockerOperator(BaseOperator):
             image = self.image
 
         if self.force_pull or len(self.cli.images(name=image)) == 0:
-            logging.info('Pulling docker image ' + image)
+            self.logger.info('Pulling docker image %s', image)
             for l in self.cli.pull(image, stream=True):
                 output = json.loads(l.decode('utf-8'))
-                logging.info("{}".format(output['status']))
+                self.logger.info("%s", output['status'])
 
         cpu_shares = int(round(self.cpus * 1024))
 
@@ -184,7 +184,7 @@ class DockerOperator(BaseOperator):
                 line = line.strip()
                 if hasattr(line, 'decode'):
                     line = line.decode('utf-8')
-                logging.info(line)
+                self.logger.info(line)
 
             exit_code = self.cli.wait(self.container['Id'])
             if exit_code != 0:
@@ -202,5 +202,5 @@ class DockerOperator(BaseOperator):
 
     def on_kill(self):
         if self.cli is not None:
-            logging.info('Stopping docker container')
+            self.logger.info('Stopping docker container')
             self.cli.stop(self.container['Id'])

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/generic_transfer.py
----------------------------------------------------------------------
diff --git a/airflow/operators/generic_transfer.py b/airflow/operators/generic_transfer.py
index de3bf73..790749a 100644
--- a/airflow/operators/generic_transfer.py
+++ b/airflow/operators/generic_transfer.py
@@ -11,9 +11,6 @@
 # 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 airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from airflow.hooks.base_hook import BaseHook
@@ -64,15 +61,15 @@ class GenericTransfer(BaseOperator):
     def execute(self, context):
         source_hook = BaseHook.get_hook(self.source_conn_id)
 
-        logging.info("Extracting data from {}".format(self.source_conn_id))
-        logging.info("Executing: \n" + self.sql)
+        self.logger.info("Extracting data from %s", self.source_conn_id)
+        self.logger.info("Executing: \n %s", self.sql)
         results = source_hook.get_records(self.sql)
 
         destination_hook = BaseHook.get_hook(self.destination_conn_id)
         if self.preoperator:
-            logging.info("Running preoperator")
-            logging.info(self.preoperator)
+            self.logger.info("Running preoperator")
+            self.logger.info(self.preoperator)
             destination_hook.run(self.preoperator)
 
-        logging.info("Inserting rows into {}".format(self.destination_conn_id))
+        self.logger.info("Inserting rows into %s", self.destination_conn_id)
         destination_hook.insert_rows(table=self.destination_table, rows=results)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/hive_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/hive_operator.py b/airflow/operators/hive_operator.py
index 06a83e3..983069b 100644
--- a/airflow/operators/hive_operator.py
+++ b/airflow/operators/hive_operator.py
@@ -11,8 +11,6 @@
 # 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
 import re
 
 from airflow.hooks.hive_hooks import HiveCliHook
@@ -95,7 +93,7 @@ class HiveOperator(BaseOperator):
             self.hql = "\n".join(self.hql.split(self.script_begin_tag)[1:])
 
     def execute(self, context):
-        logging.info('Executing: ' + self.hql)
+        self.logger.info('Executing: %s', self.hql)
         self.hook = self.get_hook()
         self.hook.run_cli(hql=self.hql, schema=self.schema,
                           hive_conf=context_to_airflow_vars(context))

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/hive_stats_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/hive_stats_operator.py b/airflow/operators/hive_stats_operator.py
index b31c6b5..025e427 100644
--- a/airflow/operators/hive_stats_operator.py
+++ b/airflow/operators/hive_stats_operator.py
@@ -12,11 +12,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-from builtins import str
 from builtins import zip
 from collections import OrderedDict
 import json
-import logging
 
 from airflow.exceptions import AirflowException
 from airflow.hooks.mysql_hook import MySqlHook
@@ -141,15 +139,15 @@ class HiveStatsCollectionOperator(BaseOperator):
         """.format(**locals())
 
         hook = PrestoHook(presto_conn_id=self.presto_conn_id)
-        logging.info('Executing SQL check: ' + sql)
+        self.logger.info('Executing SQL check: %s', sql)
         row = hook.get_first(hql=sql)
-        logging.info("Record: " + str(row))
+        self.logger.info("Record: %s", row)
         if not row:
             raise AirflowException("The query returned None")
 
         part_json = json.dumps(self.partition, sort_keys=True)
 
-        logging.info("Deleting rows from previous runs if they exist")
+        self.logger.info("Deleting rows from previous runs if they exist")
         mysql = MySqlHook(self.mysql_conn_id)
         sql = """
         SELECT 1 FROM hive_stats
@@ -169,7 +167,7 @@ class HiveStatsCollectionOperator(BaseOperator):
             """.format(**locals())
             mysql.run(sql)
 
-        logging.info("Pivoting and loading cells into the Airflow db")
+        self.logger.info("Pivoting and loading cells into the Airflow db")
         rows = [
             (self.ds, self.dttm, self.table, part_json) +
             (r[0][0], r[0][1], r[1])

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/hive_to_druid.py
----------------------------------------------------------------------
diff --git a/airflow/operators/hive_to_druid.py b/airflow/operators/hive_to_druid.py
index 70d7825..7ac0b02 100644
--- a/airflow/operators/hive_to_druid.py
+++ b/airflow/operators/hive_to_druid.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.hive_hooks import HiveCliHook, HiveMetastoreHook
 from airflow.hooks.druid_hook import DruidHook
 from airflow.models import BaseOperator
@@ -90,7 +87,7 @@ class HiveToDruidTransfer(BaseOperator):
 
     def execute(self, context):
         hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id)
-        logging.info("Extracting data from Hive")
+        self.logger.info("Extracting data from Hive")
         hive_table = 'druid.' + context['task_instance_key_str'].replace('.', '_')
         sql = self.sql.strip().strip(';')
         hql = """\
@@ -104,7 +101,7 @@ class HiveToDruidTransfer(BaseOperator):
         AS
         {sql}
         """.format(**locals())
-        logging.info("Running command:\n {}".format(hql))
+        self.logger.info("Running command:\n %s", hql)
         hive.run_cli(hql)
 
         m = HiveMetastoreHook(self.metastore_conn_id)
@@ -128,15 +125,16 @@ class HiveToDruidTransfer(BaseOperator):
                 columns=columns,
             )
 
-            logging.info("Inserting rows into Druid, hdfs path: {}".format(static_path))
+            self.logger.info("Inserting rows into Druid, hdfs path: %s", static_path)
 
             druid.submit_indexing_job(index_spec)
 
-            logging.info("Load seems to have succeeded!")
+            self.logger.info("Load seems to have succeeded!")
         finally:
-            logging.info(
-                "Cleaning up by dropping the temp "
-                "Hive table {}".format(hive_table))
+            self.logger.info(
+                "Cleaning up by dropping the temp Hive table %s",
+                hive_table
+            )
             hql = "DROP TABLE IF EXISTS {}".format(hive_table)
             hive.run_cli(hql)
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/hive_to_mysql.py
----------------------------------------------------------------------
diff --git a/airflow/operators/hive_to_mysql.py b/airflow/operators/hive_to_mysql.py
index 4a64749..e82a099 100644
--- a/airflow/operators/hive_to_mysql.py
+++ b/airflow/operators/hive_to_mysql.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.hive_hooks import HiveServer2Hook
 from airflow.hooks.mysql_hook import MySqlHook
 from airflow.models import BaseOperator
@@ -80,8 +77,7 @@ class HiveToMySqlTransfer(BaseOperator):
 
     def execute(self, context):
         hive = HiveServer2Hook(hiveserver2_conn_id=self.hiveserver2_conn_id)
-        logging.info("Extracting data from Hive")
-        logging.info(self.sql)
+        self.logger.info("Extracting data from Hive: %s", self.sql)
 
         if self.bulk_load:
             tmpfile = NamedTemporaryFile()
@@ -92,10 +88,10 @@ class HiveToMySqlTransfer(BaseOperator):
 
         mysql = MySqlHook(mysql_conn_id=self.mysql_conn_id)
         if self.mysql_preoperator:
-            logging.info("Running MySQL preoperator")
+            self.logger.info("Running MySQL preoperator")
             mysql.run(self.mysql_preoperator)
 
-        logging.info("Inserting rows into MySQL")
+        self.logger.info("Inserting rows into MySQL")
 
         if self.bulk_load:
             mysql.bulk_load(table=self.mysql_table, tmp_file=tmpfile.name)
@@ -104,7 +100,7 @@ class HiveToMySqlTransfer(BaseOperator):
             mysql.insert_rows(table=self.mysql_table, rows=results)
 
         if self.mysql_postoperator:
-            logging.info("Running MySQL postoperator")
+            self.logger.info("Running MySQL postoperator")
             mysql.run(self.mysql_postoperator)
 
-        logging.info("Done.")
+        self.logger.info("Done.")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/hive_to_samba_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/hive_to_samba_operator.py b/airflow/operators/hive_to_samba_operator.py
index 8f18dd9..d6e6dec 100644
--- a/airflow/operators/hive_to_samba_operator.py
+++ b/airflow/operators/hive_to_samba_operator.py
@@ -11,8 +11,6 @@
 # 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
 import tempfile
 
 from airflow.hooks.hive_hooks import HiveServer2Hook
@@ -55,7 +53,7 @@ class Hive2SambaOperator(BaseOperator):
         samba = SambaHook(samba_conn_id=self.samba_conn_id)
         hive = HiveServer2Hook(hiveserver2_conn_id=self.hiveserver2_conn_id)
         tmpfile = tempfile.NamedTemporaryFile()
-        logging.info("Fetching file from Hive")
+        self.logger.info("Fetching file from Hive")
         hive.to_csv(hql=self.hql, csv_filepath=tmpfile.name)
-        logging.info("Pushing to samba")
+        self.logger.info("Pushing to samba")
         samba.push_from_local(self.destination_filepath, tmpfile.name)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/http_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/http_operator.py b/airflow/operators/http_operator.py
index 9884566..d92c931 100644
--- a/airflow/operators/http_operator.py
+++ b/airflow/operators/http_operator.py
@@ -12,8 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
-
 from airflow.exceptions import AirflowException
 from airflow.hooks.http_hook import HttpHook
 from airflow.models import BaseOperator
@@ -45,7 +43,7 @@ class SimpleHttpOperator(BaseOperator):
         depends on the option that's being modified.
     """
 
-    template_fields = ('endpoint','data',)
+    template_fields = ('endpoint', 'data',)
     template_ext = ()
     ui_color = '#f4a460'
 
@@ -75,7 +73,9 @@ class SimpleHttpOperator(BaseOperator):
 
     def execute(self, context):
         http = HttpHook(self.method, http_conn_id=self.http_conn_id)
-        logging.info("Calling HTTP method")
+
+        self.logger.info("Calling HTTP method")
+
         response = http.run(self.endpoint,
                             self.data,
                             self.headers,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/jdbc_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/jdbc_operator.py b/airflow/operators/jdbc_operator.py
index 28977db..942e312 100644
--- a/airflow/operators/jdbc_operator.py
+++ b/airflow/operators/jdbc_operator.py
@@ -11,11 +11,6 @@
 # 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.
-
-__author__ = 'janomar'
-
-import logging
-
 from airflow.hooks.jdbc_hook import JdbcHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -60,6 +55,6 @@ class JdbcOperator(BaseOperator):
         self.autocommit = autocommit
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         self.hook = JdbcHook(jdbc_conn_id=self.jdbc_conn_id)
         self.hook.run(self.sql, self.autocommit, parameters=self.parameters)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/latest_only_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/latest_only_operator.py b/airflow/operators/latest_only_operator.py
index 909a211..58f7e67 100644
--- a/airflow/operators/latest_only_operator.py
+++ b/airflow/operators/latest_only_operator.py
@@ -13,7 +13,6 @@
 # limitations under the License.
 
 import datetime
-import logging
 
 from airflow.models import BaseOperator, SkipMixin
 
@@ -33,29 +32,29 @@ class LatestOnlyOperator(BaseOperator, SkipMixin):
         # If the DAG Run is externally triggered, then return without
         # skipping downstream tasks
         if context['dag_run'] and context['dag_run'].external_trigger:
-            logging.info("""Externally triggered DAG_Run:
-                         allowing execution to proceed.""")
+            self.logger.info("Externally triggered DAG_Run: allowing execution to proceed.")
             return
 
         now = datetime.datetime.now()
         left_window = context['dag'].following_schedule(
             context['execution_date'])
         right_window = context['dag'].following_schedule(left_window)
-        logging.info(
-            'Checking latest only with left_window: %s right_window: %s '
-            'now: %s', left_window, right_window, now)
+        self.logger.info(
+            'Checking latest only with left_window: %s right_window: %s now: %s',
+            left_window, right_window, now
+        )
 
         if not left_window < now <= right_window:
-            logging.info('Not latest execution, skipping downstream.')
+            self.logger.info('Not latest execution, skipping downstream.')
 
             downstream_tasks = context['task'].get_flat_relatives(upstream=False)
-            logging.debug("Downstream task_ids {}".format(downstream_tasks))
+            self.logger.debug("Downstream task_ids %s", downstream_tasks)
 
             if downstream_tasks:
                 self.skip(context['dag_run'],
                           context['ti'].execution_date,
                           downstream_tasks)
 
-            logging.info('Done.')
+            self.logger.info('Done.')
         else:
-            logging.info('Latest, allowing execution to proceed.')
+            self.logger.info('Latest, allowing execution to proceed.')

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/mssql_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/mssql_operator.py b/airflow/operators/mssql_operator.py
index 9ae2fff..bc0822f 100644
--- a/airflow/operators/mssql_operator.py
+++ b/airflow/operators/mssql_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.mssql_hook import MsSqlHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -47,7 +44,7 @@ class MsSqlOperator(BaseOperator):
         self.database = database
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         hook = MsSqlHook(mssql_conn_id=self.mssql_conn_id,
                          schema=self.database)
         hook.run(self.sql, autocommit=self.autocommit,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/mssql_to_hive.py
----------------------------------------------------------------------
diff --git a/airflow/operators/mssql_to_hive.py b/airflow/operators/mssql_to_hive.py
index a0a2e10..719ddd2 100644
--- a/airflow/operators/mssql_to_hive.py
+++ b/airflow/operators/mssql_to_hive.py
@@ -15,11 +15,9 @@
 from builtins import chr
 from collections import OrderedDict
 import unicodecsv as csv
-import logging
 from tempfile import NamedTemporaryFile
 import pymssql
 
-
 from airflow.hooks.hive_hooks import HiveCliHook
 from airflow.hooks.mssql_hook import MsSqlHook
 from airflow.models import BaseOperator
@@ -104,7 +102,7 @@ class MsSqlToHiveTransfer(BaseOperator):
         hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id)
         mssql = MsSqlHook(mssql_conn_id=self.mssql_conn_id)
 
-        logging.info("Dumping Microsoft SQL Server query results to local file")
+        self.logger.info("Dumping Microsoft SQL Server query results to local file")
         conn = mssql.get_conn()
         cursor = conn.cursor()
         cursor.execute(self.sql)
@@ -120,7 +118,7 @@ class MsSqlToHiveTransfer(BaseOperator):
             f.flush()
             cursor.close()
             conn.close()
-            logging.info("Loading file into Hive")
+            self.logger.info("Loading file into Hive")
             hive.load_file(
                 f.name,
                 self.hive_table,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/mysql_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/mysql_operator.py b/airflow/operators/mysql_operator.py
index 156ada8..923eaf8 100644
--- a/airflow/operators/mysql_operator.py
+++ b/airflow/operators/mysql_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.mysql_hook import MySqlHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -49,7 +46,7 @@ class MySqlOperator(BaseOperator):
         self.database = database
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         hook = MySqlHook(mysql_conn_id=self.mysql_conn_id,
                          schema=self.database)
         hook.run(

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/mysql_to_hive.py
----------------------------------------------------------------------
diff --git a/airflow/operators/mysql_to_hive.py b/airflow/operators/mysql_to_hive.py
index ad3ecae..fde92b5 100644
--- a/airflow/operators/mysql_to_hive.py
+++ b/airflow/operators/mysql_to_hive.py
@@ -15,7 +15,6 @@
 from builtins import chr
 from collections import OrderedDict
 import unicodecsv as csv
-import logging
 from tempfile import NamedTemporaryFile
 import MySQLdb
 
@@ -111,7 +110,7 @@ class MySqlToHiveTransfer(BaseOperator):
         hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id)
         mysql = MySqlHook(mysql_conn_id=self.mysql_conn_id)
 
-        logging.info("Dumping MySQL query results to local file")
+        self.logger.info("Dumping MySQL query results to local file")
         conn = mysql.get_conn()
         cursor = conn.cursor()
         cursor.execute(self.sql)
@@ -124,7 +123,7 @@ class MySqlToHiveTransfer(BaseOperator):
             f.flush()
             cursor.close()
             conn.close()
-            logging.info("Loading file into Hive")
+            self.logger.info("Loading file into Hive")
             hive.load_file(
                 f.name,
                 self.hive_table,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/oracle_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/oracle_operator.py b/airflow/operators/oracle_operator.py
index ab7bdb2..f87bbf9 100644
--- a/airflow/operators/oracle_operator.py
+++ b/airflow/operators/oracle_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.oracle_hook import OracleHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -45,7 +42,7 @@ class OracleOperator(BaseOperator):
         self.parameters = parameters
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         hook = OracleHook(oracle_conn_id=self.oracle_conn_id)
         hook.run(
             self.sql,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/pig_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/pig_operator.py b/airflow/operators/pig_operator.py
index 4a21ecc..cdce48a 100644
--- a/airflow/operators/pig_operator.py
+++ b/airflow/operators/pig_operator.py
@@ -11,8 +11,6 @@
 # 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
 import re
 
 from airflow.hooks.pig_hook import PigCliHook
@@ -61,7 +59,7 @@ class PigOperator(BaseOperator):
                 "(\$([a-zA-Z_][a-zA-Z0-9_]*))", "{{ \g<2> }}", self.pig)
 
     def execute(self, context):
-        logging.info('Executing: ' + self.pig)
+        self.logger.info('Executing: %s', self.pig)
         self.hook = self.get_hook()
         self.hook.run_cli(pig=self.pig)
 

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/postgres_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/postgres_operator.py b/airflow/operators/postgres_operator.py
index 0de5aa5..55c1573 100644
--- a/airflow/operators/postgres_operator.py
+++ b/airflow/operators/postgres_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.postgres_hook import PostgresHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -52,7 +49,7 @@ class PostgresOperator(BaseOperator):
         self.database = database
 
     def execute(self, context):
-        logging.info('Executing: ' + str(self.sql))
+        self.logger.info('Executing: %s', self.sql)
         self.hook = PostgresHook(postgres_conn_id=self.postgres_conn_id,
                                  schema=self.database)
         self.hook.run(self.sql, self.autocommit, parameters=self.parameters)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/presto_to_mysql.py
----------------------------------------------------------------------
diff --git a/airflow/operators/presto_to_mysql.py b/airflow/operators/presto_to_mysql.py
index 7ff2ad6..48158ca 100644
--- a/airflow/operators/presto_to_mysql.py
+++ b/airflow/operators/presto_to_mysql.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.presto_hook import PrestoHook
 from airflow.hooks.mysql_hook import MySqlHook
 from airflow.models import BaseOperator
@@ -64,15 +61,14 @@ class PrestoToMySqlTransfer(BaseOperator):
 
     def execute(self, context):
         presto = PrestoHook(presto_conn_id=self.presto_conn_id)
-        logging.info("Extracting data from Presto")
-        logging.info(self.sql)
+        self.logger.info("Extracting data from Presto: %s", self.sql)
         results = presto.get_records(self.sql)
 
         mysql = MySqlHook(mysql_conn_id=self.mysql_conn_id)
         if self.mysql_preoperator:
-            logging.info("Running MySQL preoperator")
-            logging.info(self.mysql_preoperator)
+            self.logger.info("Running MySQL preoperator")
+            self.logger.info(self.mysql_preoperator)
             mysql.run(self.mysql_preoperator)
 
-        logging.info("Inserting rows into MySQL")
+        self.logger.info("Inserting rows into MySQL")
         mysql.insert_rows(table=self.mysql_table, rows=results)

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/python_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/python_operator.py b/airflow/operators/python_operator.py
index bef9bb0..552996f 100644
--- a/airflow/operators/python_operator.py
+++ b/airflow/operators/python_operator.py
@@ -11,10 +11,6 @@
 # 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.
-
-from builtins import str
-import logging
-
 from airflow.exceptions import AirflowException
 from airflow.models import BaseOperator, SkipMixin
 from airflow.utils.decorators import apply_defaults
@@ -78,7 +74,7 @@ class PythonOperator(BaseOperator):
             self.op_kwargs = context
 
         return_value = self.python_callable(*self.op_args, **self.op_kwargs)
-        logging.info("Done. Returned value was: " + str(return_value))
+        self.logger.info("Done. Returned value was: %s", return_value)
         return return_value
 
 
@@ -103,17 +99,17 @@ class BranchPythonOperator(PythonOperator, SkipMixin):
     """
     def execute(self, context):
         branch = super(BranchPythonOperator, self).execute(context)
-        logging.info("Following branch {}".format(branch))
-        logging.info("Marking other directly downstream tasks as skipped")
+        self.logger.info("Following branch %s", branch)
+        self.logger.info("Marking other directly downstream tasks as skipped")
 
         downstream_tasks = context['task'].downstream_list
-        logging.debug("Downstream task_ids {}".format(downstream_tasks))
+        self.logger.debug("Downstream task_ids %s", downstream_tasks)
 
         skip_tasks = [t for t in downstream_tasks if t.task_id != branch]
         if downstream_tasks:
             self.skip(context['dag_run'], context['ti'].execution_date, skip_tasks)
 
-        logging.info("Done.")
+        self.logger.info("Done.")
 
 
 class ShortCircuitOperator(PythonOperator, SkipMixin):
@@ -130,18 +126,18 @@ class ShortCircuitOperator(PythonOperator, SkipMixin):
     """
     def execute(self, context):
         condition = super(ShortCircuitOperator, self).execute(context)
-        logging.info("Condition result is {}".format(condition))
+        self.logger.info("Condition result is %s", condition)
 
         if condition:
-            logging.info('Proceeding with downstream tasks...')
+            self.logger.info('Proceeding with downstream tasks...')
             return
 
-        logging.info('Skipping downstream tasks...')
+        self.logger.info('Skipping downstream tasks...')
 
         downstream_tasks = context['task'].get_flat_relatives(upstream=False)
-        logging.debug("Downstream task_ids {}".format(downstream_tasks))
+        self.logger.debug("Downstream task_ids %s", downstream_tasks)
 
         if downstream_tasks:
             self.skip(context['dag_run'], context['ti'].execution_date, downstream_tasks)
 
-        logging.info("Done.")
+        self.logger.info("Done.")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/redshift_to_s3_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/redshift_to_s3_operator.py b/airflow/operators/redshift_to_s3_operator.py
index fda88d9..e25d613 100644
--- a/airflow/operators/redshift_to_s3_operator.py
+++ b/airflow/operators/redshift_to_s3_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.postgres_hook import PostgresHook
 from airflow.hooks.S3_hook import S3Hook
 from airflow.models import BaseOperator
@@ -71,9 +68,9 @@ class RedshiftToS3Transfer(BaseOperator):
         self.hook = PostgresHook(postgres_conn_id=self.redshift_conn_id)
         self.s3 = S3Hook(s3_conn_id=self.s3_conn_id)
         a_key, s_key = self.s3.get_credentials()
-        unload_options = ('\n\t\t\t').join(self.unload_options)
+        unload_options = '\n\t\t\t'.join(self.unload_options)
 
-        logging.info("Retrieving headers from %s.%s..." % (self.schema, self.table))
+        self.logger.info("Retrieving headers from %s.%s...", self.schema, self.table)
 
         columns_query = """SELECT column_name
                             FROM information_schema.columns
@@ -86,9 +83,9 @@ class RedshiftToS3Transfer(BaseOperator):
         cursor.execute(columns_query)
         rows = cursor.fetchall()
         columns = map(lambda row: row[0], rows)
-        column_names = (', ').join(map(lambda c: "\\'{0}\\'".format(c), columns))
-        column_castings = (', ').join(map(lambda c: "CAST({0} AS text) AS {0}".format(c),
-                                            columns))
+        column_names = ', '.join(map(lambda c: "\\'{0}\\'".format(c), columns))
+        column_castings = ', '.join(map(lambda c: "CAST({0} AS text) AS {0}".format(c),
+                                        columns))
 
         unload_query = """
                         UNLOAD ('SELECT {0}
@@ -102,6 +99,6 @@ class RedshiftToS3Transfer(BaseOperator):
                         """.format(column_names, column_castings, self.schema, self.table,
                                 self.s3_bucket, self.s3_key, a_key, s_key, unload_options)
 
-        logging.info('Executing UNLOAD command...')
+        self.logger.info('Executing UNLOAD command...')
         self.hook.run(unload_query, self.autocommit)
-        logging.info("UNLOAD command complete...")
+        self.logger.info("UNLOAD command complete...")

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/s3_file_transform_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/s3_file_transform_operator.py b/airflow/operators/s3_file_transform_operator.py
index 1cdd0e5..5de5127 100644
--- a/airflow/operators/s3_file_transform_operator.py
+++ b/airflow/operators/s3_file_transform_operator.py
@@ -12,7 +12,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-import logging
 from tempfile import NamedTemporaryFile
 import subprocess
 
@@ -75,15 +74,15 @@ class S3FileTransformOperator(BaseOperator):
     def execute(self, context):
         source_s3 = S3Hook(s3_conn_id=self.source_s3_conn_id)
         dest_s3 = S3Hook(s3_conn_id=self.dest_s3_conn_id)
-        logging.info("Downloading source S3 file {0}"
-                     "".format(self.source_s3_key))
+        self.logger.info("Downloading source S3 file %s", self.source_s3_key)
         if not source_s3.check_for_key(self.source_s3_key):
-            raise AirflowException("The source key {0} does not exist"
-                            "".format(self.source_s3_key))
+            raise AirflowException("The source key {0} does not exist".format(self.source_s3_key))
         source_s3_key_object = source_s3.get_key(self.source_s3_key)
         with NamedTemporaryFile("w") as f_source, NamedTemporaryFile("w") as f_dest:
-            logging.info("Dumping S3 file {0} contents to local file {1}"
-                         "".format(self.source_s3_key, f_source.name))
+            self.logger.info(
+                "Dumping S3 file %s contents to local file %s",
+                self.source_s3_key, f_source.name
+            )
             source_s3_key_object.get_contents_to_file(f_source)
             f_source.flush()
             source_s3.connection.close()
@@ -91,21 +90,20 @@ class S3FileTransformOperator(BaseOperator):
                 [self.transform_script, f_source.name, f_dest.name],
                 stdout=subprocess.PIPE, stderr=subprocess.PIPE)
             (transform_script_stdoutdata, transform_script_stderrdata) = transform_script_process.communicate()
-            logging.info("Transform script stdout "
-                         "" + transform_script_stdoutdata)
+            self.logger.info("Transform script stdout %s", transform_script_stdoutdata)
             if transform_script_process.returncode > 0:
-                raise AirflowException("Transform script failed "
-                                "" + transform_script_stderrdata)
+                raise AirflowException("Transform script failed %s", transform_script_stderrdata)
             else:
-                logging.info("Transform script successful."
-                             "Output temporarily located at {0}"
-                             "".format(f_dest.name))
-            logging.info("Uploading transformed file to S3")
+                self.logger.info(
+                    "Transform script successful. Output temporarily located at %s",
+                    f_dest.name
+                )
+            self.logger.info("Uploading transformed file to S3")
             f_dest.flush()
             dest_s3.load_file(
                 filename=f_dest.name,
                 key=self.dest_s3_key,
                 replace=self.replace
             )
-            logging.info("Upload successful")
+            self.logger.info("Upload successful")
             dest_s3.connection.close()

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/s3_to_hive_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/s3_to_hive_operator.py b/airflow/operators/s3_to_hive_operator.py
index 7ae0616..68fe903 100644
--- a/airflow/operators/s3_to_hive_operator.py
+++ b/airflow/operators/s3_to_hive_operator.py
@@ -14,7 +14,6 @@
 
 from builtins import next
 from builtins import zip
-import logging
 from tempfile import NamedTemporaryFile
 from airflow.utils.file import TemporaryDirectory
 import gzip
@@ -29,6 +28,7 @@ from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from airflow.utils.compression import uncompress_file
 
+
 class S3ToHiveTransfer(BaseOperator):
     """
     Moves data from S3 to Hive. The operator downloads a file from S3,
@@ -129,7 +129,7 @@ class S3ToHiveTransfer(BaseOperator):
         # Downloading file from S3
         self.s3 = S3Hook(s3_conn_id=self.s3_conn_id)
         self.hive = HiveCliHook(hive_cli_conn_id=self.hive_cli_conn_id)
-        logging.info("Downloading S3 file")
+        self.logger.info("Downloading S3 file")
 
         if self.wildcard_match:
             if not self.s3.check_for_wildcard_key(self.s3_key):
@@ -146,13 +146,13 @@ class S3ToHiveTransfer(BaseOperator):
                 NamedTemporaryFile(mode="w",
                                    dir=tmp_dir,
                                    suffix=file_ext) as f:
-            logging.info("Dumping S3 key {0} contents to local"
-                         " file {1}".format(s3_key_object.key, f.name))
+            self.logger.info("Dumping S3 key {0} contents to local file {1}"
+                             .format(s3_key_object.key, f.name))
             s3_key_object.get_contents_to_file(f)
             f.flush()
             self.s3.connection.close()
             if not self.headers:
-                logging.info("Loading file {0} into Hive".format(f.name))
+                self.logger.info("Loading file %s into Hive", f.name)
                 self.hive.load_file(
                     f.name,
                     self.hive_table,
@@ -165,11 +165,11 @@ class S3ToHiveTransfer(BaseOperator):
             else:
                 # Decompressing file
                 if self.input_compressed:
-                    logging.info("Uncompressing file {0}".format(f.name))
+                    self.logger.info("Uncompressing file %s", f.name)
                     fn_uncompressed = uncompress_file(f.name,
                                                       file_ext,
                                                       tmp_dir)
-                    logging.info("Uncompressed to {0}".format(fn_uncompressed))
+                    self.logger.info("Uncompressed to %s", fn_uncompressed)
                     # uncompressed file available now so deleting
                     # compressed file to save disk space
                     f.close()
@@ -178,20 +178,19 @@ class S3ToHiveTransfer(BaseOperator):
 
                 # Testing if header matches field_dict
                 if self.check_headers:
-                    logging.info("Matching file header against field_dict")
+                    self.logger.info("Matching file header against field_dict")
                     header_list = self._get_top_row_as_list(fn_uncompressed)
                     if not self._match_headers(header_list):
                         raise AirflowException("Header check failed")
 
                 # Deleting top header row
-                logging.info("Removing header from file {0}".
-                             format(fn_uncompressed))
+                self.logger.info("Removing header from file %s", fn_uncompressed)
                 headless_file = (
                     self._delete_top_row_and_compress(fn_uncompressed,
                                                       file_ext,
                                                       tmp_dir))
-                logging.info("Headless file {0}".format(headless_file))
-                logging.info("Loading file {0} into Hive".format(headless_file))
+                self.logger.info("Headless file %s", headless_file)
+                self.logger.info("Loading file %s into Hive", headless_file)
                 self.hive.load_file(headless_file,
                                     self.hive_table,
                                     field_dict=self.field_dict,
@@ -212,18 +211,18 @@ class S3ToHiveTransfer(BaseOperator):
             raise AirflowException("Unable to retrieve header row from file")
         field_names = self.field_dict.keys()
         if len(field_names) != len(header_list):
-            logging.warning("Headers count mismatch"
-                            "File headers:\n {header_list}\n"
-                            "Field names: \n {field_names}\n"
-                            "".format(**locals()))
+            self.logger.warning("Headers count mismatch"
+                              "File headers:\n {header_list}\n"
+                              "Field names: \n {field_names}\n"
+                              "".format(**locals()))
             return False
         test_field_match = [h1.lower() == h2.lower()
                             for h1, h2 in zip(header_list, field_names)]
         if not all(test_field_match):
-            logging.warning("Headers do not match field names"
-                            "File headers:\n {header_list}\n"
-                            "Field names: \n {field_names}\n"
-                            "".format(**locals()))
+            self.logger.warning("Headers do not match field names"
+                              "File headers:\n {header_list}\n"
+                              "Field names: \n {field_names}\n"
+                              "".format(**locals()))
             return False
         else:
             return True

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/sensors.py
----------------------------------------------------------------------
diff --git a/airflow/operators/sensors.py b/airflow/operators/sensors.py
index 409c18d..ea301dc 100644
--- a/airflow/operators/sensors.py
+++ b/airflow/operators/sensors.py
@@ -14,12 +14,14 @@
 
 from __future__ import print_function
 from future import standard_library
+
+from airflow.utils.log.LoggingMixin import LoggingMixin
+
 standard_library.install_aliases()
 from builtins import str
 from past.builtins import basestring
 
 from datetime import datetime
-import logging
 from urllib.parse import urlparse
 from time import sleep
 import re
@@ -80,7 +82,7 @@ class BaseSensorOperator(BaseOperator):
                 else:
                     raise AirflowSensorTimeout('Snap. Time is OUT.')
             sleep(self.poke_interval)
-        logging.info("Success criteria met. Exiting.")
+        self.logger.info("Success criteria met. Exiting.")
 
 
 class SqlSensor(BaseSensorOperator):
@@ -106,7 +108,7 @@ class SqlSensor(BaseSensorOperator):
     def poke(self, context):
         hook = BaseHook.get_connection(self.conn_id).get_hook()
 
-        logging.info('Poking: ' + self.sql)
+        self.logger.info('Poking: %s', self.sql)
         records = hook.get_records(self.sql)
         if not records:
             return False
@@ -235,7 +237,7 @@ class ExternalTaskSensor(BaseSensorOperator):
         serialized_dttm_filter = ','.join(
             [datetime.isoformat() for datetime in dttm_filter])
 
-        logging.info(
+        self.logger.info(
             'Poking for '
             '{self.external_dag_id}.'
             '{self.external_task_id} on '
@@ -311,7 +313,7 @@ class NamedHivePartitionSensor(BaseSensorOperator):
 
             schema, table, partition = self.parse_partition_name(partition)
 
-            logging.info(
+            self.logger.info(
                 'Poking for {schema}.{table}/{partition}'.format(**locals())
             )
             return self.hook.check_for_named_partition(
@@ -369,7 +371,7 @@ class HivePartitionSensor(BaseSensorOperator):
     def poke(self, context):
         if '.' in self.table:
             self.schema, self.table = self.table.split('.')
-        logging.info(
+        self.logger.info(
             'Poking for table {self.schema}.{self.table}, '
             'partition {self.partition}'.format(**locals()))
         if not hasattr(self, 'hook'):
@@ -415,10 +417,11 @@ class HdfsSensor(BaseSensorOperator):
         :return: (bool) depending on the matching criteria
         """
         if size:
-            logging.debug('Filtering for file size >= %s in files: %s', size, map(lambda x: x['path'], result))
+            log = LoggingMixin().logger
+            log.debug('Filtering for file size >= %s in files: %s', size, map(lambda x: x['path'], result))
             size *= settings.MEGABYTE
             result = [x for x in result if x['length'] >= size]
-            logging.debug('HdfsSensor.poke: after size filter result is %s', result)
+            log.debug('HdfsSensor.poke: after size filter result is %s', result)
         return result
 
     @staticmethod
@@ -432,31 +435,33 @@ class HdfsSensor(BaseSensorOperator):
         :return: (list) of dicts which were not removed
         """
         if ignore_copying:
+            log = LoggingMixin().logger
             regex_builder = "^.*\.(%s$)$" % '$|'.join(ignored_ext)
             ignored_extentions_regex = re.compile(regex_builder)
-            logging.debug('Filtering result for ignored extentions: %s in files %s', ignored_extentions_regex.pattern,
-                          map(lambda x: x['path'], result))
+            log.debug(
+                'Filtering result for ignored extensions: %s in files %s',
+                ignored_extentions_regex.pattern, map(lambda x: x['path'], result)
+            )
             result = [x for x in result if not ignored_extentions_regex.match(x['path'])]
-            logging.debug('HdfsSensor.poke: after ext filter result is %s', result)
+            log.debug('HdfsSensor.poke: after ext filter result is %s', result)
         return result
 
     def poke(self, context):
         sb = self.hook(self.hdfs_conn_id).get_conn()
-        logging.getLogger("snakebite").setLevel(logging.WARNING)
-        logging.info('Poking for file {self.filepath} '.format(**locals()))
+        self.logger.info('Poking for file {self.filepath}'.format(**locals()))
         try:
             # IMOO it's not right here, as there no raise of any kind.
             # if the filepath is let's say '/data/mydirectory', it's correct but if it is '/data/mydirectory/*',
             # it's not correct as the directory exists and sb does not raise any error
             # here is a quick fix
             result = [f for f in sb.ls([self.filepath], include_toplevel=False)]
-            logging.debug('HdfsSensor.poke: result is %s', result)
+            self.logger.debug('HdfsSensor.poke: result is %s', result)
             result = self.filter_for_ignored_ext(result, self.ignored_ext, self.ignore_copying)
             result = self.filter_for_filesize(result, self.file_size)
             return bool(result)
         except:
             e = sys.exc_info()
-            logging.debug("Caught an exception !: %s", str(e))
+            self.logger.debug("Caught an exception !: %s", str(e))
             return False
 
 
@@ -479,8 +484,7 @@ class WebHdfsSensor(BaseSensorOperator):
     def poke(self, context):
         from airflow.hooks.webhdfs_hook import WebHDFSHook
         c = WebHDFSHook(self.webhdfs_conn_id)
-        logging.info(
-            'Poking for file {self.filepath} '.format(**locals()))
+        self.logger.info('Poking for file {self.filepath}'.format(**locals()))
         return c.check_for_path(hdfs_path=self.filepath)
 
 
@@ -531,7 +535,7 @@ class S3KeySensor(BaseSensorOperator):
         from airflow.hooks.S3_hook import S3Hook
         hook = S3Hook(s3_conn_id=self.s3_conn_id)
         full_url = "s3://" + self.bucket_name + "/" + self.bucket_key
-        logging.info('Poking for key : {full_url}'.format(**locals()))
+        self.logger.info('Poking for key : {full_url}'.format(**locals()))
         if self.wildcard_match:
             return hook.check_for_wildcard_key(self.bucket_key,
                                                self.bucket_name)
@@ -573,7 +577,7 @@ class S3PrefixSensor(BaseSensorOperator):
         self.s3_conn_id = s3_conn_id
 
     def poke(self, context):
-        logging.info('Poking for prefix : {self.prefix}\n'
+        self.logger.info('Poking for prefix : {self.prefix}\n'
                      'in bucket s3://{self.bucket_name}'.format(**locals()))
         from airflow.hooks.S3_hook import S3Hook
         hook = S3Hook(s3_conn_id=self.s3_conn_id)
@@ -598,8 +602,7 @@ class TimeSensor(BaseSensorOperator):
         self.target_time = target_time
 
     def poke(self, context):
-        logging.info(
-            'Checking if the time ({0}) has come'.format(self.target_time))
+        self.logger.info('Checking if the time (%s) has come', self.target_time)
         return datetime.now().time() > self.target_time
 
 
@@ -624,7 +627,7 @@ class TimeDeltaSensor(BaseSensorOperator):
         dag = context['dag']
         target_dttm = dag.following_schedule(context['execution_date'])
         target_dttm += self.delta
-        logging.info('Checking if the time ({0}) has come'.format(target_dttm))
+        self.logger.info('Checking if the time (%s) has come', target_dttm)
         return datetime.now() > target_dttm
 
 
@@ -676,7 +679,7 @@ class HttpSensor(BaseSensorOperator):
             http_conn_id=http_conn_id)
 
     def poke(self, context):
-        logging.info('Poking: ' + self.endpoint)
+        self.logger.info('Poking: %s', self.endpoint)
         try:
             response = self.hook.run(self.endpoint,
                                      data=self.request_params,

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/slack_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/slack_operator.py b/airflow/operators/slack_operator.py
index 86659d9..4f2d7bc 100644
--- a/airflow/operators/slack_operator.py
+++ b/airflow/operators/slack_operator.py
@@ -12,12 +12,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+import json
+
 from slackclient import SlackClient
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
 from airflow.exceptions import AirflowException
-import json
-import logging
 
 
 class SlackAPIOperator(BaseOperator):
@@ -66,8 +66,9 @@ class SlackAPIOperator(BaseOperator):
         sc = SlackClient(self.token)
         rc = sc.api_call(self.method, **self.api_params)
         if not rc['ok']:
-            logging.error("Slack API call failed ({})".format(rc['error']))
-            raise AirflowException("Slack API call failed: ({})".format(rc['error']))
+            msg = "Slack API call failed (%s)".format(rc['error'])
+            self.logger.error(msg)
+            raise AirflowException(msg)
 
 
 class SlackAPIPostOperator(SlackAPIOperator):

http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/a7a51890/airflow/operators/sqlite_operator.py
----------------------------------------------------------------------
diff --git a/airflow/operators/sqlite_operator.py b/airflow/operators/sqlite_operator.py
index 0ff4d05..7c85847 100644
--- a/airflow/operators/sqlite_operator.py
+++ b/airflow/operators/sqlite_operator.py
@@ -11,9 +11,6 @@
 # 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 airflow.hooks.sqlite_hook import SqliteHook
 from airflow.models import BaseOperator
 from airflow.utils.decorators import apply_defaults
@@ -44,6 +41,6 @@ class SqliteOperator(BaseOperator):
         self.parameters = parameters or []
 
     def execute(self, context):
-        logging.info('Executing: ' + self.sql)
+        self.logger.info('Executing: %s', self.sql)
         hook = SqliteHook(sqlite_conn_id=self.sqlite_conn_id)
         hook.run(self.sql, parameters=self.parameters)