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/03/22 22:14:32 UTC

[airflow] branch master updated: Running tests in parallel (#14915)

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 01a5d36  Running tests in parallel (#14915)
01a5d36 is described below

commit 01a5d36e6bbc1d9e7afd4e984376301ea378a94a
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Mon Mar 22 23:14:15 2021 +0100

    Running tests in parallel (#14915)
    
    This is by far the biggest improvements of the test execution time
    we can get now when we are using self-hosted runners.
    
    This change drives down the time of executing all tests on
    self-hosted runners from ~ 50 minutes to ~ 13 minutes due to heavy
    parallelisation we can implement for different test types and the
    fact that our machines for self-hosted runners are far more
    capable - they have more CPU, more memory and the fact that
    we are using tmpfs for everything.
    
    This change will also drive the cost of our self-hosted runners
    down. Since we have auto-scaling infrastructure we will simply need
    the machines to run tests for far shorter time. Since the number
    of test jobs we run on those self hosted runners is substantial
    (10 jobs), we are going to save ~ 6 build hours per one PR/merged
    commit!
    
    This also allows the developers to use the power of their
    development machines - when you use
    `./scripts/ci/testing/ci_run_airflow_testing.sh` the script
    detects how many CPU cores are available and it will run as
    many parallel test types as many cores you have.
    
    Also in case of Integration tests - they require more memory to run
    all the integrations, so in case there is less than ~ 32 GB of RAM
    available to Docker, the integration tests are run sequentially
    at the end. This drives stability up for machines with lower memory.
    
    On one personal PC (64GB RAM, 8 CPUS/16 cores, fast SSD) the full
    test suite execution went down from 30 minutes to 5 minutes.
    
    There is a continuous progress information printed every 10 seconds when
    either parallel or sequential tests are run, and the full output is
    shown at the end - failed tests are marked in red groups, and succesful are
    marked in green groups. This makes it easier to see and analyse errors.
---
 .github/workflows/build-images-workflow-run.yml    |   2 +
 .github/workflows/ci.yml                           |  37 ++-
 .github/workflows/scheduled_quarantined.yml        |   4 +-
 BREEZE.rst                                         |   4 +-
 TESTING.rst                                        |  80 ++++++
 airflow/__init__.py                                |   1 +
 breeze                                             |  31 ++-
 breeze-complete                                    |   2 +-
 .../backend-mysql-port.yml}                        |  14 +-
 scripts/ci/docker-compose/backend-mysql.yml        |   3 -
 .../backend-postgres-port.yml}                     |  14 +-
 scripts/ci/docker-compose/backend-postgres.yml     |   2 -
 .../backend-sqlite-port.yml}                       |  10 +-
 scripts/ci/libraries/_all_libs.sh                  |   2 +
 scripts/ci/libraries/_docker_engine_resources.sh   |  29 +-
 scripts/ci/libraries/_initialization.sh            |   9 +-
 scripts/ci/libraries/_parallel.sh                  | 111 ++++++++
 scripts/ci/libraries/_start_end.sh                 |   2 +-
 scripts/ci/mysql/conf.d/airflow.cnf                |   7 +
 scripts/ci/selective_ci_checks.sh                  |   2 +-
 scripts/ci/testing/ci_run_airflow_testing.sh       | 310 ++++++++++++++-------
 ....sh => ci_run_single_airflow_test_in_docker.sh} | 242 ++++++++--------
 scripts/ci/tools/ci_free_space_on_ci.sh            |   7 +
 scripts/in_container/entrypoint_ci.sh              |  71 +++--
 scripts/in_container/run_ci_tests.sh               |  56 +---
 tests/cli/commands/test_task_command.py            |   2 +
 tests/jobs/test_local_task_job.py                  |   2 +-
 tests/jobs/test_scheduler_job.py                   |  18 +-
 tests/www/test_views.py                            |   7 +-
 29 files changed, 683 insertions(+), 398 deletions(-)

diff --git a/.github/workflows/build-images-workflow-run.yml b/.github/workflows/build-images-workflow-run.yml
index 1f1edc9..e00cdc5 100644
--- a/.github/workflows/build-images-workflow-run.yml
+++ b/.github/workflows/build-images-workflow-run.yml
@@ -272,6 +272,7 @@ jobs:
       needs.build-info.outputs.image-build == 'true' &&
       needs.cancel-workflow-runs.outputs.buildImages == 'true'
     env:
+      RUNS_ON: ${{ fromJson(needs.cancel-workflow-runs.outputs.runsOn)[0] }}
       BACKEND: postgres
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       GITHUB_REGISTRY_PUSH_IMAGE_TAG: ${{ github.event.workflow_run.id }}
@@ -399,6 +400,7 @@ jobs:
       needs.build-info.outputs.image-build == 'true' &&
       needs.cancel-workflow-runs.outputs.buildImages == 'true'
     env:
+      RUNS_ON: ${{ fromJson(needs.cancel-workflow-runs.outputs.runsOn)[0] }}
       BACKEND: postgres
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       GITHUB_REGISTRY_PUSH_IMAGE_TAG: ${{ github.event.workflow_run.id }}
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index dfc2e49..00d82c0 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -236,6 +236,7 @@ jobs:
     needs: [build-info]
     if: needs.build-info.outputs.image-build == 'true'
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: sqlite
       UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
       WAIT_FOR_IMAGE: ${{ needs.build-info.outputs.waitForImage }}
@@ -283,6 +284,7 @@ jobs:
       matrix:
         python-version: ${{ fromJson(needs.build-info.outputs.pythonVersions) }}
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: sqlite
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     if: needs.build-info.outputs.image-build == 'true'
@@ -308,6 +310,7 @@ jobs:
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, ci-images]
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       SKIP: "pylint,identity"
       MOUNT_SELECTED_LOCAL_SOURCES: "true"
       PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
@@ -359,6 +362,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info]
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       SKIP: "build,mypy,flake8,pylint,bats-in-container-tests,identity"
       MOUNT_SELECTED_LOCAL_SOURCES: "true"
       PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
@@ -411,6 +415,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     needs: [build-info, ci-images]
     if: needs.build-info.outputs.basic-checks-only == 'false'
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       # We want to make sure we have latest sources as only in_container scripts are added
       # to the image but we want to static-check all of them
       MOUNT_SELECTED_LOCAL_SOURCES: "true"
@@ -460,6 +465,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     needs: [build-info, ci-images]
     if: needs.build-info.outputs.docs-build == 'true'
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     steps:
       - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
@@ -492,6 +498,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, ci-images]
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       INSTALL_AIRFLOW_VERSION: "${{ matrix.package-format }}"
       AIRFLOW_EXTRAS: "all"
       PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
@@ -538,6 +545,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, ci-images]
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       INSTALL_AIRFLOW_VERSION: "2.0.0"
       AIRFLOW_EXTRAS: "all"
       PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
@@ -575,8 +583,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs: [build-info, ci-images]
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       MOUNT_SELECTED_LOCAL_SOURCES: "true"
-      RUN_TESTS: true
       TEST_TYPES: "Helm"
       BACKEND: "sqlite"
       PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
@@ -635,12 +643,11 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         exclude: ${{ fromJson(needs.build-info.outputs.postgresExclude) }}
       fail-fast: false
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: postgres
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       POSTGRES_VERSION: ${{ matrix.postgres-version }}
-      RUN_TESTS: true
       TEST_TYPES: "${{needs.build-info.outputs.testTypes}}"
-      TEST_TYPE: ""
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     if: needs.build-info.outputs.run-tests == 'true'
     steps:
@@ -693,12 +700,11 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         exclude: ${{ fromJson(needs.build-info.outputs.mysqlExclude) }}
       fail-fast: false
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: mysql
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       MYSQL_VERSION: ${{ matrix.mysql-version }}
-      RUN_TESTS: true
       TEST_TYPES: "${{needs.build-info.outputs.testTypes}}"
-      TEST_TYPE: ""
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     if: needs.build-info.outputs.run-tests == 'true'
     steps:
@@ -749,11 +755,10 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         exclude: ${{ fromJson(needs.build-info.outputs.sqliteExclude) }}
       fail-fast: false
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: sqlite
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
-      RUN_TESTS: true
       TEST_TYPES: "${{needs.build-info.outputs.testTypes}}"
-      TEST_TYPE: ""
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     if: needs.build-info.outputs.run-tests == 'true'
     steps:
@@ -805,13 +810,12 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           - backend: postgres
           - backend: sqlite
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: ${{ matrix.backend }}
       PYTHON_MAJOR_MINOR_VERSION: ${{ needs.build-info.outputs.defaultPythonVersion }}
       MYSQL_VERSION: ${{needs.build-info.outputs.defaultMySQLVersion}}
       POSTGRES_VERSION: ${{needs.build-info.outputs.defaultPostgresVersion}}
-      RUN_TESTS: true
       TEST_TYPES: "Quarantined"
-      TEST_TYPE: ""
       NUM_RUNS: 10
       GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
@@ -848,7 +852,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         if: always()
         with:
           name: quarantined_tests
-          path: "files/test_result.xml"
+          path: "files/test_result-*.xml"
           retention-days: 7
       - name: "Upload airflow logs"
         uses: actions/upload-artifact@v2
