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/04/05 21:48:14 UTC

[airflow] 15/16: Merges quarantined tests into single job (#15153)

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

potiuk pushed a commit to branch v2-0-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit e87cd1f259f591aeaa0521ef79b9d6249739d411
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Mon Apr 5 19:58:10 2021 +0200

    Merges quarantined tests into single job (#15153)
    
    (cherry picked from commit 1087226f756b3ff9ea48398e53f9074b0ed4c1cc)
---
 .github/workflows/ci.yml                           |   9 +-
 scripts/ci/libraries/_all_libs.sh                  |   2 +
 scripts/ci/libraries/_initialization.sh            |   3 +-
 scripts/ci/libraries/_parallel.sh                  |  35 +++++-
 scripts/ci/libraries/_testing.sh                   | 116 +++++++++++++++++
 scripts/ci/testing/ci_run_airflow_testing.sh       | 140 +++------------------
 scripts/ci/testing/ci_run_quarantined_tests.sh     |  87 +++++++++++++
 .../ci_run_single_airflow_test_in_docker.sh        |   6 +-
 8 files changed, 259 insertions(+), 139 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index ddc985b..dc98f5c 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -817,15 +817,8 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
     runs-on: ${{ fromJson(needs.build-info.outputs.runsOn) }}
     continue-on-error: true
     needs: [build-info, ci-images]
-    strategy:
-      matrix:
-        include:
-          - backend: mysql
-          - 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}}
@@ -860,7 +853,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - name: "Prepare CI image ${{env.PYTHON_MAJOR_MINOR_VERSION}}:${{ env.GITHUB_REGISTRY_PULL_IMAGE_TAG }}"
         run: ./scripts/ci/images/ci_prepare_ci_image_on_ci.sh
       - name: "Tests: Quarantined"
-        run: ./scripts/ci/testing/ci_run_airflow_testing.sh
+        run: ./scripts/ci/testing/ci_run_quarantined_tests.sh
       - name: "Upload Quarantine test results"
         uses: actions/upload-artifact@v2
         if: always()
diff --git a/scripts/ci/libraries/_all_libs.sh b/scripts/ci/libraries/_all_libs.sh
index 09a147d..04e25e8 100755
--- a/scripts/ci/libraries/_all_libs.sh
+++ b/scripts/ci/libraries/_all_libs.sh
@@ -60,6 +60,8 @@ readonly SCRIPTS_CI_DIR
 . "${LIBRARIES_DIR}"/_spinner.sh
 # shellcheck source=scripts/ci/libraries/_start_end.sh
 . "${LIBRARIES_DIR}"/_start_end.sh
+# shellcheck source=scripts/ci/libraries/_testing.sh
+. "${LIBRARIES_DIR}"/_testing.sh
 # shellcheck source=scripts/ci/libraries/_verbosity.sh
 . "${LIBRARIES_DIR}"/_verbosity.sh
 # shellcheck source=scripts/ci/libraries/_verify_image.sh
diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index f924962..f82cb55 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -710,7 +710,7 @@ Initialization variables:
 
 Test variables:
 
-    TEST_TYPE: '${TEST_TYPE}'
+    TEST_TYPE: '${TEST_TYPE=}'
 
 EOF
     if [[ "${CI}" == "true" ]]; then
