You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2020/08/07 09:08:03 UTC

[airflow] branch master updated: Pylint checks should be way faster now (#10207)

This is an automated email from the ASF dual-hosted git repository.

potiuk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new 9e3b7d9  Pylint checks should be way faster now (#10207)
9e3b7d9 is described below

commit 9e3b7d9a1eba3eed7d2a1b355b9696a3ee0b8f5c
Author: Jarek Potiuk <ja...@polidea.com>
AuthorDate: Fri Aug 7 11:07:15 2020 +0200

    Pylint checks should be way faster now (#10207)
    
    * Pylint checks should be way faster now
    
    Instead of running separate pylint checks for tests and main source
    we are running a single check now. This is possible thanks to a
    nice hack - we have pylint plugin that injects the right
    "# pylint: disable=" comment for all test files while reading
    the file content by astroid (just before tokenization)
    
    Thanks to that we can also separate out pylint checks
    to a separate job in CI - this way all pylint checks will
    be run in parallel to all other checks effectively halfing
    the time needed to get the static check feedback and potentially
    cancelling other jobs much faster.
    
    * fixup! Pylint checks should be way faster now
---
 .github/workflows/cancel_other_workflow_runs.yml   |  4 +-
 .github/workflows/ci.yml                           | 33 ++++++++++--
 .pre-commit-config.yaml                            | 14 ++---
 STATIC_CODE_CHECKS.rst                             | 12 ++---
 airflow/jobs/scheduler_job.py                      |  2 +-
 pylintrc                                           |  2 +-
 .../{run_pylint_main.sh => run_pylint.sh}          |  2 -
 scripts/ci/in_container/run_pylint_tests.sh        | 51 ------------------
 ..._commit_pylint_main.sh => pre_commit_pylint.sh} |  4 +-
 scripts/ci/pre_commit/pre_commit_pylint_tests.sh   | 22 --------
 .../{ci_pylint_main.sh => ci_pylint.sh}            | 10 ++--
 scripts/ci/static_checks/ci_pylint_tests.sh        | 57 --------------------
 tests/airflow_pylint/disable_checks_for_tests.py   | 60 ++++++++++++++++++++++
 tests/dags/test_logging_in_dag.py                  |  5 ++
 tests/dags/test_task_view_type_check.py            |  3 ++
 15 files changed, 116 insertions(+), 165 deletions(-)

diff --git a/.github/workflows/cancel_other_workflow_runs.yml b/.github/workflows/cancel_other_workflow_runs.yml
index 1ea965b..5431918 100644
--- a/.github/workflows/cancel_other_workflow_runs.yml
+++ b/.github/workflows/cancel_other_workflow_runs.yml
@@ -32,5 +32,5 @@ jobs:
           token: ${{ secrets.GITHUB_TOKEN }}
           workflow: ci.yml
           failFastJobNames: >
-            ["^Static checks$", "^Build docs$", "^Backport packages$",
-             "^Checks: Helm tests$", "^Build prod image .*", "^Test OpenAPI*"]
+            ["^Static checks.*", "^Build docs$", "^Backport packages$",
+             "^Checks: Helm tests$", "^Build prod image .*", "^Test OpenAPI.*"]
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 48c9353..d3c9d87 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -55,11 +55,12 @@ jobs:
 
   static-checks:
     timeout-minutes: 60
-    name: "Static checks"
+    name: "Static checks: no pylint"
     runs-on: ubuntu-latest
     needs: [cancel-previous-workflow-run]
     env:
       MOUNT_SOURCE_DIR_FOR_STATIC_CHECKS: "true"
+      SKIP: "pylint"
     steps:
       - uses: actions/checkout@v2
       - uses: actions/setup-python@v2
@@ -68,7 +69,7 @@ jobs:
       - name: Cache pre-commit env
         uses: actions/cache@v2
         env:
-          cache-name: cache-pre-commit-v1
+          cache-name: cache-pre-commit-no-pylint-v1
         with:
           path: ~/.cache/pre-commit
           key: ${{ env.cache-name }}-${{ github.job }}-${{ hashFiles('.pre-commit-config.yaml') }}
@@ -76,9 +77,35 @@ jobs:
         run: ./scripts/ci/tools/ci_free_space_on_ci.sh
       - name: "Build CI image"
         run: ./scripts/ci/images/ci_prepare_ci_image_on_ci.sh
-      - name: "Static checks"
+      - name: "Static checks: no pylint"
         run: ./scripts/ci/static_checks/ci_run_static_checks.sh
 
+  static-checks-pylint:
+    timeout-minutes: 60
+    name: "Static checks: pylint"
+    runs-on: ubuntu-latest
+    needs: [cancel-previous-workflow-run]
+    env:
+      MOUNT_SOURCE_DIR_FOR_STATIC_CHECKS: "true"
+    steps:
+      - uses: actions/checkout@v2
+      - uses: actions/setup-python@v2
+        with:
+          python-version: '3.7'
+      - name: Cache pre-commit env
+        uses: actions/cache@v2
+        env:
+          cache-name: cache-pre-commit-pylint-v1
+        with:
+          path: ~/.cache/pre-commit
+          key: ${{ env.cache-name }}-${{ github.job }}-${{ hashFiles('.pre-commit-config.yaml') }}
+      - name: "Free space"
+        run: ./scripts/ci/tools/ci_free_space_on_ci.sh
+      - name: "Build CI image"
+        run: ./scripts/ci/images/ci_prepare_ci_image_on_ci.sh
+      - name: "Static checks: pylint"
+        run: ./scripts/ci/static_checks/ci_run_static_checks.sh pylint
+
   docs:
     timeout-minutes: 60
     name: "Build docs"
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index f2e349a..30f0a89 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -363,20 +363,12 @@ repos:
         exclude: ^dev|^backport_packages
         require_serial: true
       - id: pylint
-        name: Run pylint for main sources
+        name: Run pylint
         language: system
-        entry: "./scripts/ci/pre_commit/pre_commit_pylint_main.sh"
+        entry: "./scripts/ci/pre_commit/pre_commit_pylint.sh"
         files: \.py$
-        exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^backport_packages|^kubernetes_tests
-        pass_filenames: true
-        require_serial: true  # Pylint tests should be run in one chunk to detect all cycles
-      - id: pylint-tests
-        name: Run pylint for tests
-        language: system
-        entry: "./scripts/ci/pre_commit/pre_commit_pylint_tests.sh"
-        files: ^tests/.*\.py$
+        exclude: ^scripts/.*\.py$|^dev|^backport_packages
         pass_filenames: true
-        require_serial: true
       - id: flake8
         name: Run flake8
         language: system
diff --git a/STATIC_CODE_CHECKS.rst b/STATIC_CODE_CHECKS.rst
index 42522c3..2c4d304 100644
--- a/STATIC_CODE_CHECKS.rst
+++ b/STATIC_CODE_CHECKS.rst
@@ -230,13 +230,11 @@ To fix a pylint issue, do the following:
 1.  Remove module/modules from the
     `scripts/ci/static_checks/pylint_todo.txt <scripts/ci/pylint_todo.txt>`__.
 
-2.  Run `scripts/ci/static_checks/ci_pylint_main.sh <scripts/ci/ci_pylint_main.sh>`__ and
-    `scripts/ci/ci_pylint_tests.sh <scripts/ci/static_checks/ci_pylint_tests.sh>`__.
+2.  Run `scripts/ci/static_checks/ci_pylint.sh <scripts/ci/ci_pylint.sh>`__.
 
 3.  Fix all the issues reported by pylint.
 
-4.  Re-run `scripts/ci/static_checks/ci_pylint_main.sh <scripts/ci/ci_pylint_main.sh>`__ and
-    `scripts/ci/ci_pylint_tests.sh <scripts/ci/static_checks/ci_pylint_tests.sh>`__.
+4.  Re-run `scripts/ci/static_checks/ci_pylint.sh <scripts/ci/ci_pylint.sh>`__.
 
 5.  If you see "success", submit a PR following
     `Pull Request guidelines <#pull-request-guidelines>`__.
@@ -375,8 +373,7 @@ this, run the following scripts:
 * `<scripts/ci/static_checks/ci_flake8.sh>`_ - runs Flake8 source code style enforcement tool.
 * `<scripts/ci/static_checks/ci_lint_dockerfile.sh>`_ - runs lint checker for the dockerfiles.
 * `<scripts/ci/static_checks/ci_mypy.sh>`_ - runs a check for mypy type annotation consistency.
-* `<scripts/ci/static_checks/ci_pylint_main.sh>`_ - runs pylint static code checker for main files.
-* `<scripts/ci/static_checks/ci_pylint_tests.sh>`_ - runs pylint static code checker for tests.
+* `<scripts/ci/static_checks/ci_pylint.sh>`_ - runs pylint static code checker.
 
 The scripts may ask you to rebuild the images, if needed.
 
@@ -393,8 +390,7 @@ If you are already in the Breeze Docker environment (by running the ``./breeze``
 you can also run the same static checks via run_scripts:
 
 * Mypy: ``./scripts/ci/in_container/run_mypy.sh airflow tests``
-* Pylint for main files: ``./scripts/ci/in_container/run_pylint_main.sh``
-* Pylint for test files: ``./scripts/ci/in_container/run_pylint_tests.sh``
+* Pylint: ``./scripts/ci/in_container/run_pylint.sh``
 * Flake8: ``./scripts/ci/in_container/run_flake8.sh``
 * License check: ``./scripts/ci/in_container/run_check_licence.sh``
 * Documentation: ``./scripts/ci/in_container/run_docs_build.sh``
diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py
index 146be5c..90b2e1c 100644
--- a/airflow/jobs/scheduler_job.py
+++ b/airflow/jobs/scheduler_job.py
@@ -982,7 +982,7 @@ class DagFileProcessor(LoggingMixin):
         return simple_dags
 
 
-class SchedulerJob(BaseJob):
+class SchedulerJob(BaseJob):  # pylint: disable=too-many-instance-attributes
     """
     This SchedulerJob runs for a specific time interval and schedules the jobs
     that are ready to run. It figures out the latest runs for each
diff --git a/pylintrc b/pylintrc
index 43f9658..c9c7996 100644
--- a/pylintrc
+++ b/pylintrc
@@ -38,7 +38,7 @@ limit-inference-results=100
 
 # List of plugins (as comma separated values of python modules names) to load,
 # usually to register additional checkers.
-load-plugins=tests.airflow_pylint.do_not_use_asserts
+load-plugins=tests.airflow_pylint.do_not_use_asserts,tests.airflow_pylint.disable_checks_for_tests
 
 # Pickle collected data for later comparisons.
 persistent=yes
diff --git a/scripts/ci/in_container/run_pylint_main.sh b/scripts/ci/in_container/run_pylint.sh
similarity index 96%
rename from scripts/ci/in_container/run_pylint_main.sh
rename to scripts/ci/in_container/run_pylint.sh
index 3f25100..2220802 100755
--- a/scripts/ci/in_container/run_pylint_main.sh
+++ b/scripts/ci/in_container/run_pylint.sh
@@ -38,8 +38,6 @@ if [[ ${#@} == "0" ]]; then
     -path "./.eggs" -prune -o \
     -path "./docs/_build" -prune -o \
     -path "./build" -prune -o \
-    -path "./tests" -prune -o \
-    -path "./kubernetes_tests" -prune -o \
     -name "*.py" \
     -not -name 'webserver_config.py' | \
         grep  ".*.py$" | \
diff --git a/scripts/ci/in_container/run_pylint_tests.sh b/scripts/ci/in_container/run_pylint_tests.sh
deleted file mode 100755
index c434137..0000000
--- a/scripts/ci/in_container/run_pylint_tests.sh
+++ /dev/null
@@ -1,51 +0,0 @@
-#!/usr/bin/env bash
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-# shellcheck source=scripts/ci/in_container/_in_container_script_init.sh
-. "$( dirname "${BASH_SOURCE[0]}" )/_in_container_script_init.sh"
-
-export PYTHONPATH=${AIRFLOW_SOURCES}
-
-set +e
-
-if [[ ${#@} == "0" ]]; then
-    echo
-    echo "Running pylint for 'tests' and 'kubernetes_tests' folder"
-    echo
-    find "./tests" "./kubernetes_tests" -name "*.py" | \
-    grep -vFf scripts/ci/pylint_todo.txt | \
-    # running pylint using built-in parallel functionality might speed it up
-    xargs pylint -j 0 --disable="${DISABLE_CHECKS_FOR_TESTS}" --output-format=colorized | sort -u
-    RES=$?
-else
-    # running pylint using built-in parallel functionality might speed it up
-    pylint -j 0 --disable="${DISABLE_CHECKS_FOR_TESTS}" --output-format=colorized "$@" | sort -u
-    RES=$?
-fi
-
-set -e
-
-if [[ "${RES}" != 0 ]]; then
-    echo >&2
-    echo >&2 "There were some pylint errors. Exiting"
-    echo >&2
-    exit 1
-else
-    echo
-    echo "Pylint check succeeded"
-    echo
-fi
diff --git a/scripts/ci/pre_commit/pre_commit_pylint_main.sh b/scripts/ci/pre_commit/pre_commit_pylint.sh
similarity index 86%
rename from scripts/ci/pre_commit/pre_commit_pylint_main.sh
rename to scripts/ci/pre_commit/pre_commit_pylint.sh
index 39c9d78..7646178 100755
--- a/scripts/ci/pre_commit/pre_commit_pylint_main.sh
+++ b/scripts/ci/pre_commit/pre_commit_pylint.sh
@@ -18,5 +18,5 @@
 export FORCE_ANSWER_TO_QUESTIONS=${FORCE_ANSWER_TO_QUESTIONS:="quit"}
 export REMEMBER_LAST_ANSWER="true"
 
-# shellcheck source=scripts/ci/static_checks/ci_pylint_main.sh
-. "$( dirname "${BASH_SOURCE[0]}" )/../static_checks/ci_pylint_main.sh" "${@}"
+# shellcheck source=scripts/ci/static_checks/ci_pylint.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../static_checks/ci_pylint.sh" "${@}"
diff --git a/scripts/ci/pre_commit/pre_commit_pylint_tests.sh b/scripts/ci/pre_commit/pre_commit_pylint_tests.sh
deleted file mode 100755
index 3fc1965..0000000
--- a/scripts/ci/pre_commit/pre_commit_pylint_tests.sh
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/usr/bin/env bash
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-export FORCE_ANSWER_TO_QUESTIONS=${FORCE_ANSWER_TO_QUESTIONS:="quit"}
-export REMEMBER_LAST_ANSWER="true"
-
-# shellcheck source=scripts/ci/static_checks/ci_pylint_tests.sh
-. "$( dirname "${BASH_SOURCE[0]}" )/../static_checks/ci_pylint_tests.sh" "${@}"
diff --git a/scripts/ci/static_checks/ci_pylint_main.sh b/scripts/ci/static_checks/ci_pylint.sh
similarity index 93%
rename from scripts/ci/static_checks/ci_pylint_main.sh
rename to scripts/ci/static_checks/ci_pylint.sh
index 827a5a0..72f8d59 100755
--- a/scripts/ci/static_checks/ci_pylint_main.sh
+++ b/scripts/ci/static_checks/ci_pylint.sh
@@ -20,19 +20,19 @@ export PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION:-3.6}
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-function run_pylint_main() {
+function run_pylint() {
     FILES=("$@")
     if [[ "${#FILES[@]}" == "0" ]]; then
         docker run "${EXTRA_DOCKER_FLAGS[@]}" \
             --entrypoint "/usr/local/bin/dumb-init"  \
             "${AIRFLOW_CI_IMAGE}" \
-            "--" "/opt/airflow/scripts/ci/in_container/run_pylint_main.sh" \
+            "--" "/opt/airflow/scripts/ci/in_container/run_pylint.sh" \
             | tee -a "${OUTPUT_LOG}"
     else
         docker run "${EXTRA_DOCKER_FLAGS[@]}" \
             --entrypoint "/usr/local/bin/dumb-init" \
             "${AIRFLOW_CI_IMAGE}" \
-            "--" "/opt/airflow/scripts/ci/in_container/run_pylint_main.sh" "${FILES[@]}" \
+            "--" "/opt/airflow/scripts/ci/in_container/run_pylint.sh" "${FILES[@]}" \
             | tee -a "${OUTPUT_LOG}"
     fi
 }
@@ -49,8 +49,8 @@ if [[ "${#@}" != "0" ]]; then
     if [[ "${#FILTERED_FILES[@]}" == "0" ]]; then
         echo "Filtered out all files. Skipping pylint."
     else
-        run_pylint_main "${FILTERED_FILES[@]}"
+        run_pylint "${FILTERED_FILES[@]}"
     fi
 else
-    run_pylint_main
+    run_pylint
 fi
diff --git a/scripts/ci/static_checks/ci_pylint_tests.sh b/scripts/ci/static_checks/ci_pylint_tests.sh
deleted file mode 100755
index e4bac02..0000000
--- a/scripts/ci/static_checks/ci_pylint_tests.sh
+++ /dev/null
@@ -1,57 +0,0 @@
-#!/usr/bin/env bash
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-export PYTHON_MAJOR_MINOR_VERSION=${PYTHON_MAJOR_MINOR_VERSION:-3.6}
-
-# shellcheck source=scripts/ci/libraries/_script_init.sh
-. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
-
-function run_pylint_tests() {
-    FILES=("$@")
-    if [[ "${#FILES[@]}" == "0" ]]; then
-        docker run "${EXTRA_DOCKER_FLAGS[@]}" \
-            --entrypoint "/usr/local/bin/dumb-init"  \
-            "${AIRFLOW_CI_IMAGE}" \
-            "--" "/opt/airflow/scripts/ci/in_container/run_pylint_tests.sh" \
-            | tee -a "${OUTPUT_LOG}"
-    else
-        docker run "${EXTRA_DOCKER_FLAGS[@]}" \
-            --entrypoint "/usr/local/bin/dumb-init"  \
-            "${AIRFLOW_CI_IMAGE}" \
-            "--" "/opt/airflow/scripts/ci/in_container/run_pylint_tests.sh" "${FILES[@]}" \
-            | tee -a "${OUTPUT_LOG}"
-    fi
-}
-
-get_environment_for_builds_on_ci
-
-prepare_ci_build
-
-rebuild_ci_image_if_needed
-
-if [[ "${#@}" != "0" ]]; then
-    filter_out_files_from_pylint_todo_list "$@"
-
-    if [[ "${#FILTERED_FILES[@]}" == "0" ]]; then
-        echo "Filtered out all files. Skipping pylint."
-    else
-        run_pylint_tests "${FILTERED_FILES[@]}"
-    fi
-else
-    run_pylint_tests
-fi
diff --git a/tests/airflow_pylint/disable_checks_for_tests.py b/tests/airflow_pylint/disable_checks_for_tests.py
new file mode 100644
index 0000000..ddaae0b
--- /dev/null
+++ b/tests/airflow_pylint/disable_checks_for_tests.py
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+
+from astroid import MANAGER, scoped_nodes
+from pylint.lint import PyLinter
+
+DISABLED_CHECKS_FOR_TESTS = \
+    "missing-docstring, no-self-use, too-many-public-methods, protected-access, do-not-use-asserts"
+
+
+def register(_: PyLinter):
+    """
+    Skip registering any plugin. This is not a real plugin - we only need it to register transform before
+    running pylint.
+
+    :param _:
+    :return:
+    """
+
+
+def transform(mod):
+    """
+    It's a small hack but one that gives us a lot of speedup in pylint tests. We are replacing the first
+    line of the file with pylint-disable (or update existing one) when file name start with `test_` or
+    (for providers) when it is the full path of the package (both cases occur in pylint)
+
+    :param mod: astroid module
+    :return: None
+    """
+    if mod.name.startswith("test_") or \
+            mod.name.startswith("tests.") or \
+            mod.name.startswith("kubernetes_tests."):
+        decoded_lines = mod.stream().read().decode("utf-8").split("\n")
+        if decoded_lines[0].startswith("# pylint: disable="):
+            decoded_lines[0] = decoded_lines[0] + " " + DISABLED_CHECKS_FOR_TESTS
+        elif decoded_lines[0].startswith("#") or decoded_lines[0].strip() == "":
+            decoded_lines[0] = "# pylint: disable=" + DISABLED_CHECKS_FOR_TESTS
+        else:
+            raise Exception(f"The first line of module {mod.name} is not a comment or empty. "
+                            f"Please make sure it is!")
+        # pylint will read from `.file_bytes` attribute later when tokenization
+        mod.file_bytes = "\n".join(decoded_lines).encode("utf-8")
+
+
+MANAGER.register_transform(scoped_nodes.Module, transform)
diff --git a/tests/dags/test_logging_in_dag.py b/tests/dags/test_logging_in_dag.py
index 900e430..6d828fa 100644
--- a/tests/dags/test_logging_in_dag.py
+++ b/tests/dags/test_logging_in_dag.py
@@ -25,6 +25,11 @@ logger = logging.getLogger(__name__)
 
 
 def test_logging_fn(**kwargs):
+    """
+    Tests DAG logging.
+    :param kwargs:
+    :return:
+    """
     logger.info("Log from DAG Logger")
     kwargs["ti"].log.info("Log from TI Logger")
     print("Log from Print statement")
diff --git a/tests/dags/test_task_view_type_check.py b/tests/dags/test_task_view_type_check.py
index 6fa3e3c..fa04b5e 100644
--- a/tests/dags/test_task_view_type_check.py
+++ b/tests/dags/test_task_view_type_check.py
@@ -34,6 +34,9 @@ default_args = dict(
 
 
 class CallableClass:
+    """
+    Class that is callable.
+    """
     def __call__(self):
         """ A __call__ method """