@@ -883,6 +887,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - tests-sqlite
       - tests-mysql
       - tests-quarantined
+    env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     steps:
       - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
         uses: actions/checkout@v2
@@ -907,6 +913,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     needs: [build-info, ci-images]
     if: needs.build-info.outputs.image-build == 'true'
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: sqlite
       PYTHON_MAJOR_MINOR_VERSION: ${{ needs.build-info.outputs.defaultPythonVersion }}
       UPGRADE_TO_NEWER_DEPENDENCIES: ${{ needs.build-info.outputs.upgradeToNewerDependencies }}
@@ -952,6 +959,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       matrix:
         python-version: ${{ fromJson(needs.build-info.outputs.pythonVersions) }}
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: sqlite
       GITHUB_REGISTRY: ${{ needs.prod-images.outputs.githubRegistry }}
     steps:
@@ -982,6 +990,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         exclude: ${{ fromJson(needs.build-info.outputs.kubernetesExclude) }}
       fail-fast: false
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       BACKEND: postgres
       RUN_TESTS: "true"
       RUNTIME: "kubernetes"
@@ -1069,6 +1078,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       matrix:
         python-version: ${{ fromJson(needs.build-info.outputs.pythonVersions) }}
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       GITHUB_REGISTRY_PUSH_IMAGE_TAG: "latest"
       GITHUB_REGISTRY: ${{ needs.prod-images.outputs.githubRegistry }}
@@ -1116,6 +1126,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       matrix:
         python-version: ${{ fromJson(needs.build-info.outputs.pythonVersions) }}
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       GITHUB_REGISTRY_PUSH_IMAGE_TAG: "latest"
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
@@ -1147,6 +1158,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - build-info
       - ci-images
     env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       GITHUB_REGISTRY: ${{ needs.ci-images.outputs.githubRegistry }}
     # Only run it for direct pushes
@@ -1204,6 +1216,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     if: >
       github.ref == 'refs/heads/master' || github.ref == 'refs/heads/v1-10-test' ||
       github.ref == 'refs/heads/v2-0-test'
+    env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     steps:
       - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
         uses: actions/checkout@v2
@@ -1238,6 +1252,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     needs:
       - docs
+      - build-info
       - static-checks
       - static-checks-pylint
       - tests-sqlite
@@ -1247,6 +1262,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - constraints-push
       - prepare-provider-packages
     if: github.event_name == 'schedule' &&  github.repository == 'apache/airflow'
+    env:
+      RUNS_ON: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     steps:
       - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
         uses: actions/checkout@v2
diff --git a/.github/workflows/scheduled_quarantined.yml b/.github/workflows/scheduled_quarantined.yml
index 5181b02..2a5c4a2 100644
--- a/.github/workflows/scheduled_quarantined.yml
+++ b/.github/workflows/scheduled_quarantined.yml
@@ -61,7 +61,7 @@ jobs:
       PYTHON_MAJOR_MINOR_VERSION: ${{ matrix.python-version }}
       POSTGRES_VERSION: ${{ matrix.postgres-version }}
       RUN_TESTS: "true"
-      TEST_TYPE: Quarantined
+      FORCE_TEST_TYPE: Quarantined
       NUM_RUNS: 20
       GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
     steps:
@@ -96,7 +96,7 @@ jobs:
         if: always()
         with:
           name: 'quarantined_tests'
-          path: 'files/test_result.xml'
+          path: 'files/test_result-*.xml'
           retention-days: 7
       - uses: actions/upload-artifact@v2
         name: Upload airflow logs
diff --git a/BREEZE.rst b/BREEZE.rst
index 6e71868..315c85b 100644
--- a/BREEZE.rst
+++ b/BREEZE.rst
@@ -2344,7 +2344,7 @@ This is the current syntax for  `./breeze <./breeze>`_:
   --test-type TEST_TYPE
           Type of the test to run. One of:
 
-                 All,Core,Providers,API,CLI,Integration,Other,WWW,Postgres,MySQL,Helm
+                 All,Core,Providers,API,CLI,Integration,Other,WWW,Postgres,MySQL,Helm,Quarantined
 
           Default: All
 
@@ -2765,7 +2765,7 @@ This is the current syntax for  `./breeze <./breeze>`_:
   --test-type TEST_TYPE
           Type of the test to run. One of:
 
-                 All,Core,Providers,API,CLI,Integration,Other,WWW,Postgres,MySQL,Helm
+                 All,Core,Providers,API,CLI,Integration,Other,WWW,Postgres,MySQL,Helm,Quarantined
 
           Default: All
 
diff --git a/TESTING.rst b/TESTING.rst
index 0382080..7119129 100644
--- a/TESTING.rst
+++ b/TESTING.rst
@@ -418,6 +418,86 @@ Those tests are marked with ``@pytest.mark.quarantined`` annotation.
 Those tests are skipped by default. You can enable them with ``--include-quarantined`` flag. You
 can also decide to only run tests with ``-m quarantined`` flag to run only those tests.
 
+
+Airflow test types
+==================
+
+Airflow tests in the CI environment are split into several test types:
+
+* Always - those are tests that should be always executed (always folder)
+* Core - for the core Airflow functionality (core folder)
+* API - Tests for the Airflow API (api and api_connexion folders)
+* CLI - Tests for the Airflow CLI (cli folder)
+* WWW - Tests for the Airflow webserver (www and www_rbac in 1.10 folders)
+* Providers - Tests for all Providers of Airflow (providers folder)
+* Other - all other tests (all other folders that are not part of any of the above)
+
+This is done for three reasons:
+
+1. in order to selectively run only subset of the test types for some PRs
+2. in order to allow parallel execution of the tests on Self-Hosted runners
+
+For case 1. see `Pull Request Workflow <PULL_REQUEST_WORKFLOW.rst#selective-ci-checks>`_  for details.
+
+For case 2. We can utilise memory and CPUs available on both CI and local development machines to run
+test in parallel. This way we can decrease the time of running all tests in self-hosted runners from
+60 minutes to ~15 minutes.
+
+.. note::
+
+  We need to split tests manually into separate suites rather than utilise
+  ``pytest-xdist`` or ``pytest-parallel`` which could ba a simpler and much more "native" parallelization
+  mechanism. Unfortunately, we cannot utilise those tools because our tests are not truly ``unit`` tests that
+  can run in parallel. A lot of our tests rely on shared databases - and they update/reset/cleanup the
+  databases while they are executing. They are also exercising features of the Database such as locking which
+  further increases cross-dependency between tests. Until we make all our tests truly unit tests (and not
+  touching the database or until we isolate all such tests to a separate test type, we cannot really rely on
+  frameworks that run tests in parallel. In our solution each of the test types is run in parallel with its
+  own database (!) so when we have 8 test types running in parallel, there are in fact 8 databases run
+  behind the scenes to support them and each of the test types executes its own tests sequentially.
+
+
+Running full Airflow test suite in parallel
+===========================================
+
+If you run ``./scripts/ci/testing/ci_run_airflow_testing.sh`` tests run in parallel
+on your development machine - maxing out the number of parallel runs at the number of cores you
+have available in your Docker engine.
+
+In case you do not have enough memory available to your Docker (~32 GB), the ``Integration`` test type
+is always run sequentially - after all tests are completed (docker cleanup is performed in-between).
+
+This allows for massive speedup in full test execution. On 8 CPU machine with 16 cores and 64 GB memory
+and fast SSD disk, the whole suite of tests completes in about 5 minutes (!). Same suite of tests takes
+more than 30 minutes on the same machine when tests are run sequentially.
+
+.. note::
+
+  On MacOS you might have less CPUs and less memory available to run the tests than you have in the host,
+  simply because your Docker engine runs in a Linux Virtual Machine under-the-hood. If you want to make
+  use of the paralllelism and memory usage for the CI tests you might want to increase the resources available
+  to your docker engine. See the `Resources <https://docs.docker.com/docker-for-mac/#resources>`_ chapter
+  in the ``Docker for Mac`` documentation on how to do it.
+
+You can also limit the parallelism by specifying the maximum number of parallel jobs via
+MAX_PARALLEL_TEST_JOBS variable. If you set it to "1", all the test types will be run sequentially.
+
+.. code-block:: bash
+
+    MAX_PARALLEL_TEST_JOBS="1" ./scripts/ci/testing/ci_run_airflow_testing.sh
+
+.. note::
+
+  In case you would like to cleanup after execution of such tests you might have to cleanup
+  some of the docker containers running in case you use ctrl-c to stop execution. You can easily do it by
+  running this command (it will kill all docker containers running so do not use it if you want to keep some
+  docker containers running):
+
+  .. code-block:: bash
+
+      docker kill $(docker ps -q)
+
+
 Running Tests with provider packages
 ====================================
 
diff --git a/airflow/__init__.py b/airflow/__init__.py
index 3e84cc6..7ecc487 100644
--- a/airflow/__init__.py
+++ b/airflow/__init__.py
@@ -47,6 +47,7 @@ settings.initialize()
 
 login: Optional[Callable] = None
 
+PY36 = sys.version_info >= (3, 6)
 PY37 = sys.version_info >= (3, 7)
 PY38 = sys.version_info >= (3, 8)
 
diff --git a/breeze b/breeze
index c17bcb4..94a8b86 100755
--- a/breeze
+++ b/breeze
@@ -665,6 +665,7 @@ function breeze::prepare_command_files() {
     local main_ci_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/base.yml
     local main_prod_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/base.yml
     local backend_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml
+    local backend_port_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}-port.yml
     local local_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/local.yml
     local local_all_sources_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/local-all-sources.yml
     local files_docker_compose_file=${SCRIPTS_CI_DIR}/docker-compose/files.yml
@@ -676,13 +677,13 @@ function breeze::prepare_command_files() {
     local compose_prod_file=${main_prod_docker_compose_file}:${backend_docker_compose_file}:${files_docker_compose_file}
 
     if [[ "${MOUNT_SELECTED_LOCAL_SOURCES}" != "false" ]]; then
-        compose_ci_file=${compose_ci_file}:${local_docker_compose_file}
-        compose_prod_file=${compose_prod_file}:${local_prod_docker_compose_file}
+        compose_ci_file=${compose_ci_file}:${local_docker_compose_file}:${backend_port_docker_compose_file}
+        compose_prod_file=${compose_prod_file}:${local_prod_docker_compose_file}:${backend_port_docker_compose_file}
     fi
 
     if [[ "${MOUNT_ALL_LOCAL_SOURCES}" != "false" ]]; then
-        compose_ci_file=${compose_ci_file}:${local_all_sources_docker_compose_file}
-        compose_prod_file=${compose_prod_file}:${local_all_sources_docker_compose_file}
+        compose_ci_file=${compose_ci_file}:${local_all_sources_docker_compose_file}:${backend_port_docker_compose_file}
+        compose_prod_file=${compose_prod_file}:${local_all_sources_docker_compose_file}:${backend_port_docker_compose_file}
     fi
 
     if [[ ${FORWARD_CREDENTIALS} == "true" ]]; then
@@ -3433,7 +3434,7 @@ function breeze::run_breeze_command() {
 
     case "${command_to_run}" in
     enter_breeze)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         if [[ ${PRODUCTION_IMAGE} == "true" ]]; then
             ${run_command} "${dc_run_file}" run --service-ports --rm airflow "${@}"
             ${run_command} "${SCRIPTS_CI_DIR}/tools/ci_fix_ownership.sh"
@@ -3442,7 +3443,7 @@ function breeze::run_breeze_command() {
         fi
         ;;
     run_exec)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         # Unfortunately `docker-compose exec` does not support exec'ing into containers started with run :(
         # so we have to find it manually
         set +e
@@ -3454,19 +3455,19 @@ function breeze::run_breeze_command() {
             "/opt/airflow/scripts/in_container/entrypoint_exec.sh" "${@}"
         ;;
     run_tests)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         export RUN_TESTS="true"
         readonly RUN_TESTS
         ${run_command} "${BUILD_CACHE_DIR}/${DOCKER_COMPOSE_RUN_SCRIPT_FOR_CI}" run --service-ports --rm airflow "$@"
         ;;
     run_docker_compose)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         set +u
         ${run_command} "${dc_run_file}" "${docker_compose_command}" "${EXTRA_DC_OPTIONS[@]}" "$@"
         set -u
         ;;
     perform_static_checks)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         breeze::make_sure_precommit_is_installed
         breeze::run_static_checks "${@}"
         ;;