@@ -776,7 +776,6 @@ function initialization::make_constants_read_only() {
     readonly HELM_VERSION
     readonly KUBECTL_VERSION
 
-    readonly BACKEND
     readonly POSTGRES_VERSION
     readonly MYSQL_VERSION
 
diff --git a/scripts/ci/libraries/_parallel.sh b/scripts/ci/libraries/_parallel.sh
index dfe1c4d..739bae1 100644
--- a/scripts/ci/libraries/_parallel.sh
+++ b/scripts/ci/libraries/_parallel.sh
@@ -73,7 +73,7 @@ function parallel::monitor_loop() {
         do
             parallel_process=$(basename "${directory}")
 
-            echo "${COLOR_BLUE}### The last lines for ${parallel_process} process ###${COLOR_RESET}"
+            echo "${COLOR_BLUE}### The last lines for ${parallel_process} process: ${directory}/stdout ###${COLOR_RESET}"
             echo
             tail -2 "${directory}/stdout" || true
             echo
@@ -160,3 +160,36 @@ function parallel::print_job_summary_and_return_status_code() {
     done
     return "${return_code}"
 }
+
+function parallel::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 parallel::system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker_v system prune --force --volumes
+    echo
+}
+
+# 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 parallel::cleanup_runner() {
+    start_end::group_start "Cleanup runner"
+    parallel::kill_all_running_docker_containers
+    parallel::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
+    parallel::kill_stale_semaphore_locks
+    start_end::group_end
+}
diff --git a/scripts/ci/libraries/_testing.sh b/scripts/ci/libraries/_testing.sh
new file mode 100644
index 0000000..28d1fc6
--- /dev/null
+++ b/scripts/ci/libraries/_testing.sh
@@ -0,0 +1,116 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+export MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN=33000
+
+function testing::skip_tests_if_requested(){
+    if [[ -f ${BUILD_CACHE_DIR}/.skip_tests ]]; then
+        echo
+        echo "Skipping running tests !!!!!"
+        echo
+        exit
+    fi
+}
+
+function testing::get_docker_compose_local() {
+    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")
+    fi
+    if [[ ${MOUNT_ALL_LOCAL_SOURCES} == "true" ]]; then
+        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/local-all-sources.yml")
+    fi
+
+    if [[ ${GITHUB_ACTIONS} == "true" ]]; then
+        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/ga.yml")
+    fi
+
+    if [[ ${FORWARD_CREDENTIALS} == "true" ]]; then
+        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/forward-credentials.yml")
+    fi
+
+    if [[ -n ${INSTALL_AIRFLOW_VERSION=} || -n ${INSTALL_AIRFLOW_REFERENCE} ]]; then
+        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/remove-sources.yml")
+    fi
+    readonly DOCKER_COMPOSE_LOCAL
+}
+
+function testing::get_maximum_parallel_test_jobs() {
+    docker_engine_resources::get_available_cpus_in_docker
+    if [[ ${RUNS_ON} != *"self-hosted"* ]]; then
+        echo
+        echo "${COLOR_YELLOW}This is a Github Public runner - for now we are forcing max parallel Quarantined tests jobs to 1 for those${COLOR_RESET}"
+        echo
+        export MAX_PARALLEL_QUARANTINED_TEST_JOBS="1"
+    else
+        if [[ ${MAX_PARALLEL_QUARANTINED_TEST_JOBS=} != "" ]]; then
+            echo
+            echo "${COLOR_YELLOW}Maximum parallel Quarantined test jobs forced via MAX_PARALLEL_QUARANTINED_TEST_JOBS = ${MAX_PARALLEL_QUARANTINED_TEST_JOBS}${COLOR_RESET}"
+            echo
+        else
+            MAX_PARALLEL_QUARANTINED_TEST_JOBS=${CPUS_AVAILABLE_FOR_DOCKER}
+            echo
+            echo "${COLOR_YELLOW}Maximum parallel Quarantined test jobs set to number of CPUs available for Docker = ${MAX_PARALLEL_QUARANTINED_TEST_JOBS}${COLOR_RESET}"
+            echo
+        fi
+
+    fi
+
+    if [[ ${MAX_PARALLEL_TEST_JOBS=} != "" ]]; then
+        echo
+        echo "${COLOR_YELLOW}Maximum parallel test jobs forced via 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
+}
+
+function testing::get_test_types_to_run() {
+    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. Adding all: ${TEST_TYPES}"
+        echo
+    fi
+
+    if [[ -z "${FORCE_TEST_TYPE=}" ]]; then
+        # Add Postgres/MySQL special test types in case we are running several test types
+        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" && ${TEST_TYPES} != "Quarantined" ]]; then
+            TEST_TYPES="${TEST_TYPES} MySQL"
+            echo
+            echo "Added MySQL. Tests to run: ${TEST_TYPES}"
+            echo
+        fi
+    fi
+    readonly TEST_TYPES
+}
diff --git a/scripts/ci/testing/ci_run_airflow_testing.sh b/scripts/ci/testing/ci_run_airflow_testing.sh
index af147ad..fa8c044 100755
--- a/scripts/ci/testing/ci_run_airflow_testing.sh
+++ b/scripts/ci/testing/ci_run_airflow_testing.sh
@@ -23,128 +23,13 @@ export RUN_TESTS
 SKIPPED_FAILED_JOB="Quarantined"
 export SKIPPED_FAILED_JOB
 
