You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/03/20 15:21:57 UTC

[GitHub] [airflow] potiuk opened a new pull request #14915: Running tests in parallel

potiuk opened a new pull request #14915:
URL: https://github.com/apache/airflow/pull/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.
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598128235



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \

Review comment:
       ```suggestion
       MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --rm --entrypoint /bin/bash \
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598130298



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'grep -cE "cpu[0-9]+" </proc/stat')
+    echo
+    echo "${COLOR_BLUE}CPUS available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${CPUS_AVAILABLE_FOR_DOCKER}"
+    echo
+    export CPUS_AVAILABLE_FOR_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
-        export ENABLED_INTEGRATIONS=""
-        export RUN_INTEGRATION_TESTS=""
+        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
+    print_available_space_in_host
+    print_available_memory_in_host
+    get_available_memory_in_docker
+    get_available_cpus_in_docker
+    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)
+# ${@} - 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}" \

Review comment:
       There is no built-in parallel command on MacOS. This script is mainly for CI and having it run on MacOS is a nice-to-have (though useful). 
   
   However I have taken care of it  and tested on macos (In the scrpits I made sure to install GNU Parallel from brew). do not use MacOS with Airflow - but if you do and you could double-check if this command works for MacOS, I would really appreciate.
   
   ```
   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"
   }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803658269


   Looks like the culprit is `tests/cli/commands/test_task_command.py::TestLogsfromTaskRunCommand::test_logging_with_run_task_subprocess` -> it runs with CAN_FORK = `False` and well, it starts the whole new airlfow process to run the single task and apparently it gets memory up. 
   
   Do we really want to to test it :)  ?
   
   For now  I added additional check for memory available and - similarly to Integration test type I run it sequentially not in parallel


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598136844



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'grep -cE "cpu[0-9]+" </proc/stat')
+    echo
+    echo "${COLOR_BLUE}CPUS available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${CPUS_AVAILABLE_FOR_DOCKER}"
+    echo
+    export CPUS_AVAILABLE_FOR_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
-        export ENABLED_INTEGRATIONS=""
-        export RUN_INTEGRATION_TESTS=""
+        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
+    print_available_space_in_host
+    print_available_memory_in_host
+    get_available_memory_in_docker
+    get_available_cpus_in_docker
+    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)
+# ${@} - 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}" \

Review comment:
       Ok. I decided to take My Mac  to a spin and it was all surprisingly easy and straightforward. When  I gave my 64GB Mac - 40GB Memory + 8 CPUs it took 581 second (10 minutes) to run full suite of tests (!)
   
   Parallel was not a problem, but I had to remove old commands where I checked memory and disk on host (which is not needed). I remove those and the script works flawlessly on Mac as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598130298



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'grep -cE "cpu[0-9]+" </proc/stat')
+    echo
+    echo "${COLOR_BLUE}CPUS available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${CPUS_AVAILABLE_FOR_DOCKER}"
+    echo
+    export CPUS_AVAILABLE_FOR_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
-        export ENABLED_INTEGRATIONS=""
-        export RUN_INTEGRATION_TESTS=""
+        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
+    print_available_space_in_host
+    print_available_memory_in_host
+    get_available_memory_in_docker
+    get_available_cpus_in_docker
+    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)
+# ${@} - 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}" \