@@ -3476,19 +3477,19 @@ function breeze::run_breeze_command() {
         breeze::remove_images
         ;;
     perform_generate_constraints)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         runs::run_generate_constraints
         ;;
     perform_prepare_airflow_packages)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         build_airflow_packages::build_airflow_packages
         ;;
     perform_prepare_provider_packages)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         runs::run_prepare_provider_packages "${@}"
         ;;
     perform_prepare_provider_documentation)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         runs::run_prepare_provider_documentation "${@}"
         ;;
     perform_push_image)
@@ -3505,13 +3506,13 @@ function breeze::run_breeze_command() {
         breeze::setup_autocomplete
         ;;
     manage_kind_cluster)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         kind::make_sure_kubernetes_tools_are_installed
         kind::get_kind_cluster_name
         kind::perform_kind_cluster_operation "${KIND_CLUSTER_OPERATION}"
         ;;
     build_docs)
-        docker_engine::check_all_resources
+        docker_engine_resources::check_all_resources
         runs::run_docs "${@}"
         ;;
     toggle_suppress_cheatsheet)
diff --git a/breeze-complete b/breeze-complete
index 901bd6a..9c9f575 100644
--- a/breeze-complete
+++ b/breeze-complete
@@ -36,7 +36,7 @@ _breeze_allowed_kind_versions="v0.8.0"
 _breeze_allowed_mysql_versions="5.7 8"
 _breeze_allowed_postgres_versions="9.6 10 11 12 13"
 _breeze_allowed_kind_operations="start stop restart status deploy test shell k9s"
-_breeze_allowed_test_types="All Core Providers API CLI Integration Other WWW Postgres MySQL Helm"
+_breeze_allowed_test_types="All Core Providers API CLI Integration Other WWW Postgres MySQL Helm Quarantined"
 _breeze_allowed_package_formats="both sdist wheel"
 _breeze_allowed_installation_methods=". apache-airflow"
 
diff --git a/scripts/ci/mysql/conf.d/airflow.cnf b/scripts/ci/docker-compose/backend-mysql-port.yml
similarity index 85%
copy from scripts/ci/mysql/conf.d/airflow.cnf
copy to scripts/ci/docker-compose/backend-mysql-port.yml
index 9b266a2..495f5f7 100644
--- a/scripts/ci/mysql/conf.d/airflow.cnf
+++ b/scripts/ci/docker-compose/backend-mysql-port.yml
@@ -1,4 +1,3 @@
-#
 # 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
@@ -15,10 +14,9 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-
-[mysqld]
-explicit_defaults_for_timestamp = 1
-secure_file_priv = "/var/lib/mysql"
-local_infile = 1
-innodb_print_all_deadlocks = 1
+---
+version: "2.2"
+services:
+  mysql:
+    ports:
+      - "${MYSQL_HOST_PORT}:3306"
diff --git a/scripts/ci/docker-compose/backend-mysql.yml b/scripts/ci/docker-compose/backend-mysql.yml
index 94b03ce..6574dd7 100644
--- a/scripts/ci/docker-compose/backend-mysql.yml
+++ b/scripts/ci/docker-compose/backend-mysql.yml
@@ -44,8 +44,5 @@ services:
       timeout: 10s
       retries: 5
     restart: always
-
-    ports:
-      - "${MYSQL_HOST_PORT}:3306"
     command: ['mysqld', '--character-set-server=utf8mb4',
               '--collation-server=utf8mb4_unicode_ci']
diff --git a/scripts/ci/mysql/conf.d/airflow.cnf b/scripts/ci/docker-compose/backend-postgres-port.yml
similarity index 85%
copy from scripts/ci/mysql/conf.d/airflow.cnf
copy to scripts/ci/docker-compose/backend-postgres-port.yml
index 9b266a2..a7d78f9 100644
--- a/scripts/ci/mysql/conf.d/airflow.cnf
+++ b/scripts/ci/docker-compose/backend-postgres-port.yml
@@ -1,4 +1,3 @@
-#
 # 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
@@ -15,10 +14,9 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-
-[mysqld]
-explicit_defaults_for_timestamp = 1
-secure_file_priv = "/var/lib/mysql"
-local_infile = 1
-innodb_print_all_deadlocks = 1
+---
+version: "2.2"
+services:
+  postgres:
+    ports:
+      - "${POSTGRES_HOST_PORT}:5432"
diff --git a/scripts/ci/docker-compose/backend-postgres.yml b/scripts/ci/docker-compose/backend-postgres.yml
index be78339..df3615d 100644
--- a/scripts/ci/docker-compose/backend-postgres.yml
+++ b/scripts/ci/docker-compose/backend-postgres.yml
@@ -36,8 +36,6 @@ services:
     volumes:
       - /dev/urandom:/dev/random   # Required to get non-blocking entropy source
       - postgres-db-volume:/var/lib/postgresql/data
-    ports:
-      - "${POSTGRES_HOST_PORT}:5432"
     healthcheck:
       test: ["CMD", "psql", "-h", "localhost", "-U", "postgres", "-c", "select 1", "airflow"]
       interval: 10s
diff --git a/scripts/ci/mysql/conf.d/airflow.cnf b/scripts/ci/docker-compose/backend-sqlite-port.yml
similarity index 85%
copy from scripts/ci/mysql/conf.d/airflow.cnf
copy to scripts/ci/docker-compose/backend-sqlite-port.yml
index 9b266a2..c7bbb7d 100644
--- a/scripts/ci/mysql/conf.d/airflow.cnf
+++ b/scripts/ci/docker-compose/backend-sqlite-port.yml
@@ -1,4 +1,3 @@
-#
 # 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
@@ -15,10 +14,5 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-
-[mysqld]
-explicit_defaults_for_timestamp = 1
-secure_file_priv = "/var/lib/mysql"
-local_infile = 1
-innodb_print_all_deadlocks = 1
+---
+version: "2.2"
diff --git a/scripts/ci/libraries/_all_libs.sh b/scripts/ci/libraries/_all_libs.sh
index aa48552..09a147d 100755
--- a/scripts/ci/libraries/_all_libs.sh
+++ b/scripts/ci/libraries/_all_libs.sh
@@ -28,6 +28,8 @@ readonly SCRIPTS_CI_DIR
 . "${LIBRARIES_DIR}"/_traps.sh
 # shellcheck source=scripts/ci/libraries/_initialization.sh
 . "${LIBRARIES_DIR}"/_initialization.sh