-# shellcheck source=scripts/ci/libraries/_script_init.sh
-. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
-
-if [[ -f ${BUILD_CACHE_DIR}/.skip_tests ]]; then
-    echo
-    echo "Skipping running tests !!!!!"
-    echo
-    exit
-fi
-
-# 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"
+export SEMAPHORE_NAME
 
-function prepare_tests_to_run() {
-    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")
-    fi
-    if [[ ${MOUNT_ALL_LOCAL_SOURCES} == "true" ]]; then
-        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/local-all-sources.yml")
-    fi
-
-    if [[ ${GITHUB_ACTIONS} == "true" ]]; then
-        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/ga.yml")
-    fi
-
-    if [[ ${FORWARD_CREDENTIALS} == "true" ]]; then
-        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/forward-credentials.yml")
-    fi
-
-    if [[ -n ${INSTALL_AIRFLOW_VERSION=} || -n ${INSTALL_AIRFLOW_REFERENCE} ]]; then
-        DOCKER_COMPOSE_LOCAL+=("-f" "${SCRIPTS_CI_DIR}/docker-compose/remove-sources.yml")
-    fi
-    readonly DOCKER_COMPOSE_LOCAL
-
-    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. Adding all: ${TEST_TYPES}"
-        echo
-    fi
-
-    if [[ -z "${FORCE_TEST_TYPE=}" ]]; then
-        # Add Postgres/MySQL special test types in case we are running several test types
-        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" && ${TEST_TYPES} != "Quarantined" ]]; then
-            TEST_TYPES="${TEST_TYPES} MySQL"
-            echo
-            echo "Added MySQL. Tests to run: ${TEST_TYPES}"
-            echo
-        fi
-    fi
-    readonly TEST_TYPES
-}
-
-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
-}
+# shellcheck source=scripts/ci/libraries/_script_init.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
 
-function system_prune_docker() {
-    echo
-    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
-    echo
-    docker_v system prune --force --volumes
-    echo
-}
 
-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
-}
 
 # Starts test types in parallel
 # test_types_to_run - list of test types (it's not an array, it is space-separate list)
@@ -171,9 +56,6 @@ function run_test_types_in_parallel() {
     start_end::group_end
 }
 
-
-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.
 #