Review comment:
       There is no built-in parallel command on MacOS. This script is mainly for CI and having it run on MacOS is a nice-to-have (though useful). 
   
   However I have taken care of it  and tested parallels coming from brew on MacOS (In the scripts I made sure to install GNU Parallel from brew). do not use MacOS with Airflow - but if you do and you could double-check if this command works for MacOS, I would really appreciate.
   
   ```
   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"
   }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk merged pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #14915:
URL: https://github.com/apache/airflow/pull/14915


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-804431987


   Let's observe it for a while ...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598129070



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \

Review comment:
       Ah yeah. I added it already in #14908 but yeah. I can add it here




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk edited a comment on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803540351


   Monitoring helped a lot:
   
   * I already decreased the use of MySQL memory from ~ 450 MB in peak to ~200MB. The default MySQL configuration we used was not optimised at all for our test usage
   * However it seems that we have a serious problem in our CLI Tests that often spins out of control:
   
   https://github.com/potiuk/airflow/runs/2158887308?check_suite_focus=true#step:6:4589
   
   Usually our test `airflow` container max at around 400MB, but in cases where we have Exit 137 and failed tests, those spin out of control a lot a
   
   ```
     Overall resource statistics
     
     CONTAINER ID                                                       NAME                                         CPU %     MEM USAGE / LIMIT     MEM %     NET I/O           BLOCK I/O        PIDS
     7a4dffd2e4123a8f567704e22d13a41010c3e3f56f6759eac0c56dbb02138bea   airflow-providers_airflow_run_92452ab2adba   22.43%    643MiB / 6.791GiB     9.25%     2.21MB / 2.66MB   21.7MB / 307kB   8
     7605d1260aea0e05c3b5c927a43f85bfd0c89f9165e59218d9b7b5773caf84c1   airflow-providers_mysql_1                    0.20%     94.01MiB / 6.791GiB   1.35%     2.66MB / 2.21MB   36.9kB / 432MB   32
     88d5fd7c47ef1dc1c1269a488b2800183c262a68a3b73a6333bebff2b365e288   airflow-cli_airflow_run_17f7dab0ae27         142.86%   1.829GiB / 6.791GiB   26.93%    9.15MB / 8.32MB   15.1MB / 479kB   54
     c1141763d37ed964f6d6f76e6f842c45e0bc99d5d91d323b3bd069b456c6511d   airflow-cli_mysql_1                          0.19%     104.8MiB / 6.791GiB   1.51%     8.32MB / 9.14MB   684kB / 467MB    38
   ````
   
   See 1.829GB use of CLI airflow container
   
   and then 
   
   ```
     CONTAINER ID                                                       NAME                                         CPU %     MEM USAGE / LIMIT     MEM %     NET I/O           BLOCK I/O        PIDS
     7a4dffd2e4123a8f567704e22d13a41010c3e3f56f6759eac0c56dbb02138bea   airflow-providers_airflow_run_92452ab2adba   58.75%    633.5MiB / 6.791GiB   9.11%     2.26MB / 2.71MB   31.6MB / 307kB   8
     7605d1260aea0e05c3b5c927a43f85bfd0c89f9165e59218d9b7b5773caf84c1   airflow-providers_mysql_1                    4.69%     90MiB / 6.791GiB      1.29%     2.71MB / 2.26MB   1.26MB / 432MB   32
     88d5fd7c47ef1dc1c1269a488b2800183c262a68a3b73a6333bebff2b365e288   airflow-cli_airflow_run_17f7dab0ae27         71.13%    3.781GiB / 6.791GiB   55.68%    9.32MB / 8.43MB   16.1MB / 483kB   42
     c1141763d37ed964f6d6f76e6f842c45e0bc99d5d91d323b3bd069b456c6511d   airflow-cli_mysql_1                          0.05%     101.4MiB / 6.791GiB   1.46%     8.43MB / 9.32MB   684kB / 469MB    38
   ```
   
   Then it gets to 3.78 GB
   
   This is always around this state of CLI tests:
   
   ```
     ### The last lines for CLI process ###
     
     tests/cli/commands/test_sync_perm_command.py .                           [ 67%]
     tests/cli/commands/test_task_command.py ..........s...........
   ```
   
   so I think it will be rather easy to find the culprit.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598128480



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'grep -cE "cpu[0-9]+" </proc/stat')
+    echo
+    echo "${COLOR_BLUE}CPUS available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${CPUS_AVAILABLE_FOR_DOCKER}"
+    echo
+    export CPUS_AVAILABLE_FOR_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
-        export ENABLED_INTEGRATIONS=""
-        export RUN_INTEGRATION_TESTS=""
+        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
+    print_available_space_in_host
+    print_available_memory_in_host
+    get_available_memory_in_docker
+    get_available_cpus_in_docker
+    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)
+# ${@} - 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}" \

Review comment:
       Have you tested it on macOS? I am afraid that this command may not have all the necessary options on this system.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598129078



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \

Review comment:
       same




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803540351


   Monitoring helped a lot:
   
   * I already decreased the use of MySQL memory from ~ 450 MB in peak to 200MB. The default MySQL configuration we used was not optimised at all for our test usage
   * However it seems that we have a serious problem in our CLI Tests that often spins out of control:
   
   https://github.com/potiuk/airflow/runs/2158887308?check_suite_focus=true#step:6:4589
   
   Usually our test `airflow` container max at around 400MB, but in cases where we have Exit 137 and failed tests, those spin out of control a lot a
   
   ```
     Overall resource statistics
     
     CONTAINER ID                                                       NAME                                         CPU %     MEM USAGE / LIMIT     MEM %     NET I/O           BLOCK I/O        PIDS
     7a4dffd2e4123a8f567704e22d13a41010c3e3f56f6759eac0c56dbb02138bea   airflow-providers_airflow_run_92452ab2adba   22.43%    643MiB / 6.791GiB     9.25%     2.21MB / 2.66MB   21.7MB / 307kB   8
     7605d1260aea0e05c3b5c927a43f85bfd0c89f9165e59218d9b7b5773caf84c1   airflow-providers_mysql_1                    0.20%     94.01MiB / 6.791GiB   1.35%     2.66MB / 2.21MB   36.9kB / 432MB   32
     88d5fd7c47ef1dc1c1269a488b2800183c262a68a3b73a6333bebff2b365e288   airflow-cli_airflow_run_17f7dab0ae27         142.86%   1.829GiB / 6.791GiB   26.93%    9.15MB / 8.32MB   15.1MB / 479kB   54
     c1141763d37ed964f6d6f76e6f842c45e0bc99d5d91d323b3bd069b456c6511d   airflow-cli_mysql_1                          0.19%     104.8MiB / 6.791GiB   1.51%     8.32MB / 9.14MB   684kB / 467MB    38
   ````
   
   See 1.829GB use of CLI airflow container
   
   and then 
   
   ```
     CONTAINER ID                                                       NAME                                         CPU %     MEM USAGE / LIMIT     MEM %     NET I/O           BLOCK I/O        PIDS
     7a4dffd2e4123a8f567704e22d13a41010c3e3f56f6759eac0c56dbb02138bea   airflow-providers_airflow_run_92452ab2adba   58.75%    633.5MiB / 6.791GiB   9.11%     2.26MB / 2.71MB   31.6MB / 307kB   8
     7605d1260aea0e05c3b5c927a43f85bfd0c89f9165e59218d9b7b5773caf84c1   airflow-providers_mysql_1                    4.69%     90MiB / 6.791GiB      1.29%     2.71MB / 2.26MB   1.26MB / 432MB   32
     88d5fd7c47ef1dc1c1269a488b2800183c262a68a3b73a6333bebff2b365e288   airflow-cli_airflow_run_17f7dab0ae27         71.13%    3.781GiB / 6.791GiB   55.68%    9.32MB / 8.43MB   16.1MB / 483kB   42
     c1141763d37ed964f6d6f76e6f842c45e0bc99d5d91d323b3bd069b456c6511d   airflow-cli_mysql_1                          0.05%     101.4MiB / 6.791GiB   1.46%     8.43MB / 9.32MB   684kB / 469MB    38
   ```
   
   Then it gets to 3.78 GB
   
   This is always around this state of CLI tests:
   
   ```
     ### The last lines for CLI process ###
     
     tests/cli/commands/test_sync_perm_command.py .                           [ 67%]
     tests/cli/commands/test_task_command.py ..........s...........
   ```
   
   so I think it will be rather easy to find the culprit.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598128032



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -15,77 +15,34 @@
 # 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"
+RUN_TESTS="true"

Review comment:
       ```suggestion
   RUN_TESTS="true"
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803450506


   One more thing to check. In non-self-hosted runs I have consistent Exit 137 with MySQL and Python 3.6 when they run 2 test streams in parallel (and I think it's only this combination that has problem in both MySQL versions) . I want to take a closer look at the memory usage pattern for those cases. Maybe it's something I will be able to fix as well. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803377955


   With LAST rebase I simply rebased a fixup rather than main change :facepalm: . This is the actual change with parallel tests @ashb @kaxil @mik-laj 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] mik-laj commented on a change in pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#discussion_r598128246



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -109,66 +66,265 @@ 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
+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
+}
 
-build_images::rebuild_ci_image_if_needed_with_group
+function system_prune_docker() {
+    echo
+    echo "${COLOR_BLUE}System-prune docker${COLOR_RESET}"
+    echo
+    docker system prune --force --volumes
+    echo
+}
 
-prepare_tests_to_run
+function print_available_space_in_host() {
+    echo "${COLOR_BLUE}Print available space${COLOR_RESET}"
+    echo
+    df --human
+}
 
+function print_available_memory_in_host() {
+    echo
+    echo "${COLOR_BLUE}Print available memory${COLOR_RESET}"
+    echo
+    free --human
+}
 
-for TEST_TYPE in ${TEST_TYPES}
-do
-    start_end::group_start "Running tests ${TEST_TYPE}"
+function get_available_memory_in_docker() {
+    MEMORY_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \
+        "${AIRFLOW_CI_IMAGE}" -c \
+        'echo $(($(getconf _PHYS_PAGES) * $(getconf PAGE_SIZE) / (1024 * 1024)))')
+    echo
+    echo "${COLOR_BLUE}Memory available for Docker${COLOR_RESET}"
+    echo
+    echo "    ${MEMORY_AVAILABLE_FOR_DOCKER}"
+    echo
+    export MEMORY_AVAILABLE_FOR_DOCKER
+}
 
-    INTEGRATIONS=()
-    export INTEGRATIONS
 
-    if [[ ${TEST_TYPE:=} == "Integration" ]]; then
-        export ENABLED_INTEGRATIONS="${AVAILABLE_INTEGRATIONS}"
-        export RUN_INTEGRATION_TESTS="${AVAILABLE_INTEGRATIONS}"
+function get_available_cpus_in_docker() {
+    CPUS_AVAILABLE_FOR_DOCKER=$(docker run --entrypoint /bin/bash \

Review comment:
       ```suggestion
       CPUS_AVAILABLE_FOR_DOCKER=$(docker run --rm --entrypoint /bin/bash \
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803395178


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest master at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-803378903


   :pray: approvals please :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] potiuk commented on pull request #14915: Running tests in parallel

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #14915:
URL: https://github.com/apache/airflow/pull/14915#issuecomment-804431895


   Aaand merged . I quarantined the "Log tests" for now and Added #14937  to fix it in the future - but it seems it's much more stable now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org