+# shellcheck source=scripts/ci/libraries/_parallel.sh
+. "${LIBRARIES_DIR}"/_parallel.sh
 # shellcheck source=scripts/ci/libraries/_docker_engine_resources.sh
 . "${LIBRARIES_DIR}"/_docker_engine_resources.sh
 # shellcheck source=scripts/ci/libraries/_repeats.sh
diff --git a/scripts/ci/libraries/_docker_engine_resources.sh b/scripts/ci/libraries/_docker_engine_resources.sh
index a3bd12a..18b223d 100644
--- a/scripts/ci/libraries/_docker_engine_resources.sh
+++ b/scripts/ci/libraries/_docker_engine_resources.sh
@@ -16,7 +16,18 @@
 # specific language governing permissions and limitations
 # under the License.
 
-function docker_engine::get_available_memory_in_docker() {
+
+function docker_engine_resources::print_overall_stats() {
+    echo
+    echo "Overall resource statistics"
+    echo
+    docker stats --all --no-stream --no-trunc
+    docker run --rm --entrypoint /bin/bash "${AIRFLOW_CI_IMAGE}" -c "free -h"
+    df --human || true
+}
+
+
+function docker_engine_resources::get_available_memory_in_docker() {
     MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --rm --entrypoint /bin/bash \
         "${AIRFLOW_CI_IMAGE}" -c \
         'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
@@ -24,7 +35,7 @@ function docker_engine::get_available_memory_in_docker() {
     export MEMORY_AVAILABLE_FOR_DOCKER
 }
 
-function docker_engine::get_available_cpus_in_docker() {
+function docker_engine_resources::get_available_cpus_in_docker() {
     CPUS_AVAILABLE_FOR_DOCKER=$(docker run --rm --entrypoint /bin/bash \
         "${AIRFLOW_CI_IMAGE}" -c \
         'grep -cE "cpu[0-9]+" </proc/stat')
@@ -32,7 +43,7 @@ function docker_engine::get_available_cpus_in_docker() {
     export CPUS_AVAILABLE_FOR_DOCKER
 }
 
-function docker_engine::get_available_disk_space_in_docker() {
+function docker_engine_resources::get_available_disk_space_in_docker() {
     DISK_SPACE_AVAILABLE_FOR_DOCKER=$(docker run --rm --entrypoint /bin/bash \
         "${AIRFLOW_CI_IMAGE}" -c \
         'df  / | tail -1 | awk '\''{print $4}'\')
@@ -40,7 +51,7 @@ function docker_engine::get_available_disk_space_in_docker() {
     export DISK_SPACE_AVAILABLE_FOR_DOCKER
 }
 
-function docker_engine::check_enough_resources() {
+function docker_engine_resources::check_enough_resources() {
     local successful_resource_check="true"
     if (( MEMORY_AVAILABLE_FOR_DOCKER < 4000 )) ; then
         successful_resource_check="false"
@@ -67,9 +78,9 @@ function docker_engine::check_enough_resources() {
     fi
 }
 
-function docker_engine::check_all_resources() {
-    docker_engine::get_available_memory_in_docker
-    docker_engine::get_available_cpus_in_docker
-    docker_engine::get_available_disk_space_in_docker
-    docker_engine::check_enough_resources
+function docker_engine_resources::check_all_resources() {
+    docker_engine_resources::get_available_memory_in_docker
+    docker_engine_resources::get_available_cpus_in_docker
+    docker_engine_resources::get_available_disk_space_in_docker
+    docker_engine_resources::check_enough_resources
 }
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index 24af1e6..66923a6 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -529,7 +529,10 @@ function initialization::initialize_github_variables() {
 }
 
 function initialization::initialize_test_variables() {
-    export TEST_TYPE=${TEST_TYPE:=""}
+
+    # In case we want to force certain test type to run, this variable should be set to this type
+    # Otherwise TEST_TYPEs to run will be derived from TEST_TYPES space-separated string
+    export FORCE_TEST_TYPE=${FORCE_TEST_TYPE:=""}
 }
 
 function initialization::initialize_package_variables() {
@@ -746,10 +749,6 @@ function initialization::make_constants_read_only() {
     # Set the arguments as read-only
     readonly PYTHON_MAJOR_MINOR_VERSION
 
-    readonly WEBSERVER_HOST_PORT
-    readonly POSTGRES_HOST_PORT
-    readonly MYSQL_HOST_PORT
-
     readonly HOST_USER_ID
     readonly HOST_GROUP_ID
     readonly HOST_HOME
diff --git a/scripts/ci/libraries/_parallel.sh b/scripts/ci/libraries/_parallel.sh
new file mode 100644
index 0000000..f8a92ab
--- /dev/null
+++ b/scripts/ci/libraries/_parallel.sh
@@ -0,0 +1,111 @@
+#!/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.
+
+function parallel::initialize_monitoring() {
+    PARALLEL_MONITORED_DIR="$(mktemp -d)"
+    export PARALLEL_MONITORED_DIR
+
+    PARALLEL_JOBLOG="$(mktemp)"
+    export PARALLEL_JOBLOG
+}
+
+function parallel::make_sure_gnu_parallel_is_installed() {
+    start_end::group_start "Making sure GNU Parallels is installed"
+    echo
+    echo "Making sure you have GNU parallel installed"
+    echo
+    echo "You might need to provide root password if you do not have it"
+    echo
+    (command -v parallel || apt install parallel || sudo apt install parallel || brew install parallel) >/dev/null
+    start_end::group_end "Making sure GNU Parallels is installed"
+}
+
+function parallel::kill_stale_semaphore_locks() {
+    local pid
+    echo
+    echo "${COLOR_BLUE}Killing stale semaphore locks${COLOR_RESET}"
+    echo
+    for s in "${HOME}/.parallel/semaphores/id-${SEMAPHORE_NAME}/"*@*
+    do
+        pid="${s%%@*}"
+        if [[ ${pid} != "-*" ]]; then
+            kill -15 -- -"$(basename "${s%%@*}")" 2>/dev/null || true
+            rm -f "${s}" 2>/dev/null
+        fi
+    done
+}
+
+
+# Periodical loop to print summary of all the processes run by parallel
+function parallel::monitor_loop() {
+    echo
+    echo "Start monitoring of parallel execution in ${PARALLEL_MONITORED_DIR} directory."
+    echo
+    local progress_report_number=1
+    local start_time
+    local end_time
+    start_time=${SECONDS}
+    while true
+    do
+        echo
+        echo "${COLOR_YELLOW}########### Monitoring progress start: ${progress_report_number} #################${COLOR_RESET}"
+        echo
+        echo "${COLOR_BLUE}########### STATISTICS #################"
+        docker_engine_resources::print_overall_stats
+        echo "########### STATISTICS #################${COLOR_RESET}"
+        for directory in "${PARALLEL_MONITORED_DIR}"/*/*
+        do
+            parallel_process=$(basename "${directory}")
+            echo "${COLOR_BLUE}### The last lines for ${parallel_process} process ###${COLOR_RESET}"
+            echo
+            tail -2 "${directory}/stdout" || true
+            echo
+            echo
+        done
+        echo
+        echo "${COLOR_YELLOW}########### Monitoring progress end: ${progress_report_number} #################${COLOR_RESET}}"
+        echo
+        end_time=${SECONDS}
+        echo "${COLOR_YELLOW}############## $((end_time - start_time)) seconds passed since start ####################### ${COLOR_RESET}"
+        sleep 10
+        progress_report_number=$((progress_report_number + 1))
+    done
+    echo "${COLOR_BLUE}########### STATISTICS #################"
+    docker_engine_resources::print_overall_stats
+    echo "########### STATISTICS #################${COLOR_RESET}"
+}
+
+# Monitors progress of parallel execution and periodically summarizes stdout entries created by
+# the parallel execution. Sets PAPARALLEL_MONITORED_DIR which should be be passed as --results
+# parameter to GNU parallel execution.
+function parallel::monitor_progress() {
+    echo "Parallel results are stored in: ${PARALLEL_MONITORED_DIR}"
+    echo "Parallel joblog is stored in: ${PARALLEL_JOBLOG}"
+
+    parallel::monitor_loop 2>/dev/null &
+
+    # shellcheck disable=SC2034
+    PARALLEL_MONITORING_PID=$!
+    # shellcheck disable=SC2016
+    traps::add_trap 'parallel::kill_monitor' EXIT
+}
+
+
+function parallel::kill_monitor() {
+    kill -9 ${PARALLEL_MONITORING_PID} >/dev/null 2>&1 || true
+}
diff --git a/scripts/ci/libraries/_start_end.sh b/scripts/ci/libraries/_start_end.sh
index 32ef4f4..b7fa74f 100644
--- a/scripts/ci/libraries/_start_end.sh
+++ b/scripts/ci/libraries/_start_end.sh
@@ -124,7 +124,7 @@ function start_end::script_end {
 
     END_SCRIPT_TIME=$(date +%s)
     RUN_SCRIPT_TIME=$((END_SCRIPT_TIME-START_SCRIPT_TIME))
-    if [[ ${BREEZE:=} != "true" ]]; then
+    if [[ ${BREEZE:=} != "true" && ${RUN_TESTS=} != "true" ]]; then
         verbosity::print_info
         verbosity::print_info "Finished the script ${COLOR_GREEN}$(basename "$0")${COLOR_RESET}"
         verbosity::print_info "Elapsed time spent in the script: ${COLOR_BLUE}${RUN_SCRIPT_TIME} seconds${COLOR_RESET}"
diff --git a/scripts/ci/mysql/conf.d/airflow.cnf b/scripts/ci/mysql/conf.d/airflow.cnf
index 9b266a2..a5ab88c 100644
--- a/scripts/ci/mysql/conf.d/airflow.cnf
+++ b/scripts/ci/mysql/conf.d/airflow.cnf
@@ -22,3 +22,10 @@ explicit_defaults_for_timestamp = 1
 secure_file_priv = "/var/lib/mysql"
 local_infile = 1
 innodb_print_all_deadlocks = 1
+
+# Optimizes memory usage during tests - by default it 132 MB but we need far less than that
+innodb_buffer_pool_size = 32M
+
+# Performance schema monitoring uses on its own a lot of memory. We save ~130 MB by disabling it
+performance_schema     = OFF
+performance-schema-instrument='memory/%=COUNTED'
diff --git a/scripts/ci/selective_ci_checks.sh b/scripts/ci/selective_ci_checks.sh
index 456fcef..e1ca0be 100755
--- a/scripts/ci/selective_ci_checks.sh
+++ b/scripts/ci/selective_ci_checks.sh
@@ -199,7 +199,7 @@ function set_upgrade_to_newer_dependencies() {
 }
 
 
-ALL_TESTS="Always Core Other API CLI Providers WWW Integration"
+ALL_TESTS="Always API Core Other CLI Providers WWW Integration"
 readonly ALL_TESTS
 
 function set_outputs_run_everything_and_exit() {
diff --git a/scripts/ci/testing/ci_run_airflow_testing.sh b/scripts/ci/testing/ci_run_airflow_testing.sh
index 7ea01d7..1cd1c36 100755
--- a/scripts/ci/testing/ci_run_airflow_testing.sh
+++ b/scripts/ci/testing/ci_run_airflow_testing.sh
@@ -15,11 +15,14 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+
+# Enable automated tests execution
+RUN_TESTS="true"
+export RUN_TESTS
+
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-DOCKER_COMPOSE_LOCAL=()
-
 if [[ -f ${BUILD_CACHE_DIR}/.skip_tests ]]; then
     echo
     echo "Skipping running tests !!!!!"
@@ -27,65 +30,18 @@ if [[ -f ${BUILD_CACHE_DIR}/.skip_tests ]]; then
     exit
 fi
 
-function run_airflow_testing_in_docker() {
-    set +u
-    set +e
-    local exit_code
-    for try_num in {1..5}
-    do
-        echo
-        echo "Making sure docker-compose is down and remnants removed"
-        echo
-        docker-compose --log-level INFO -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-            down --remove-orphans --volumes --timeout 10
-        echo
-        echo "System-prune docker"
-        echo
-        docker system prune --force --volumes
-        echo
-        echo "Check available space"
-        echo
-        df --human
-        echo
-        echo "Check available memory"
-        echo
-        free --human
-        echo
-        echo "Starting try number ${try_num}"
-        echo
-        docker-compose --log-level INFO \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml" \
-          "${INTEGRATIONS[@]}" \
-          "${DOCKER_COMPOSE_LOCAL[@]}" \
-             run airflow "${@}"
-        exit_code=$?
-        if [[ ${exit_code} == "254" && ${try_num} != "5" ]]; then
-            echo
-            echo "Failed try num ${try_num}. Sleeping 5 seconds for retry"
-            echo
-            sleep 5
-            continue
-        else
-            break
-        fi
-    done
-    if [[ ${TEST_TYPE:=} == "Quarantined" ]]; then
-        if [[ ${exit_code} == "1" ]]; then
-            echo
-            echo "Some Quarantined tests failed. but we recorded it in an issue"
-            echo
-            exit_code="0"
-        else
-            echo
-            echo "All Quarantined tests succeeded"
-            echo
-        fi
-    fi
-    set -u
-    set -e
-    return "${exit_code}"
-}
+# In case we see too many failures on regular PRs from our users using GitHub Public runners
+# We can uncomment this and come back to sequential test-type execution
+#if [[ ${RUNS_ON} != *"self-hosted"* ]]; then
+#    echo
+#    echo "${COLOR_YELLOW}This is a Github Public runner - for now we are forcing max parallel jobs to 1 for those${COLOR_RESET}"
+#    echo "${COLOR_YELLOW}Until we fix memory usage to allow up to 2 parallel runs on those runners${COLOR_RESET}"
+#    echo
+#    # Forces testing in parallel in case the script is run on self-hosted runners
+#    export MAX_PARALLEL_TEST_JOBS="1"
+#fi
+
+SEMAPHORE_NAME="tests"
 
 function prepare_tests_to_run() {
     DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/files.yml")
@@ -109,76 +65,228 @@ function prepare_tests_to_run() {
     fi
     readonly DOCKER_COMPOSE_LOCAL
 
-    if [[ -n "${TEST_TYPE=}" ]]; then
-        # Handle case where test type is passed from outside
-        export TEST_TYPES="${TEST_TYPE}"
+    if [[ -n "${FORCE_TEST_TYPE=}" ]]; then
+        # Handle case where test type is forced from outside
+        export TEST_TYPES="${FORCE_TEST_TYPE}"
     fi
 
     if [[ -z "${TEST_TYPES=}" ]]; then
         TEST_TYPES="Core Providers API CLI Integration Other WWW"
         echo
-        echo "Test types not specified. Running all: ${TEST_TYPES}"
+        echo "Test types not specified. Adding all: ${TEST_TYPES}"
         echo
     fi
 
-    if [[ -n "${TEST_TYPE=}" ]]; then
+    if [[ -z "${FORCE_TEST_TYPE=}" ]]; then
         # Add Postgres/MySQL special test types in case we are running several test types
-        if [[ ${BACKEND} == "postgres" ]]; then
+        if [[ ${BACKEND} == "postgres" && ${TEST_TYPES} != "Quarantined" ]]; then
             TEST_TYPES="${TEST_TYPES} Postgres"
+            echo
+            echo "Added Postgres. Tests to run: ${TEST_TYPES}"
+            echo
         fi
-        if [[ ${BACKEND} == "mysql" ]]; then
+        if [[ ${BACKEND} == "mysql" && ${TEST_TYPES} != "Quarantined" ]]; then
             TEST_TYPES="${TEST_TYPES} MySQL"
+            echo
+            echo "Added MySQL. Tests to run: ${TEST_TYPES}"
+            echo
         fi
     fi
     readonly TEST_TYPES
 }
 
-build_images::prepare_ci_build
-
-build_images::rebuild_ci_image_if_needed_with_group
-
-prepare_tests_to_run
+function kill_all_running_docker_containers() {
+    echo
+    echo "${COLOR_BLUE}Kill all running docker containers${COLOR_RESET}"
+    echo
+    # shellcheck disable=SC2046
+    docker kill $(docker ps -q) || true
+}
 
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-docker_engine::get_available_memory_in_docker
-docker_engine::get_available_cpus_in_docker
-docker_engine::get_available_disk_space_in_docker
+function get_maximum_parallel_test_jobs() {
+    if [[ ${MAX_PARALLEL_TEST_JOBS=} != "" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Maximum parallel test jobs forced vi MAX_PARALLEL_TEST_JOBS = ${MAX_PARALLEL_TEST_JOBS}${COLOR_RESET}"
+        echo
+    else
+        MAX_PARALLEL_TEST_JOBS=${CPUS_AVAILABLE_FOR_DOCKER}
+        echo
+        echo "${COLOR_YELLOW}Maximum parallel test jobs set to number of CPUs available for Docker = ${MAX_PARALLEL_TEST_JOBS}${COLOR_RESET}"
+        echo
+    fi
+    export MAX_PARALLEL_TEST_JOBS
+}
 
+# Cleans up runner before test execution.
+#  * Kills all running docker containers
+#  * System prune to clean all the temporary/unnamed images and left-over volumes
+#  * Print information about available space and memory
+#  * Kills stale semaphore locks
+function cleanup_runner() {
+    start_end::group_start "Cleanup runner"
+    kill_all_running_docker_containers
+    system_prune_docker
+    docker_engine_resources::get_available_memory_in_docker
+    docker_engine_resources::get_available_cpus_in_docker
+    docker_engine_resources::get_available_disk_space_in_docker
+    docker_engine_resources::print_overall_stats
+    get_maximum_parallel_test_jobs
+    parallel::kill_stale_semaphore_locks
+    start_end::group_end
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+# Starts test types in parallel
+# test_types_to_run - list of test types (it's not an array, it is space-separate list)
+# ${@} - additional arguments to pass to test execution
+function run_test_types_in_parallel() {
+    start_end::group_start "Monitoring tests: ${test_types_to_run}"
+    parallel::monitor_progress
+    mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}"
+    for TEST_TYPE in ${test_types_to_run}
+    do
+        export TEST_TYPE
+        mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${TEST_TYPE}"
+        mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${TEST_TYPE}"
+        export JOB_LOG="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${TEST_TYPE}/stdout"
+        export PARALLEL_JOB_STATUS="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${TEST_TYPE}/status"
+        # shellcheck disable=SC2086
+        parallel --ungroup --bg --semaphore --semaphorename "${SEMAPHORE_NAME}" \
+            --jobs "${MAX_PARALLEL_TEST_JOBS}" \
+            "$( dirname "${BASH_SOURCE[0]}" )/ci_run_single_airflow_test_in_docker.sh" "${@}" >${JOB_LOG} 2>&1
+    done
+    parallel --semaphore --semaphorename "${SEMAPHORE_NAME}" --wait
+    parallel::kill_monitor
+    start_end::group_end
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
+# Outputs logs for successful test type
+# $1 test type
+function output_log_for_successful_test_type(){
+    local test_type=$1
+    local log_dir="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${test_type}"
+    start_end::group_start "${COLOR_GREEN}Output for successful ${test_type}${COLOR_RESET}"
+    echo "${COLOR_GREEN}##### Test type ${test_type} succeeded ##### ${COLOR_RESET}"
+    echo
+    cat "${log_dir}"/stdout
+    echo
+    echo "${COLOR_GREEN}##### Test type ${test_type} succeeded ##### ${COLOR_RESET}"
+    echo
+    start_end::group_end
+}
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
-    else
-        export ENABLED_INTEGRATIONS=""
-        export RUN_INTEGRATION_TESTS=""
-    fi
+# Outputs logs for failed test type
+# $1 test type
+function output_log_for_failed_test_type(){
+    local test_type=$1
+    local log_dir="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${test_type}"
+    start_end::group_start "${COLOR_RED}Output: for failed ${test_type}${COLOR_RESET}"
+    echo "${COLOR_RED}##### Test type ${test_type} failed ##### ${COLOR_RESET}"
+    echo
+    cat "${log_dir}"/stdout
+    echo
+    echo
+    echo "${COLOR_RED}##### Test type ${test_type} failed ##### ${COLOR_RESET}"
+    echo
+    start_end::group_end
+}
 
-    for _INT in ${ENABLED_INTEGRATIONS}
+# Prints summary of tests and returns status:
+# 0 - all test types succeeded (Quarantine is not counted)
+# >0 - number of failed test types (except Quarantine)
+function print_test_summary_and_return_test_status_code() {
+    local return_code="0"
+    local test_type
+    for test_type in ${TEST_TYPES}
     do
-        INTEGRATIONS+=("-f")
-        INTEGRATIONS+=("${SCRIPTS_CI_DIR}/docker-compose/integration-${_INT}.yml")
+        status=$(cat "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${test_type}/status")
+        if [[ ${status} == "0" ]]; then
+            output_log_for_successful_test_type "${test_type}"
+        else
+            output_log_for_failed_test_type "${test_type}"
+            # Quarantined tests failure does not trigger whole test failure
+            if [[ ${TEST_TYPE} != "Quarantined" ]]; then
+                return_code=$((return_code + 1))
+            fi
+        fi
     done
+    return "${return_code}"
+}
+
+export MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN=33000
+
+# Runs all test types in parallel depending on the number of CPUs available
+# We monitors their progress, display the progress  and summarize the result when finished.
+#
+# In case there is not enough memory (MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN) available for
+# the docker engine, the integration tests (which take a lot of memory for all the integrations)
+# are run sequentially after all other tests were run in parallel.
+#
+# Input:
+#   * TEST_TYPES  - contains all test types that should be executed
+#   * MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN - memory in bytes required to run integration tests
+#             in parallel to other tests
+#   * MEMORY_AVAILABLE_FOR_DOCKER - memory that is available in docker (set by cleanup_runners)
+#
+function run_all_test_types_in_parallel() {
+    local test_type
 
-    export TEST_TYPE
+    cleanup_runner
 
-    echo "**********************************************************************************************"
+    start_end::group_start "Determine how to run the tests"
     echo
-    echo "      TEST_TYPE: ${TEST_TYPE}, ENABLED INTEGRATIONS: ${ENABLED_INTEGRATIONS}"
+    echo "${COLOR_YELLOW}Running maximum ${MAX_PARALLEL_TEST_JOBS} test types in parallel${COLOR_RESET}"
     echo
-    echo "**********************************************************************************************"
-
-    run_airflow_testing_in_docker "${@}"
 
-    docker_engine::get_available_memory_in_docker
-    docker_engine::get_available_cpus_in_docker
-    docker_engine::get_available_disk_space_in_docker
+    local run_integration_tests_separately="false"
+    local test_types_to_run=${TEST_TYPES}
 
+    if [[ ${test_types_to_run} == *"Integration"* ]]; then
+        if (( MEMORY_AVAILABLE_FOR_DOCKER < MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN )) ; then
+            # In case of Integration tests - they need more resources (Memory) thus we only run them in
+            # parallel if we have more than 32 GB memory available. Otherwise we run them sequentially
+            # after cleaning up the memory and stopping all docker instances
+            echo ""
+            echo "${COLOR_YELLOW}There is not enough memory to run Integration test in parallel${COLOR_RESET}"
+            echo "${COLOR_YELLOW}   Available memory: ${MEMORY_AVAILABLE_FOR_DOCKER}${COLOR_RESET}"
+            echo "${COLOR_YELLOW}   Required memory: ${MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN}${COLOR_RESET}"
+            echo ""
+            echo "${COLOR_YELLOW}Integration tests will be run separately at the end after cleaning up docker${COLOR_RESET}"
+            echo ""
+            # Remove Integration from list of tests to run in parallel
+            test_types_to_run="${test_types_to_run//Integration/}"
+            run_integration_tests_separately="true"
+        fi
+    fi
+    set +e
     start_end::group_end
-done
+
+    parallel::initialize_monitoring
+
+    run_test_types_in_parallel "${@}"
+    if [[ ${run_integration_tests_separately} == "true" ]]; then
+        cleanup_runner
+        test_types_to_run="Integration"
+        run_test_types_in_parallel "${@}"
+    fi
+    set -e
+    # this will exit with error code in case some of the non-Quarantined tests failed
+    print_test_summary_and_return_test_status_code
+}
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed_with_group
+
+prepare_tests_to_run
+
+parallel::make_sure_gnu_parallel_is_installed
+
+run_all_test_types_in_parallel "${@}"
diff --git a/scripts/ci/testing/ci_run_airflow_testing.sh b/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
similarity index 52%
copy from scripts/ci/testing/ci_run_airflow_testing.sh
copy to scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
index 7ea01d7..76b710e 100755
--- a/scripts/ci/testing/ci_run_airflow_testing.sh
+++ b/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
@@ -15,79 +15,18 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+# Skip printing groups in CI
+PRINT_INFO_FROM_SCRIPTS="false"
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-DOCKER_COMPOSE_LOCAL=()
-
-if [[ -f ${BUILD_CACHE_DIR}/.skip_tests ]]; then
-    echo
-    echo "Skipping running tests !!!!!"
-    echo
-    exit
-fi
+PRINT_INFO_FROM_SCRIPTS="true"
+export PRINT_INFO_FROM_SCRIPTS
 
-function run_airflow_testing_in_docker() {
-    set +u
-    set +e
-    local exit_code
-    for try_num in {1..5}
-    do
-        echo
-        echo "Making sure docker-compose is down and remnants removed"
-        echo
-        docker-compose --log-level INFO -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-            down --remove-orphans --volumes --timeout 10
-        echo
-        echo "System-prune docker"
-        echo
-        docker system prune --force --volumes
-        echo
-        echo "Check available space"
-        echo
-        df --human
-        echo
-        echo "Check available memory"
-        echo
-        free --human
-        echo
-        echo "Starting try number ${try_num}"
-        echo
-        docker-compose --log-level INFO \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml" \
-          "${INTEGRATIONS[@]}" \
-          "${DOCKER_COMPOSE_LOCAL[@]}" \
-             run airflow "${@}"
-        exit_code=$?
-        if [[ ${exit_code} == "254" && ${try_num} != "5" ]]; then
-            echo
-            echo "Failed try num ${try_num}. Sleeping 5 seconds for retry"
-            echo
-            sleep 5
-            continue
-        else
-            break
-        fi
-    done
-    if [[ ${TEST_TYPE:=} == "Quarantined" ]]; then
-        if [[ ${exit_code} == "1" ]]; then
-            echo
-            echo "Some Quarantined tests failed. but we recorded it in an issue"
-            echo
-            exit_code="0"
-        else
-            echo
-            echo "All Quarantined tests succeeded"
-            echo
-        fi
-    fi
-    set -u
-    set -e
-    return "${exit_code}"
-}
+DOCKER_COMPOSE_LOCAL=()
+INTEGRATIONS=()
 
-function prepare_tests_to_run() {
+function prepare_tests() {
     DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/files.yml")
     if [[ ${MOUNT_SELECTED_LOCAL_SOURCES} == "true" ]]; then
         DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/local.yml")
@@ -96,7 +35,7 @@ function prepare_tests_to_run() {
         DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/local-all-sources.yml")
     fi
 
-    if [[ ${GITHUB_ACTIONS} == "true" ]]; then
+    if [[ ${GITHUB_ACTIONS=} == "true" ]]; then
         DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/ga.yml")
     fi
 
@@ -109,49 +48,6 @@ function prepare_tests_to_run() {
     fi
     readonly DOCKER_COMPOSE_LOCAL
 
-    if [[ -n "${TEST_TYPE=}" ]]; then
-        # Handle case where test type is passed from outside
-        export TEST_TYPES="${TEST_TYPE}"
-    fi
-
-    if [[ -z "${TEST_TYPES=}" ]]; then
-        TEST_TYPES="Core Providers API CLI Integration Other WWW"
-        echo
-        echo "Test types not specified. Running all: ${TEST_TYPES}"
-        echo
-    fi
-
-    if [[ -n "${TEST_TYPE=}" ]]; then
-        # Add Postgres/MySQL special test types in case we are running several test types
-        if [[ ${BACKEND} == "postgres" ]]; then
-            TEST_TYPES="${TEST_TYPES} Postgres"
-        fi
-        if [[ ${BACKEND} == "mysql" ]]; then
-            TEST_TYPES="${TEST_TYPES} MySQL"
-        fi
-    fi
-    readonly TEST_TYPES
-}
-
-build_images::prepare_ci_build
-
-build_images::rebuild_ci_image_if_needed_with_group
-
-prepare_tests_to_run
-
-
-docker_engine::get_available_memory_in_docker
-docker_engine::get_available_cpus_in_docker
-docker_engine::get_available_disk_space_in_docker
-
-
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
-
-    INTEGRATIONS=()
-    export INTEGRATIONS
-
     if [[ ${TEST_TYPE:=} == "Integration" ]]; then
         export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
         export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
@@ -166,19 +62,127 @@ do
         INTEGRATIONS+=("${SCRIPTS_CI_DIR}/docker-compose/integration-${_INT}.yml")
     done
 
-    export TEST_TYPE
+    readonly INTEGRATIONS
 
     echo "**********************************************************************************************"
     echo
     echo "      TEST_TYPE: ${TEST_TYPE}, ENABLED INTEGRATIONS: ${ENABLED_INTEGRATIONS}"
     echo
     echo "**********************************************************************************************"
+}
 
-    run_airflow_testing_in_docker "${@}"
+# Runs airflow testing in docker container
+# You need to set variable TEST_TYPE - test type to run
+# "${@}" - extra arguments to pass to docker command
+function run_airflow_testing_in_docker() {
+    set +u
+    set +e
+    local exit_code
+    echo
+    echo "Semaphore grabbed. Running tests for ${TEST_TYPE}"
+    echo
+    for try_num in {1..5}
+    do
+        echo
+        echo "Starting try number ${try_num}"
+        echo
+        echo
+        echo "Making sure docker-compose is down and remnants removed"
+        echo
+        docker-compose --log-level INFO -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
+            --project-name "airflow-${TEST_TYPE}" \
+            down --remove-orphans \
+            --volumes --timeout 10
+        docker-compose --log-level INFO \
+          -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
+          -f "${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml" \
+          "${INTEGRATIONS[@]}" \
+          "${DOCKER_COMPOSE_LOCAL[@]}" \
+          --project-name "airflow-${TEST_TYPE}" \
+             run airflow "${@}"
+        exit_code=$?
+        docker-compose --log-level INFO -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
+            --project-name "airflow-${TEST_TYPE}" \
+            down --remove-orphans \
+            --volumes --timeout 10
+        if [[ ${exit_code} == "254" && ${try_num} != "5" ]]; then
+            echo
+            echo "Failed try num ${try_num}. Sleeping 5 seconds for retry"
+            echo
+            sleep 5
+            continue
+        else
+            break
+        fi
+    done
+    set -u
+    set -e
+    if [[ ${exit_code} != "0" ]]; then
+        EXTRA_ARGS=""
+        if [[ ${BACKEND} == "postgres" ]]; then
+            EXTRA_ARGS="--postgres-version ${POSTGRES_VERSION} "
+        elif [[ ${BACKEND} == "mysql" ]]; then
+            EXTRA_ARGS="--mysql-version ${MYSQL_VERSION} "
+        fi
+        echo "${COLOR_RED}***********************************************************************************************${COLOR_RESET}"
+        echo "${COLOR_RED}*${COLOR_RESET}"
+        echo "${COLOR_RED}* ERROR! Some tests failed, unfortunately. Those might be transient errors,${COLOR_RESET}"
+        echo "${COLOR_RED}*        but usually you have to fix something.${COLOR_RESET}"
+        echo "${COLOR_RED}*        See the above log for details.${COLOR_RESET}"
+        echo "${COLOR_RED}*${COLOR_RESET}"
+        echo "${COLOR_RED}***********************************************************************************************${COLOR_RESET}"
+        echo """
+*  You can easily reproduce the failed tests on your dev machine/
+*
+*   When you have the source branch checked out locally:
+*
+*     Run all tests:
+*
+*       ./breeze --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} tests
+*
+*     Enter docker shell:
+*
+*       ./breeze --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} shell
+*"""
+    if [[ -n "${GITHUB_REGISTRY_PULL_IMAGE_TAG=}" ]]; then
+        echo """
+*   When you do not have sources:
+*
+*     Run all tests:
+*
+*      ./breeze --github-image-id ${GITHUB_REGISTRY_PULL_IMAGE_TAG} --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} tests
+*
+*     Enter docker shell:
+*
+*      ./breeze --github-image-id ${GITHUB_REGISTRY_PULL_IMAGE_TAG} --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} shell
+*"""
+    fi
+    echo """
+*
+*   NOTE! Once you are in the docker shell, you can run failed test with:
+*
+*            pytest [TEST_NAME]
+*
+*   You can copy the test name from the output above
+*
+***********************************************************************************************"""
+
+    fi
+
+    echo ${exit_code} > "${PARALLEL_JOB_STATUS}"
+
+    if [[ ${exit_code} == 0 ]]; then
+        echo
+        echo "${COLOR_GREEN}Test type: ${TEST_TYPE} succeeded.${COLOR_RESET}"
+        echo
+    else
+        echo
+        echo "${COLOR_RED}Test type: ${TEST_TYPE} failed.${COLOR_RESET}"
+        echo
+    fi
+    return "${exit_code}"
+}
 
-    docker_engine::get_available_memory_in_docker
-    docker_engine::get_available_cpus_in_docker
-    docker_engine::get_available_disk_space_in_docker
+prepare_tests
 
-    start_end::group_end
-done
+run_airflow_testing_in_docker "${@}"
diff --git a/scripts/ci/tools/ci_free_space_on_ci.sh b/scripts/ci/tools/ci_free_space_on_ci.sh
index e21e463..4747848 100755
--- a/scripts/ci/tools/ci_free_space_on_ci.sh
+++ b/scripts/ci/tools/ci_free_space_on_ci.sh
@@ -18,8 +18,15 @@
 # shellcheck source=scripts/ci/libraries/_script_init.sh
 . "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
+echo "${COLOR_BLUE}Disable swap${COLOR_RESET}"
 sudo swapoff -a
 sudo rm -f /swapfile
+
+echo "${COLOR_BLUE}Cleaning apt${COLOR_RESET}"
 sudo apt clean
+
+echo "${COLOR_BLUE}Pruning docker${COLOR_RESET}"
 docker system prune --all --force --volumes
+
+echo "${COLOR_BLUE}Free disk space  ${COLOR_RESET}"
 df -h
diff --git a/scripts/in_container/entrypoint_ci.sh b/scripts/in_container/entrypoint_ci.sh
index da1d383..5cb8d99 100755
--- a/scripts/in_container/entrypoint_ci.sh
+++ b/scripts/in_container/entrypoint_ci.sh
@@ -210,44 +210,39 @@ if [[ "${RUN_TESTS}" != "true" ]]; then
 fi
 set -u
 
-export RESULT_LOG_FILE="/files/test_result.xml"
-
-if [[ "${GITHUB_ACTIONS}" == "true" ]]; then
-    EXTRA_PYTEST_ARGS=(
-        "--verbosity=0"
-        "--strict-markers"
-        "--durations=100"
-        "--cov=airflow/"
-        "--cov-config=.coveragerc"
-        "--cov-report=xml:/files/coverage.xml"
-        "--color=yes"
-        "--maxfail=50"
-        "--pythonwarnings=ignore::DeprecationWarning"
-        "--pythonwarnings=ignore::PendingDeprecationWarning"
-        "--junitxml=${RESULT_LOG_FILE}"
-        # timeouts in seconds for individual tests
-        "--setup-timeout=20"
-        "--execution-timeout=60"
-        "--teardown-timeout=20"
-        # Only display summary for non-expected case
-        # f - failed
-        # E - error
-        # X - xpassed (passed even if expected to fail)
-        # The following cases are not displayed:
-        # s - skipped
-        # x - xfailed (expected to fail and failed)
-        # p - passed
-        # P - passed with output
-        "-rfEX"
-    )
-    if [[ "${TEST_TYPE}" != "Helm" ]]; then
-        EXTRA_PYTEST_ARGS+=(
-        "--with-db-init"
-        )
-    fi
-else
-    EXTRA_PYTEST_ARGS=(
-        "-rfEX"
+export RESULT_LOG_FILE="/files/test_result-${TEST_TYPE}.xml"
+
+EXTRA_PYTEST_ARGS=(
+    "--verbosity=0"
+    "--strict-markers"
+    "--durations=100"
+    "--cov=airflow/"
+    "--cov-config=.coveragerc"
+    "--cov-report=xml:/files/coverage.xml"
+    "--color=yes"
+    "--maxfail=50"
+    "--pythonwarnings=ignore::DeprecationWarning"
+    "--pythonwarnings=ignore::PendingDeprecationWarning"
+    "--junitxml=${RESULT_LOG_FILE}"
+    # timeouts in seconds for individual tests
+    "--setup-timeout=20"
+    "--execution-timeout=60"
+    "--teardown-timeout=20"
+    # Only display summary for non-expected case
+    # f - failed
+    # E - error
+    # X - xpassed (passed even if expected to fail)
+    # The following cases are not displayed:
+    # s - skipped
+    # x - xfailed (expected to fail and failed)
+    # p - passed
+    # P - passed with output
+    "-rfEX"
+)
+
+if [[ "${TEST_TYPE}" != "Helm" ]]; then
+    EXTRA_PYTEST_ARGS+=(
+    "--with-db-init"
     )
 fi
 
diff --git a/scripts/in_container/run_ci_tests.sh b/scripts/in_container/run_ci_tests.sh
index ca3c41d..f14ae00 100755
--- a/scripts/in_container/run_ci_tests.sh
+++ b/scripts/in_container/run_ci_tests.sh
@@ -31,66 +31,12 @@ set +x
 if [[ "${RES}" == "0" && ${CI:="false"} == "true" ]]; then
     echo "All tests successful"
     cp .coverage /files
-elif [[ "${RES}" != "0" ]]; then
-    EXTRA_ARGS=""
-    if [[ ${BACKEND} == "postgres" ]]; then
-        EXTRA_ARGS="--postgres-version ${POSTGRES_VERSION} "
-    elif [[ ${BACKEND} == "mysql" ]]; then
-        EXTRA_ARGS="--mysql-version ${MYSQL_VERSION} "
-    fi
-    echo """
-${COLOR_RED}ERROR:
-***********************************************************************************************
-*
-* ERROR! Some tests failed, unfortunately. Those might be transient errors,
-*        but usually you have to fix something.
-*        See the above log for details.
-*
-***********************************************************************************************
-*  You can easily reproduce the failed tests on your dev machine/
-*
-*   When you have the source branch checked out locally:
-*
-*     Run all tests:
-*
-*       ./breeze --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} tests
-*
-*     Enter docker shell:
-*
-*       ./breeze --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} shell
-*
-"""
-    if [[ -n "${GITHUB_REGISTRY_PULL_IMAGE_TAG=}" ]]; then
-        echo """
-*   When you do not have sources:
-*
-*     Run all tests:
-*
-*      ./breeze --github-image-id ${GITHUB_REGISTRY_PULL_IMAGE_TAG} --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} tests
-*
-*     Enter docker shell:
-*
-*      ./breeze --github-image-id ${GITHUB_REGISTRY_PULL_IMAGE_TAG} --backend ${BACKEND} ${EXTRA_ARGS}--python ${PYTHON_MAJOR_MINOR_VERSION} --db-reset --skip-mounting-local-sources --test-type ${TEST_TYPE} shell
-*
-"""
-    fi
-    echo """
-*
-*   NOTE! Once you are in the docker shell, you can run failed test with:
-*
-*            pytest [TEST_NAME]
-*
-*   You can copy the test name from the output above
-*
-***********************************************************************************************
-${COLOR_RESET}
-"""
 fi
 
 MAIN_GITHUB_REPOSITORY="apache/airflow"
 
 if [[ ${TEST_TYPE:=} == "Quarantined" ]]; then
-    if [[ ${GITHUB_REPOSITORY} == "${MAIN_GITHUB_REPOSITORY}" ]]; then
+    if [[ ${GITHUB_REPOSITORY=} == "${MAIN_GITHUB_REPOSITORY}" ]]; then
         if [[ ${RES} == "1" || ${RES} == "0" ]]; then
             echo
             echo "Pytest exited with ${RES} result. Updating Quarantine Issue!"
diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py
index 85d4711..b3403eb 100644
--- a/tests/cli/commands/test_task_command.py
+++ b/tests/cli/commands/test_task_command.py
@@ -313,6 +313,8 @@ class TestCliTasks(unittest.TestCase):
         assert state == State.SUCCESS
 
 
+# For this test memory spins out of control on Python 3.6. TODO(potiuk): FIXME")
+@pytest.mark.quarantined
 class TestLogsfromTaskRunCommand(unittest.TestCase):
     def setUp(self) -> None:
         self.dag_id = "test_logging_dag"
diff --git a/tests/jobs/test_local_task_job.py b/tests/jobs/test_local_task_job.py
index f716766..7f9445c 100644
--- a/tests/jobs/test_local_task_job.py
+++ b/tests/jobs/test_local_task_job.py
@@ -170,7 +170,7 @@ class TestLocalTaskJob(unittest.TestCase):
                 time2 = heartbeat_records[i]
                 # Assert that difference small enough
                 delta = (time2 - time1).total_seconds()
-                assert abs(delta - job.heartrate) < 0.05
+                assert abs(delta - job.heartrate) < 0.5
 
     @pytest.mark.quarantined
     def test_mark_success_no_kill(self):
diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py
index 193e932..be8e1ac 100644
--- a/tests/jobs/test_scheduler_job.py
+++ b/tests/jobs/test_scheduler_job.py
@@ -3540,13 +3540,13 @@ class TestSchedulerJob(unittest.TestCase):
         DummyOperator(task_id='task1', dag=dag)
 
         with patch.object(settings, "CHECK_SLAS", True):
-            scheduler_job = SchedulerJob(subdir=os.devnull)
+            self.scheduler_job = SchedulerJob(subdir=os.devnull)
             mock_agent = mock.MagicMock()
 
-            scheduler_job.processor_agent = mock_agent
+            self.scheduler_job.processor_agent = mock_agent
 
-            scheduler_job._send_sla_callbacks_to_processor(dag)
-            scheduler_job.processor_agent.send_sla_callback_request_to_execute.assert_not_called()
+            self.scheduler_job._send_sla_callbacks_to_processor(dag)
+            self.scheduler_job.processor_agent.send_sla_callback_request_to_execute.assert_not_called()
 
     def test_send_sla_callbacks_to_processor_sla_with_task_slas(self):
         """Test SLA Callbacks are sent to the DAG Processor when SLAs are defined on tasks"""
@@ -3558,13 +3558,13 @@ class TestSchedulerJob(unittest.TestCase):
         dag = SerializedDAG.from_json(SerializedDAG.to_json(dag))
 
         with patch.object(settings, "CHECK_SLAS", True):
-            scheduler_job = SchedulerJob(subdir=os.devnull)
+            self.scheduler_job = SchedulerJob(subdir=os.devnull)
             mock_agent = mock.MagicMock()
 
-            scheduler_job.processor_agent = mock_agent
+            self.scheduler_job.processor_agent = mock_agent
 
-            scheduler_job._send_sla_callbacks_to_processor(dag)
-            scheduler_job.processor_agent.send_sla_callback_request_to_execute.assert_called_once_with(
+            self.scheduler_job._send_sla_callbacks_to_processor(dag)
+            self.scheduler_job.processor_agent.send_sla_callback_request_to_execute.assert_called_once_with(
                 full_filepath=dag.fileloc, dag_id=dag_id
             )
 
@@ -4080,6 +4080,8 @@ def test_task_with_upstream_skip_process_task_instances():
         assert tis[dummy3.task_id].state == State.SKIPPED
 
 
+# TODO(potiuk): unquarantine me where we get rid of those pesky 195 -> 196 problem!
+@pytest.mark.quarantined
 class TestSchedulerJobQueriesCount(unittest.TestCase):
     """
     These tests are designed to detect changes in the number of queries for
diff --git a/tests/www/test_views.py b/tests/www/test_views.py
index 2e61c03..a8c5fe1 100644
--- a/tests/www/test_views.py
+++ b/tests/www/test_views.py
@@ -43,7 +43,7 @@ from werkzeug.wrappers import BaseResponse
 
 from airflow import models, settings, version
 from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG
-from airflow.configuration import conf
+from airflow.configuration import conf, initialize_config
 from airflow.executors.celery_executor import CeleryExecutor
 from airflow.jobs.base_job import BaseJob
 from airflow.models import DAG, Connection, DagRun, TaskInstance
@@ -1178,6 +1178,11 @@ class TestAirflowBaseViews(TestBase):
 
 
 class TestConfigurationView(TestBase):
+    def setUp(self):
+        super().setUp()
+        with mock.patch.dict(os.environ, {"AIRFLOW__CORE__UNIT_TEST_MODE": "False"}):
+            initialize_config()
+
     def test_configuration_do_not_expose_config(self):
         self.logout()
         self.login()