@@ -188,7 +70,7 @@ export MEMORY_REQUIRED_FOR_INTEGRATION_TEST_PARALLEL_RUN=33000
 #   * MEMORY_AVAILABLE_FOR_DOCKER - memory that is available in docker (set by cleanup_runners)
 #
 function run_all_test_types_in_parallel() {
-    cleanup_runner
+    parallel::cleanup_runner
 
     start_end::group_start "Determine how to run the tests"
     echo
@@ -196,6 +78,7 @@ function run_all_test_types_in_parallel() {
     echo
 
     local run_integration_tests_separately="false"
+    # shellcheck disable=SC2153
     local test_types_to_run=${TEST_TYPES}
 
     if [[ ${test_types_to_run} == *"Integration"* ]]; then
@@ -222,7 +105,7 @@ function run_all_test_types_in_parallel() {
 
     run_test_types_in_parallel "${@}"
     if [[ ${run_integration_tests_separately} == "true" ]]; then
-        cleanup_runner
+        parallel::cleanup_runner
         test_types_to_run="Integration"
         run_test_types_in_parallel "${@}"
     fi
@@ -231,12 +114,19 @@ function run_all_test_types_in_parallel() {
     parallel::print_job_summary_and_return_status_code
 }
 
+
+testing::skip_tests_if_requested
+
 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
 
+testing::get_maximum_parallel_test_jobs
+
+testing::get_test_types_to_run
+
+testing::get_docker_compose_local
+
 run_all_test_types_in_parallel "${@}"
diff --git a/scripts/ci/testing/ci_run_quarantined_tests.sh b/scripts/ci/testing/ci_run_quarantined_tests.sh
new file mode 100755
index 0000000..0c1108e
--- /dev/null
+++ b/scripts/ci/testing/ci_run_quarantined_tests.sh
@@ -0,0 +1,87 @@
+#!/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.
+
+# Enable automated tests execution
+RUN_TESTS="true"
+export RUN_TESTS
+
+SKIPPED_FAILED_JOB="Quarantined"
+export SKIPPED_FAILED_JOB
+
+SEMAPHORE_NAME="tests"
+export SEMAPHORE_NAME
+
+# shellcheck source=scripts/ci/libraries/_script_init.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+BACKEND_TEST_TYPES=(mysql postgres sqlite)
+
+# 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_quarantined_backend_tests_in_parallel() {
+    start_end::group_start "Determining how to run the tests"
+    echo
+    echo "${COLOR_YELLOW}Running maximum ${MAX_PARALLEL_QUARANTINED_TEST_JOBS} test types in parallel${COLOR_RESET}"
+    echo
+    start_end::group_end
+    start_end::group_start "Monitoring Quarantined tests : ${BACKEND_TEST_TYPES[*]}"
+    parallel::initialize_monitoring
+    parallel::monitor_progress
+    mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}"
+    TEST_TYPE="Quarantined"
+    export TEST_TYPE
+    for BACKEND in "${BACKEND_TEST_TYPES[@]}"
+    do
+        export BACKEND
+        mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${BACKEND}"
+        mkdir -p "${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${BACKEND}"
+        export JOB_LOG="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${BACKEND}/stdout"
+        export PARALLEL_JOB_STATUS="${PARALLEL_MONITORED_DIR}/${SEMAPHORE_NAME}/${BACKEND}/status"
+        # Each test job will get SIGTERM followed by SIGTERM 200ms later and SIGKILL 200ms later after 25 mins
+        # shellcheck disable=SC2086
+        parallel --ungroup --bg --semaphore --semaphorename "${SEMAPHORE_NAME}" \
+            --jobs "${MAX_PARALLEL_QUARANTINED_TEST_JOBS}" --timeout 1500 \
+            "$( 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
+}
+
+testing::skip_tests_if_requested
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed_with_group
+
+parallel::make_sure_gnu_parallel_is_installed
+
+testing::get_maximum_parallel_test_jobs
+
+testing::get_docker_compose_local
+
+run_quarantined_backend_tests_in_parallel "${@}"
+
+set +e
+
+parallel::print_job_summary_and_return_status_code
+
+echo "Those are quarantined tests so failure of those does not fail the whole build!"
+echo "Please look above for the output of failed tests to fix them!"
+echo
diff --git a/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh b/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
index 76b710e..0bf415f 100755
--- a/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
+++ b/scripts/ci/testing/ci_run_single_airflow_test_in_docker.sh
@@ -90,7 +90,7 @@ function run_airflow_testing_in_docker() {
         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}" \
+            --project-name "airflow-${TEST_TYPE}-${BACKEND}" \
             down --remove-orphans \
             --volumes --timeout 10
         docker-compose --log-level INFO \
@@ -98,11 +98,11 @@ function run_airflow_testing_in_docker() {
           -f "${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml" \
           "${INTEGRATIONS[@]}" \
           "${DOCKER_COMPOSE_LOCAL[@]}" \
-          --project-name "airflow-${TEST_TYPE}" \
+          --project-name "airflow-${TEST_TYPE}-${BACKEND}" \
              run airflow "${@}"
         exit_code=$?
         docker-compose --log-level INFO -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-            --project-name "airflow-${TEST_TYPE}" \
+            --project-name "airflow-${TEST_TYPE}-${BACKEND}" \
             down --remove-orphans \
             --volumes --timeout 10
         if [[ ${exit_code} == "254" && ${try_num} != "5" ]]; then