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 2021/11/24 17:00:23 UTC

[airflow] branch main updated: Tests for Docker images in Python (#19737)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 621d17b  Tests for Docker images in Python (#19737)
621d17b is described below

commit 621d17bb77e3160c1a927803e5d190c0e2aade3c
Author: Kamil BreguĊ‚a <mi...@users.noreply.github.com>
AuthorDate: Wed Nov 24 17:59:47 2021 +0100

    Tests for Docker images in Python (#19737)
---
 .github/CODEOWNERS                                 |   5 +-
 .github/boring-cyborg.yml                          |   2 +-
 .pre-commit-config.yaml                            |   1 +
 docker_tests/__init__.py                           |  16 +
 docker_tests/ci_image.py                           |  41 +++
 docker_tests/docker_tests_utils.py                 | 116 +++++++
 docker_tests/prod_image.py                         | 206 ++++++++++++
 scripts/ci/docker-compose/local.yml                |   1 +
 scripts/ci/images/ci_run_docker_tests.py           | 101 ++++++
 .../images/ci_wait_for_and_verify_all_ci_images.sh |   2 +
 .../ci_wait_for_and_verify_all_prod_images.sh      |   2 +
 scripts/ci/installed_providers.txt                 |  22 ++
 scripts/ci/libraries/_build_images.sh              |   3 +-
 scripts/ci/libraries/_initialization.sh            |  24 --
 scripts/ci/libraries/_local_mounts.sh              |   1 +
 scripts/ci/libraries/_verify_image.sh              | 366 +--------------------
 scripts/ci/tools/verify_docker_image.sh            |  57 ----
 17 files changed, 519 insertions(+), 447 deletions(-)

diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS
index f993eac..92b5ce2 100644
--- a/.github/CODEOWNERS
+++ b/.github/CODEOWNERS
@@ -50,8 +50,9 @@
 /.github/workflows/ @potiuk @ashb @kaxil
 breeze @potiuk
 breeze-complete @potiuk
-Dockerfile @potiuk @ashb
+Dockerfile @potiuk @ashb @mik-laj
 Dockerfile.ci @potiuk @ashb
 /dev/ @potiuk @ashb @kaxil
 /provider_packages/ @potiuk @ashb
-/scripts/ @potiuk @ashb
+/scripts/ @potiuk @ashb @mik-laj
+/docker_tests/ @potiuk @ashb @mik-laj
diff --git a/.github/boring-cyborg.yml b/.github/boring-cyborg.yml
index f1a6c4b..765caa9 100644
--- a/.github/boring-cyborg.yml
+++ b/.github/boring-cyborg.yml
@@ -190,8 +190,8 @@ labelPRBasedOnFilePath:
     - Dockerfile
     - docs/docker-stack/**/*
     - scripts/in_container/prod/*
-    - scripts/ci/tools/verify_docker_image.sh
     - scripts/ci/libraries/_verify_image.sh
+    - docker_tests/**/*
 
 # Various Flags to control behaviour of the "Labeler"
 labelerFlags:
diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml
index 9bc9443..751d132 100644
--- a/.pre-commit-config.yaml
+++ b/.pre-commit-config.yaml
@@ -238,6 +238,7 @@ repos:
           ^scripts/.*\.py$|
           ^dev|
           ^provider_packages|
+          ^docker_tests|
           ^kubernetes_tests|
           .*example_dags/.*|
           ^chart/.*\.py$|
diff --git a/docker_tests/__init__.py b/docker_tests/__init__.py
new file mode 100644
index 0000000..13a8339
--- /dev/null
+++ b/docker_tests/__init__.py
@@ -0,0 +1,16 @@
+# 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.
diff --git a/docker_tests/ci_image.py b/docker_tests/ci_image.py
new file mode 100644
index 0000000..25a9bfb
--- /dev/null
+++ b/docker_tests/ci_image.py
@@ -0,0 +1,41 @@
+# 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.
+
+import subprocess
+
+from docker_tests.docker_tests_utils import (
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash_in_docker,
+    run_command,
+)
+
+
+class TestFiles:
+    def test_dist_folder_should_exists(self):
+        run_bash_in_docker('[ -f /opt/airflow/airflow/www/static/dist/manifest.json ] || exit 1')
+
+
+class TestPythonPackages:
+    def test_pip_dependencies_conflict(self):
+        try:
+            run_command(
+                ["docker", "run", "--rm", "--entrypoint", "/bin/bash", docker_image, "-c", 'pip check']
+            )
+        except subprocess.CalledProcessError as ex:
+            display_dependency_conflict_message()
+            raise ex
diff --git a/docker_tests/docker_tests_utils.py b/docker_tests/docker_tests_utils.py
new file mode 100644
index 0000000..b0155b0
--- /dev/null
+++ b/docker_tests/docker_tests_utils.py
@@ -0,0 +1,116 @@
+# 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.
+
+import os
+import shlex
+import subprocess
+from pathlib import Path
+from typing import List
+
+docker_image = os.environ.get('DOCKER_IMAGE')
+SOURCE_ROOT = Path(__file__).resolve().parents[1]
+
+if not docker_image:
+    raise Exception("The DOCKER_IMAGE environment variable is required")
+
+
+def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    try:
+        return subprocess.check_output(cmd, **kwargs).decode()
+    except subprocess.CalledProcessError as ex:
+        if print_output_on_error:
+            print("========================= OUTPUT start ============================")
+            print(ex.stderr)
+            print(ex.stdout)
+            print("========================= OUTPUT end ============================")
+        raise
+
+
+def run_bash_in_docker(bash_script, **kwargs):
+    docker_command = [
+        "docker",
+        "run",
+        "--rm",
+        "-e",
+        "COLUMNS=180",
+        "--entrypoint",
+        "/bin/bash",
+        docker_image,
+        "-c",
+        bash_script,
+    ]
+    return run_command(docker_command, **kwargs)
+
+
+def run_python_in_docker(python_script, **kwargs):
+    docker_command = [
+        "docker",
+        "run",
+        "--rm",
+        "-e",
+        "COLUMNS=180",
+        "-e",
+        "PYTHONDONTWRITEBYTECODE=true",
+        docker_image,
+        "python",
+        "-c",
+        python_script,
+    ]
+    return run_command(docker_command, **kwargs)
+
+
+def display_dependency_conflict_message():
+    print(
+        """
+***** Beginning of the instructions ****
+
+The image did not pass 'pip check' verification. This means that there are some conflicting dependencies
+in the image.
+
+It can mean one of those:
+
+1) The main is currently broken (other PRs will fail with the same error)
+2) You changed some dependencies in setup.py or setup.cfg and they are conflicting.
+
+
+
+In case 1) - apologies for the trouble.Please let committers know and they will fix it. You might
+be asked to rebase to the latest main after the problem is fixed.
+
+In case 2) - Follow the steps below:
+
+* try to build CI and then PROD image locally with breeze, adding --upgrade-to-newer-dependencies flag
+  (repeat it for all python versions)
+
+CI image:
+
+     ./breeze build-image --upgrade-to-newer-dependencies --python 3.6
+
+Production image:
+
+     ./breeze build-image --production-image --upgrade-to-newer-dependencies --python 3.6
+
+* You will see error messages there telling which requirements are conflicting and which packages caused the
+  conflict. Add the limitation that caused the conflict to EAGER_UPGRADE_ADDITIONAL_REQUIREMENTS
+  variable in Dockerfile.ci. Note that the limitations might be different for Dockerfile.ci and Dockerfile
+  because not all packages are installed by default in the PROD Dockerfile. So you might find that you
+  only need to add the limitation to the Dockerfile.ci
+
+***** End of the instructions ****
+"""
+    )
diff --git a/docker_tests/prod_image.py b/docker_tests/prod_image.py
new file mode 100644
index 0000000..cac517c
--- /dev/null
+++ b/docker_tests/prod_image.py
@@ -0,0 +1,206 @@
+# 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.
+
+import json
+import subprocess
+import tempfile
+from pathlib import Path
+
+import pytest
+
+from docker_tests.docker_tests_utils import (
+    SOURCE_ROOT,
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash_in_docker,
+    run_command,
+    run_python_in_docker,
+)
+
+INSTALLED_PROVIDER_PATH = SOURCE_ROOT / "scripts" / "ci" / "installed_providers.txt"
+
+
+class TestCommands:
+    def test_without_command(self):
+        """Checking the image without a command. It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_command(self):
+        """Checking 'airflow' command  It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow"])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_version(self):
+        """Checking 'airflow version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow", "version"]
+        )
+        assert "2." in output
+
+    def test_python_version(self):
+        """Checking 'python --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "python", "--version"]
+        )
+        assert "Python 3." in output
+
+    def test_bash_version(self):
+        """Checking 'bash --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "bash", "--version"]
+        )
+        assert "GNU bash," in output
+
+
+class TestPythonPackages:
+    def test_required_providers_are_installed(self):
+        lines = (d.strip() for d in INSTALLED_PROVIDER_PATH.read_text().splitlines())
+        lines = (d for d in lines)
+        packages_to_install = {f"apache-airflow-providers-{d.replace('.', '-')}" for d in lines}
+        assert len(packages_to_install) != 0
+
+        output = run_bash_in_docker("airflow providers list --output json", stderr=subprocess.DEVNULL)
+        providers = json.loads(output)
+        packages_installed = {d['package_name'] for d in providers}
+        assert len(packages_installed) != 0
+
+        assert packages_to_install == packages_installed, (
+            f"List of expected installed packages and image content mismatch. "
+            f"Check {INSTALLED_PROVIDER_PATH} file."
+        )
+
+    def test_pip_dependencies_conflict(self):
+        try:
+            run_bash_in_docker("pip check")
+        except subprocess.CalledProcessError as ex:
+            display_dependency_conflict_message()
+            raise ex
+
+    PACKAGE_IMPORTS = {
+        "amazon": ["boto3", "botocore", "watchtower"],
+        "async": ["gevent", "eventlet", "greenlet"],
+        "azure": [
+            'azure.batch',
+            'azure.cosmos',
+            'azure.datalake.store',
+            'azure.identity',
+            'azure.keyvault',
+            'azure.kusto.data',
+            'azure.mgmt.containerinstance',
+            'azure.mgmt.datalake.store',
+            'azure.mgmt.resource',
+            'azure.storage',
+        ],
+        "celery": ["celery", "flower", "vine"],
+        "cncf.kubernetes": ["kubernetes", "cryptography"],
+        "dask": ["cloudpickle", "distributed"],
+        "docker": ["docker"],
+        "elasticsearch": ["elasticsearch", "es.elastic", "elasticsearch_dsl"],
+        "google": [
+            'OpenSSL',
+            'google.ads',
+            'googleapiclient',
+            'google.auth',
+            'google_auth_httplib2',
+            'google.cloud.automl',
+            'google.cloud.bigquery_datatransfer',
+            'google.cloud.bigtable',
+            'google.cloud.container',
+            'google.cloud.datacatalog',
+            'google.cloud.dataproc',
+            'google.cloud.dlp',
+            'google.cloud.kms',
+            'google.cloud.language',
+            'google.cloud.logging',
+            'google.cloud.memcache',
+            'google.cloud.monitoring',
+            'google.cloud.oslogin',
+            'google.cloud.pubsub',
+            'google.cloud.redis',
+            'google.cloud.secretmanager',
+            'google.cloud.spanner',
+            'google.cloud.speech',
+            'google.cloud.storage',
+            'google.cloud.tasks',
+            'google.cloud.texttospeech',
+            'google.cloud.translate',
+            'google.cloud.videointelligence',
+            'google.cloud.vision',
+        ],
+        "grpc": ["grpc", "google.auth", "google_auth_httplib2"],
+        "hashicorp": ["hvac"],
+        "ldap": ["ldap"],
+        "mysql": ["mysql"],
+        "postgres": ["psycopg2"],
+        "pyodbc": ["pyodbc"],
+        "redis": ["redis"],
+        "sendgrid": ["sendgrid"],
+        "sftp/ssh": ["paramiko", "pysftp", "sshtunnel"],
+        "slack": ["slack_sdk"],
+        "statsd": ["statsd"],
+        "virtualenv": ["virtualenv"],
+    }
+
+    @pytest.mark.parametrize("package_name,import_names", PACKAGE_IMPORTS.items())
+    def test_check_dependencies_imports(self, package_name, import_names):
+        run_python_in_docker(f"import {','.join(import_names)}")
+
+
+class TestExecuteAsRoot:
+    def test_execute_airflow_as_root(self):
+        run_command(
+            [
+                "docker",
+                "run",
+                "--rm",
+                "--user",
+                "0",
+                "-e",
+                "PYTHONDONTWRITEBYTECODE=true",
+                docker_image,
+                "airflow",
+                "info",
+            ]
+        )
+
+    def test_run_custom_python_packages_as_root(self):
+        with tempfile.TemporaryDirectory() as tmp_dir:
+            (Path(tmp_dir) / "__init__.py").write_text('')
+            (Path(tmp_dir) / "awesome.py").write_text('print("Awesome")')
+
+            run_command(
+                [
+                    "docker",
+                    "run",
+                    "--rm",
+                    "-e",
+                    f"PYTHONPATH={tmp_dir}",
+                    "-e",
+                    "PYTHONDONTWRITEBYTECODE=true",
+                    "-v",
+                    f"{tmp_dir}:{tmp_dir}",
+                    "--user",
+                    "0",
+                    docker_image,
+                    "python",
+                    "-c",
+                    "import awesome",
+                ]
+            )
diff --git a/scripts/ci/docker-compose/local.yml b/scripts/ci/docker-compose/local.yml
index 0f144b5..d5c1836 100644
--- a/scripts/ci/docker-compose/local.yml
+++ b/scripts/ci/docker-compose/local.yml
@@ -54,6 +54,7 @@ services:
       - ../../../setup.py:/opt/airflow/setup.py:cached
       - ../../../tests:/opt/airflow/tests:cached
       - ../../../kubernetes_tests:/opt/airflow/kubernetes_tests:cached
+      - ../../../docker_tests:/opt/airflow/docker_tests:cached
       - ../../../chart:/opt/airflow/chart:cached
       - ../../../metastore_browser:/opt/airflow/metastore_browser:cached
       # END automatically generated volumes from LOCAL_MOUNTS in _local_mounts.sh
diff --git a/scripts/ci/images/ci_run_docker_tests.py b/scripts/ci/images/ci_run_docker_tests.py
new file mode 100755
index 0000000..af81cb4
--- /dev/null
+++ b/scripts/ci/images/ci_run_docker_tests.py
@@ -0,0 +1,101 @@
+#!/usr/bin/env python3
+# 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.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parent.parent.parent
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+CBLUE = '\033[94m'
+CEND = '\033[0m'
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"{CBLUE}$ {' '.join(shlex.quote(c) for c in cmd)}{CEND}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])
+
+
+if __name__ == "__main__":
+    main()
diff --git a/scripts/ci/images/ci_wait_for_and_verify_all_ci_images.sh b/scripts/ci/images/ci_wait_for_and_verify_all_ci_images.sh
index cb45c1f..0a31ad2 100755
--- a/scripts/ci/images/ci_wait_for_and_verify_all_ci_images.sh
+++ b/scripts/ci/images/ci_wait_for_and_verify_all_ci_images.sh
@@ -23,6 +23,8 @@ LIBRARIES_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")/../libraries/" && pwd)
 # shellcheck source=scripts/ci/libraries/_all_libs.sh
 source "${LIBRARIES_DIR}/_all_libs.sh"
 
+python3 "$( dirname "${BASH_SOURCE[0]}" )/ci_run_docker_tests.py" "--initialize"
+
 initialization::set_output_color_variables
 
 export PARALLEL_TAIL_LENGTH=5
diff --git a/scripts/ci/images/ci_wait_for_and_verify_all_prod_images.sh b/scripts/ci/images/ci_wait_for_and_verify_all_prod_images.sh
index 5df66a3..bd6c336 100755
--- a/scripts/ci/images/ci_wait_for_and_verify_all_prod_images.sh
+++ b/scripts/ci/images/ci_wait_for_and_verify_all_prod_images.sh
@@ -25,6 +25,8 @@ source "${LIBRARIES_DIR}/_all_libs.sh"
 
 initialization::set_output_color_variables
 
+python3 "$( dirname "${BASH_SOURCE[0]}" )/ci_run_docker_tests.py" "--initialize"
+
 export PARALLEL_TAIL_LENGTH=5
 
 parallel::make_sure_gnu_parallel_is_installed
diff --git a/scripts/ci/installed_providers.txt b/scripts/ci/installed_providers.txt
new file mode 100644
index 0000000..c6b02bf
--- /dev/null
+++ b/scripts/ci/installed_providers.txt
@@ -0,0 +1,22 @@
+amazon
+celery
+cncf.kubernetes
+docker
+elasticsearch
+ftp
+google
+grpc
+hashicorp
+http
+imap
+microsoft.azure
+mysql
+odbc
+postgres
+redis
+sendgrid
+sftp
+slack
+sqlite
+sqlite
+ssh
diff --git a/scripts/ci/libraries/_build_images.sh b/scripts/ci/libraries/_build_images.sh
index e9572b0..21869f0 100644
--- a/scripts/ci/libraries/_build_images.sh
+++ b/scripts/ci/libraries/_build_images.sh
@@ -942,7 +942,8 @@ function build_images::build_prod_images_from_locally_built_airflow_packages() {
     build_images::cleanup_docker_context_files
 
     # Build necessary provider packages
-    runs::run_prepare_provider_packages "${INSTALLED_PROVIDERS[@]}"
+    IFS=$'\n' read -d '' -r -a installed_providers < "${AIRFLOW_SOURCES}/scripts/ci/installed_providers.txt"
+    runs::run_prepare_provider_packages "${installed_providers[@]}"
     mv "${AIRFLOW_SOURCES}/dist/"* "${AIRFLOW_SOURCES}/docker-context-files/"
 
     # Build apache airflow packages
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index 6d4842a..25ea3e7 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -416,30 +416,6 @@ function initialization::initialize_image_build_variables() {
     INSTALL_PROVIDERS_FROM_SOURCES=${INSTALL_PROVIDERS_FROM_SOURCES:="true"}
     export INSTALL_PROVIDERS_FROM_SOURCES
 
-    INSTALLED_PROVIDERS+=(
-        "amazon"
-        "celery"
-        "cncf.kubernetes"
-        "docker"
-        "elasticsearch"
-        "ftp"
-        "grpc"
-        "hashicorp"
-        "http"
-        "imap"
-        "google"
-        "microsoft.azure"
-        "mysql"
-        "postgres"
-        "redis"
-        "sendgrid"
-        "sqlite"
-        "sftp"
-        "slack"
-        "sqlite"
-        "ssh"
-    )
-    export INSTALLED_PROVIDERS
     export INSTALLED_EXTRAS="async,amazon,celery,cncf.kubernetes,docker,dask,elasticsearch,ftp,grpc,hashicorp,http,imap,ldap,google,microsoft.azure,mysql,postgres,redis,sendgrid,sftp,slack,ssh,statsd,virtualenv"
 
     AIRFLOW_PIP_VERSION=${AIRFLOW_PIP_VERSION:="21.2.4"}
diff --git a/scripts/ci/libraries/_local_mounts.sh b/scripts/ci/libraries/_local_mounts.sh
index c9cc709..c74a71f 100644
--- a/scripts/ci/libraries/_local_mounts.sh
+++ b/scripts/ci/libraries/_local_mounts.sh
@@ -50,6 +50,7 @@ function local_mounts::generate_local_mounts_list {
         "$prefix"setup.py:/opt/airflow/setup.py:cached
         "$prefix"tests:/opt/airflow/tests:cached
         "$prefix"kubernetes_tests:/opt/airflow/kubernetes_tests:cached
+        "$prefix"docker_tests:/opt/airflow/docker_tests:cached
         "$prefix"chart:/opt/airflow/chart:cached
         "$prefix"metastore_browser:/opt/airflow/metastore_browser:cached
     )
diff --git a/scripts/ci/libraries/_verify_image.sh b/scripts/ci/libraries/_verify_image.sh
index 1b6b270..dfe507d 100644
--- a/scripts/ci/libraries/_verify_image.sh
+++ b/scripts/ci/libraries/_verify_image.sh
@@ -15,373 +15,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-function verify_image::run_command_in_image() {
-    docker_v run --rm \
-            -e COLUMNS=180 \
-            --entrypoint /bin/bash "${DOCKER_IMAGE}" \
-            -c "${@}"
-}
-
-IMAGE_VALID="true"
-
-function verify_image::check_command() {
-    DESCRIPTION="${1}"
-    COMMAND=${2}
-    set +e
-    echo -n "Feature: ${DESCRIPTION} "
-    local output
-    output=$(verify_image::run_command_in_image "${COMMAND}" 2>&1)
-    local res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-    set -e
-}
-
-function verify_image::verify_prod_image_commands() {
-    start_end::group_start "Checking command supports"
-    set +e
-
-    echo -n "Feature: Checking the image without a command. It should return non-zero exit code."
-    local output
-    output=$(docker_v run --rm \
-            -e COLUMNS=180 \
-            "${DOCKER_IMAGE}" \
-            2>&1)
-    local res=$?
-    if [[ ${res} == "2" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-    echo -n "Feature: Checking 'airflow' command  It should return non-zero exit code."
-    output=$(docker_v run --rm \
-            -e COLUMNS=180 \
-            "${DOCKER_IMAGE}" \
-            "airflow" 2>&1)
-    local res=$?
-    if [[ ${res} == "2" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-
-    echo -n "Feature: Checking 'airflow version' command  It should return zero exit code."
-    output=$(docker_v run --rm \
-            -e COLUMNS=180 \
-            "${DOCKER_IMAGE}" \
-            "airflow" "version" 2>&1)
-    local res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-
-    echo -n "Feature: Checking 'python --version' command  It should return zero exit code."
-    output=$(docker_v run --rm \
-            -e COLUMNS=180 \
-            "${DOCKER_IMAGE}" \
-            python --version | grep "Python 3." 2>&1)
-    local res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-    echo -n "Feature: Checking 'bash --version' command  It should return zero exit code."
-    output=$(docker_v run --rm \
-            -e COLUMNS=180 \
-            "${DOCKER_IMAGE}" \
-            bash --version | grep "GNU bash, " 2>&1)
-    local res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-
-    set -e
-}
-
-function verify_image::verify_prod_image_has_airflow_and_providers() {
-    start_end::group_start "Verify prod image: ${DOCKER_IMAGE}"
-    echo
-    echo "Checking if Providers are installed"
-    echo
-
-    all_providers_installed_in_image=$(verify_image::run_command_in_image "airflow providers list --output table")
-
-    echo
-    echo "Installed providers:"
-    echo
-    echo "${all_providers_installed_in_image}"
-    echo
-    local error="false"
-    for provider in "${INSTALLED_PROVIDERS[@]}"; do
-        echo -n "Verifying if provider ${provider} installed: "
-        if [[ ${all_providers_installed_in_image} == *"apache-airflow-providers-${provider//./-}"* ]]; then
-            echo "${COLOR_GREEN}OK${COLOR_RESET}"
-        else
-            echo "${COLOR_RED}NOK${COLOR_RESET}"
-            error="true"
-        fi
-    done
-    if [[ ${error} == "true" ]]; then
-        echo
-        echo "${COLOR_RED}ERROR: Some expected providers are not installed!${COLOR_RESET}"
-        echo
-        IMAGE_VALID="false"
-    else
-        echo
-        echo "${COLOR_GREEN}OK. All expected providers installed!${COLOR_RESET}"
-        echo
-    fi
-    start_end::group_end
-}
-
-function verify_image::verify_ci_image_dependencies() {
-    start_end::group_start "Checking if Airflow dependencies are non-conflicting in ${DOCKER_IMAGE} image."
-    set +e
-    docker_v run --rm --entrypoint /bin/bash "${DOCKER_IMAGE}" -c 'pip check'
-    local res=$?
-    if [[ ${res} != "0" ]]; then
-        echo  "${COLOR_RED}ERROR: ^^^ Some dependencies are conflicting. See instructions below on how to deal with it.  ${COLOR_RESET}"
-        echo
-        build_images::inform_about_pip_check ""
-        IMAGE_VALID="false"
-    else
-        echo
-        echo  "${COLOR_GREEN}OK. The ${DOCKER_IMAGE} image dependencies are consistent.  ${COLOR_RESET}"
-        echo
-    fi
-    set -e
-    start_end::group_end
-}
-
-function verify_image::verify_ci_image_has_dist_folder() {
-    start_end::group_start "Verify CI image dist folder (compiled www assets): ${DOCKER_IMAGE}"
-
-    verify_image::check_command "Dist folder" '[ -f /opt/airflow/airflow/www/static/dist/manifest.json ] || exit 1'
-
-    start_end::group_end
-}
-
-
-function verify_image::verify_prod_image_dependencies() {
-    start_end::group_start "Checking if Airflow dependencies are non-conflicting in ${DOCKER_IMAGE} image."
-
-    set +e
-    verify_image::run_command_in_image 'pip check'
-    local res=$?
-    if [[ ${res} != "0" ]]; then
-        echo "${COLOR_RED}ERROR: ^^^ Some dependencies are conflicting. See instructions below on how to deal with it.  ${COLOR_RESET}"
-        echo
-        build_images::inform_about_pip_check "--production "
-        IMAGE_VALID="false"
-    else
-        echo
-        echo "${COLOR_GREEN}OK. The ${DOCKER_IMAGE} image dependencies are consistent.  ${COLOR_RESET}"
-        echo
-    fi
-    set -e
-    start_end::group_end
-}
-
-GOOGLE_IMPORTS=(
-    'OpenSSL'
-    'google.ads'
-    'googleapiclient'
-    'google.auth'
-    'google_auth_httplib2'
-    'google.cloud.automl'
-    'google.cloud.bigquery_datatransfer'
-    'google.cloud.bigtable'
-    'google.cloud.container'
-    'google.cloud.datacatalog'
-    'google.cloud.dataproc'
-    'google.cloud.dlp'
-    'google.cloud.kms'
-    'google.cloud.language'
-    'google.cloud.logging'
-    'google.cloud.memcache'
-    'google.cloud.monitoring'
-    'google.cloud.oslogin'
-    'google.cloud.pubsub'
-    'google.cloud.redis'
-    'google.cloud.secretmanager'
-    'google.cloud.spanner'
-    'google.cloud.speech'
-    'google.cloud.storage'
-    'google.cloud.tasks'
-    'google.cloud.texttospeech'
-    'google.cloud.translate'
-    'google.cloud.videointelligence'
-    'google.cloud.vision'
-)
-
-AZURE_IMPORTS=(
-    'azure.batch'
-    'azure.cosmos'
-    'azure.datalake.store'
-    'azure.identity'
-    'azure.keyvault'
-    'azure.kusto.data'
-    'azure.mgmt.containerinstance'
-    'azure.mgmt.datalake.store'
-    'azure.mgmt.resource'
-    'azure.storage'
-)
-
-function verify_image::verify_production_image_python_modules() {
-    start_end::group_start "Verify prod image features: ${DOCKER_IMAGE}"
-
-    verify_image::check_command "Import: async" "python -c 'import gevent, eventlet, greenlet'"
-    verify_image::check_command "Import: amazon" "python -c 'import boto3, botocore, watchtower'"
-    verify_image::check_command "Import: celery" "python -c 'import celery, flower, vine'"
-    verify_image::check_command "Import: cncf.kubernetes" "python -c 'import kubernetes, cryptography'"
-    verify_image::check_command "Import: docker" "python -c 'import docker'"
-    verify_image::check_command "Import: dask" "python -c 'import cloudpickle, distributed'"
-    verify_image::check_command "Import: elasticsearch" "python -c 'import elasticsearch,es.elastic, elasticsearch_dsl'"
-    verify_image::check_command "Import: grpc" "python -c 'import grpc, google.auth, google_auth_httplib2'"
-    verify_image::check_command "Import: hashicorp" "python -c 'import hvac'"
-    verify_image::check_command "Import: ldap" "python -c 'import ldap'"
-    for google_import in "${GOOGLE_IMPORTS[@]}"
-    do
-        verify_image::check_command "Import google: ${google_import}" "python -c 'import ${google_import}'"
-    done
-    for azure_import in "${AZURE_IMPORTS[@]}"
-    do
-        verify_image::check_command "Import azure: ${azure_import}" "python -c 'import ${azure_import}'"
-    done
-    verify_image::check_command "Import: mysql" "python -c 'import mysql'"
-    verify_image::check_command "Import: postgres" "python -c 'import psycopg2'"
-    verify_image::check_command "Import: redis" "python -c 'import redis'"
-    verify_image::check_command "Import: sendgrid" "python -c 'import sendgrid'"
-    verify_image::check_command "Import: sftp/ssh" "python -c 'import paramiko, pysftp, sshtunnel'"
-    verify_image::check_command "Import: slack" "python -c 'import slack_sdk'"
-    verify_image::check_command "Import: statsd" "python -c 'import statsd'"
-    verify_image::check_command "Import: virtualenv" "python -c 'import virtualenv'"
-    verify_image::check_command "Import: pyodbc" "python -c 'import pyodbc'"
-
-    start_end::group_end
-}
-
-function verify_image::verify_prod_image_as_root() {
-    start_end::group_start "Checking if the image can be run as root."
-    set +e
-    echo "Checking airflow as root"
-    local output
-    local res
-    output=$(docker_v run --rm --user 0 "${DOCKER_IMAGE}" "airflow" "info" 2>&1)
-    res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-
-    echo "Checking root container with custom PYTHONPATH"
-    local tmp_dir
-    tmp_dir="$(mktemp -d)"
-    touch "${tmp_dir}/__init__.py"
-    echo 'print("Awesome")' >> "${tmp_dir}/awesome.py"
-    output=$(docker_v run \
-        --rm \
-        -e "PYTHONPATH=${tmp_dir}" \
-        -v "${tmp_dir}:${tmp_dir}" \
-        --user 0 "${DOCKER_IMAGE}" \
-            "python" "-c" "import awesome" \
-        2>&1)
-    res=$?
-    if [[ ${res} == "0" ]]; then
-        echo "${COLOR_GREEN}OK${COLOR_RESET}"
-    else
-        echo "${COLOR_RED}NOK${COLOR_RESET}"
-        echo "${COLOR_BLUE}========================= OUTPUT start ============================${COLOR_RESET}"
-        echo "${output}"
-        echo "${COLOR_BLUE}========================= OUTPUT end   ===========================${COLOR_RESET}"
-        IMAGE_VALID="false"
-    fi
-    rm -rf "${tmp_dir}"
-    set -e
-}
-
-function verify_image::verify_production_image_has_dist_folder() {
-    start_end::group_start "Verify prod image has dist folder (compiled www assets): ${DOCKER_IMAGE}"
-    # shellcheck disable=SC2016
-    verify_image::check_command "Dist folder" '[ -f $(python -m site --user-site)/airflow/www/static/dist/manifest.json ] || exit 1'
-
-    start_end::group_end
-}
-
-function verify_image::display_result {
-    if [[ ${IMAGE_VALID} == "true" ]]; then
-        echo
-        echo "${COLOR_GREEN}OK. The ${DOCKER_IMAGE} features are all OK.  ${COLOR_RESET}"
-        echo
-    else
-        echo
-        echo "${COLOR_RED}ERROR: Some features were not ok!${COLOR_RESET}"
-        echo
-        exit 1
-    fi
-}
 
 function verify_image::verify_prod_image {
-    IMAGE_VALID="true"
     DOCKER_IMAGE="${1}"
-    verify_image::verify_prod_image_commands
-
-    verify_image::verify_prod_image_has_airflow_and_providers
-
-    verify_image::verify_production_image_python_modules
-
-    verify_image::verify_prod_image_dependencies
-
-    verify_image::verify_prod_image_as_root
-
-    verify_image::verify_production_image_has_dist_folder
-
-    verify_image::display_result
+    export DOCKER_IMAGE
+    python3 "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
 }
 
 function verify_image::verify_ci_image {
-    IMAGE_VALID="true"
     DOCKER_IMAGE="${1}"
-    verify_image::verify_ci_image_dependencies
-
-    verify_image::verify_ci_image_has_dist_folder
-
-    verify_image::display_result
+    export DOCKER_IMAGE
+    python3 "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/ci_image.py"
 }
diff --git a/scripts/ci/tools/verify_docker_image.sh b/scripts/ci/tools/verify_docker_image.sh
deleted file mode 100755
index 3ef5e3e..0000000
--- a/scripts/ci/tools/verify_docker_image.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.
-# shellcheck source=scripts/ci/libraries/_script_init.sh
-. "$(dirname "${BASH_SOURCE[0]}")/../libraries/_script_init.sh"
-
-usage() {
-local cmdname
-cmdname="$(basename -- "$0")"
-
-cat << EOF
-Usage: ${cmdname} <IMAGE_TYPE> <DOCKER_IMAGE>
-
-Verify the user-specified docker image.
-
-Image Type can be one of the two values: CI or PROD
-
-EOF
-}
-
-
-if [[ "$#" -ne 2 ]]; then
-    >&2 echo "You must provide two argument - image type [PROD/CI] and image name."
-    usage
-    exit 1
-fi
-
-IMAGE_TYPE="${1}"
-IMAGE_NAME="${2}"
-
-if ! docker image inspect "${IMAGE_NAME}" &>/dev/null; then
-    >&2 echo "Image '${IMAGE_NAME}' doesn't exists in local registry."
-    exit 1
-fi
-
-if [ "$(echo "${IMAGE_TYPE}" | tr '[:lower:]' '[:upper:]')" = "PROD" ]; then
-    verify_image::verify_prod_image "${IMAGE_NAME}"
-elif [ "$(echo "${IMAGE_TYPE}" | tr '[:lower:]' '[:upper:]')" = "CI" ]; then
-    verify_image::verify_ci_image "${IMAGE_NAME}"
-else
-    >&2 echo "Unsupported image type. Supported values: PROD, CI"
-    exit 1
-fi