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/11/22 02:43:13 UTC

[GitHub] [airflow] mik-laj opened a new pull request #19737: Tests for Docker images are in Python

mik-laj opened a new pull request #19737:
URL: https://github.com/apache/airflow/pull/19737


   The current tests for the Docker image are written in bash, which unfortunately makes them limited and difficult to maintain.  They don't always find all the problems they could find if they were written in Python. For example, when we added MSSQL support to the Docker image, we don't update the list of preinstalled packages:
   https://github.com/apache/airflow/blob/main/scripts/ci/libraries/_initialization.sh#L419-L442
   When a test is written in Python, it is easier to write a valid test because Python supports comparing complex structures like `set == set`.
   
   Thanks to `Pytest-xdist`, the tests can be executed in many threads, which significantly speeds them up. Instead of 3 m 30 sec, it now takes 1 minute on my MacBook Pro. And we don't need any specialist GNU parallels for that. 
   
   In the future, I would like to rewrite [the tests for image building examples](https://github.com/apache/airflow/blob/1b89e682a1a04adf2cf5ebb37a453e245a6e7191/scripts/ci/images/ci_test_examples_of_prod_image_building.sh#L1) in a similar way, so that they can be run just as easily. 
   In addition, I also hope to contribute [tests for docker-compose files](https://github.com/mik-laj/airflow-docker-compose-examples/blob/main/tests/test_compose_files.py), which are currently being developed in an independent repository, and despite this, they have allowed me to detect various problems many times.
   
   But for now, I would like to limit the scope of the change
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/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/main/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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on pull request #19737: Tests for Docker images are in Python

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #19737:
URL: https://github.com/apache/airflow/pull/19737#issuecomment-975467610


   @uranusjr It uses `pytest` to run the tests, the `unittest` module is not imported, but the methods from the `pytest` module is used instead. What exactly do you mean?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on pull request #19737: Tests for Docker images in Python

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


   I believe the failures are fixed in main, please rebase.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -0,0 +1,115 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import subprocess
+from pathlib import Path
+from typing import List
+
+docker_image = os.environ.get('DOCKER_IMAGE')

Review comment:
       Ah right, yeah




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       And BTW. 
   
   I really like the idea of small, dedicated venvs for different tasks - separate venv for "dev", separate for "docker_tests", separate for "chart/tests" - especially if we add some scripting that will create them when needed automatically (for casual users). So we should reapply this to other cases in the future.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: .pre-commit-config.yaml
##########
@@ -238,6 +238,7 @@ repos:
           ^scripts/.*\.py$|
           ^dev|
           ^provider_packages|
+          ^docker_tests|

Review comment:
       Why we do not want to follow pydocstyle 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       Makes sense. Also We should change python to python3 I think @mik-laj - both in shebang and here.

##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       Makes sense. Also We should change python to `python3` I think @mik-laj - both in shebang and 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -0,0 +1,115 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import subprocess
+from pathlib import Path
+from typing import List
+
+docker_image = os.environ.get('DOCKER_IMAGE')
+SOURCE_ROOT = Path(__file__).resolve().parents[1]
+
+if not docker_image:
+    raise Exception("The DOCKER_IMAGE environment variable is required")
+
+
+def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
+    print(f"$ {' '.join(c for c in cmd)}")

Review comment:
       I like this as this will give an easy way to copy&paste the command to run it manually

##########
File path: docker_tests/ci_image.py
##########
@@ -0,0 +1,42 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import subprocess
+import unittest
+
+from docker_tests.docker_tests_utils import (
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash,
+    run_command,
+)
+
+
+class TestFiles(unittest.TestCase):

Review comment:
       Thos could be ptyest tests (functions) indeed (followiung @uranusjr comments). No need to have classes.

##########
File path: docker_tests/prod_image.py
##########
@@ -0,0 +1,202 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import json
+import subprocess
+import tempfile
+from pathlib import Path
+
+import pytest
+
+from docker_tests.docker_tests_utils import (
+    SOURCE_ROOT,
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash,
+    run_command,
+    run_python,
+)
+
+
+class TestCommands:
+    def test_without_command(self):
+        """Checking the image without a command. It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_command(self):
+        """Checking 'airflow' command  It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow"])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_version(self):
+        """Checking 'airflow version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow", "version"]
+        )
+        assert "2." in output
+
+    def test_python_version(self):
+        """Checking 'python --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "python", "--version"]
+        )
+        assert "Python 3." in output
+
+    def test_bash_version(self):
+        """Checking 'bash --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "bash", "--version"]
+        )
+        assert "GNU bash," in output
+
+
+class TestPythonPackages:
+    def test_required_providers_are_installed(self):
+        lines = (
+            d.strip()
+            for d in (SOURCE_ROOT / "scripts" / "ci" / "installed_providers.txt").read_text().splitlines()
+        )
+        lines = (d for d in lines)
+        packages_to_install = {f"apache-airflow-providers-{d.replace('.', '-')}" for d in lines}
+        assert len(packages_to_install) != 0
+
+        output = run_bash("airflow providers list --output json", stderr=subprocess.DEVNULL)
+        providers = json.loads(output)
+        packages_installed = {d['package_name'] for d in providers}
+        assert len(packages_installed) != 0
+
+        assert packages_to_install == packages_installed

Review comment:
       We should add message on what to do in case they are different I think (add missing packages to installed_providers.txt  or smth)

##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -0,0 +1,115 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import subprocess
+from pathlib import Path
+from typing import List
+
+docker_image = os.environ.get('DOCKER_IMAGE')

Review comment:
       There is no need to use env variable, I think for python tool like that it would be much bettter to use `click` and pass the image as command line option.

##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+# shellcheck source=scripts/ci/libraries/_script_init.sh
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+interactive="false"
+initialize_only="false"
+declare -a tests_to_run
+declare -a pytest_args
+
+tests_to_run=()
+
+function parse_tests_to_run() {
+    if [[ $# != 0 ]]; then
+        if [[ $1 == "--help" || $1 == "-h" ]]; then
+            echo
+            echo "Running Docker tests"
+            echo
+            echo "    $0 TEST [TEST ...]      - runs tests (from docker_tests folder)"
+            echo "    $0 [-i|--interactive]   - Activates virtual environment ready to run tests and drops you in"
+            echo "    $0 [--initialize]       - Initialize virtual environment and exit"
+            echo "    $0 [--help]             - Prints this help message"
+            echo
+            exit
+        elif [[ $1 == "--interactive" || $1 == "-i" ]]; then
+            echo
+            echo "Entering interactive environment for docker testing"
+            echo
+            interactive="true"
+        elif [[ $1 == "--initialize" ]]; then
+            echo
+            echo "Initializing environment for docker testing"
+            echo
+            initialize_only="true"
+        else
+            tests_to_run=("${@}")
+        fi
+        pytest_args=(
+            "--pythonwarnings=ignore::DeprecationWarning"
+            "--pythonwarnings=ignore::PendingDeprecationWarning"
+            "-n" "auto"
+        )
+    else
+        echo "You must select the tests to run."
+        exit 1
+    fi
+}
+
+function create_virtualenv() {
+    HOST_PYTHON_VERSION=$(python3 -c 'import sys; print(f"{sys.version_info[0]}.{sys.version_info[1]}")')
+    readonly HOST_PYTHON_VERSION
+
+    local virtualenv_path="${BUILD_CACHE_DIR}/.docker_venv/host_python_${HOST_PYTHON_VERSION}"
+
+    mkdir -pv "${BUILD_CACHE_DIR}/.docker_venv/"
+    if [[ ! -d ${virtualenv_path} ]]; then
+        echo
+        echo "Creating virtualenv at ${virtualenv_path}"
+        echo
+        python3 -m venv "${virtualenv_path}"
+    fi
+
+    . "${virtualenv_path}/bin/activate"
+
+    pip install --upgrade "pip==${AIRFLOW_PIP_VERSION}" "wheel==${WHEEL_VERSION}"
+
+    local constraints=(
+        --constraint
+        "https://raw.githubusercontent.com/${CONSTRAINTS_GITHUB_REPOSITORY}/${DEFAULT_CONSTRAINTS_BRANCH}/constraints-${HOST_PYTHON_VERSION}.txt"
+    )
+    if [[ -n ${GITHUB_REGISTRY_PULL_IMAGE_TAG=} ]]; then
+        # Disable constraints when building in CI with specific version of sources
+        # In case there will be conflicting constraints
+        constraints=()
+    fi
+
+    pip install pytest pytest-xdist "${constraints[@]}"
+}
+
+function run_tests() {
+    pytest "${pytest_args[@]}" "${tests_to_run[@]}"
+}
+
+cd "${AIRFLOW_SOURCES}" || exit 1
+
+set +u
+parse_tests_to_run "${@}"
+set -u
+
+create_virtualenv
+
+if [[ ${interactive} == "true" ]]; then
+    echo
+    echo "Activating the virtual environment for docker testing"
+    echo
+    echo "You can run testing via 'pytest docker tests/....'"

Review comment:
       ```suggestion
       echo "You can run testing via 'pytest docker_tests/....'"
   ```

##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       I think if we are getting ride of bash we shoudl get rid of it entirely.
   I think this could be very easily (and better) written as pure-stdib code that will prepare and activate the env with dependencies stored as `docker_test/requirements.txt' . In our case it would be:
   `pytest, pytest-xdist, click` (the last one if we convert the script above to use click). 
   
   This could be an 'entrypoint' to run tests - both from CI (installing such a small set of requirements can be done in CI and even cached easily) as well for a "fresh" user who does not have the venv installed - adding pytest-xdist automatically to such venv and running tests with it is good reason to have such auto-installation.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Just to add a little more context on that one as well:
   
   And to add on that - we do not know what python version the user has. The idea of this script (similarly as previous BASH scripts) is that it should run pretty much everywhere without any "extra" assumptions. Just pytho3 should be enough to run the script - I think we can - for now assume safely have `python3.6+` as the "big assumption".  
   
   I would also love this to make sure it also works on Windows without WSL2 - this was the thing that was missing with `bash` approach - but ideally the workflow should be exactly the same for pure-windows developers too.
   
   So we have to be extra careful this one also works for Windows (this is something that we will likely add during the Internship - and some of our interns are using Windows primarily - so this is something they will be able to check and likely we can even add tests for it in GitHub on Windows runners (but that's a bit later stage).




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       I don't see any reason why we shouldn't support the old versions anymore. This is not a big change.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])

Review comment:
       Context why running command is better:
   
   One thing that is actually useful here is the abiliity for the user to be able to repeat what happens manually - not necessarily via the python script (and we are going to use similar approach in the future `breeze` python version for other stuff.
   
   The script is there to:
   
   a) run on CI  (replacing bash script)
   b) give an easy way to the `casual` users or users who will be using codespaces in the future to create the venv and run tests seamlessly
   
   For most of the "power" users, they will have their own venv with `pytest` and `pytes-xdist` installed and they will run the tests via IDE/commandline whatever.
   
   So the `run verbose` command is good idea in this context because it will shows the exact command that is being run. For example when tnis script fails in the CI, anyone will be able to see the exact `pytest` command that was run in the CI that caused the problem.
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]

Review comment:
       Agree. either of the two is better.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       I propose a solution later.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])

Review comment:
       Since this script ends immediately after invoking pytest, maybe it can instead run pytest in-process?
   
   https://docs.pytest.org/en/latest/how-to/usage.html#calling-pytest-from-python-code

##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]

Review comment:
       This `3` is (too) magical. What does it do?

##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       There’s `shlex.join()` on 3.8+; do we need to run this on older versions?

##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       I feel it’s better to call `python` explicitly here instead of relying on the shebang.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on pull request #19737: Tests for Docker images in Python

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


   Let’s add a line in `CODEOWNERS` for `docker_tests`?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]

Review comment:
       I think my main problem is actually that is’s entirely unclear why it’s three levels up. Also either of these might be better
   
   ```python
   Path(__file__).resolve().parent.parent.parent
   
   # or
   
   Path(__file__, "..", "..", "..").resolve()
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on pull request #19737: Tests for Docker images are in Python

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


   Since we’re writing this from scratch, would it be better to build the entire stack against Pytest instead of unittest?


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on pull request #19737: Tests for Docker images in Python

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


   I think we can safely merge this one - all the build/test images have succeeded https://github.com/apache/airflow/runs/4306213737?check_suite_focus=true 
   and:
   https://github.com/apache/airflow/runs/4306213737?check_suite_focus=true#step:5:284


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       Yeah but duplicating that will make it more difficult to remove in the future. I'd prefer to only add python code and remove bash code since we are going to get rid of bash. 
   
   Seems like in this case we are trading one bash for another. Not a good idea.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       Done. We use python3 to start the script and python3 in shebang. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Basically: with all those BashToPython changes we should assume that they are used by someone who has no dev env setup, nothing installed, just python (3.6+) installed on their OS (Modern Linux, MacOS, Windows). There should be no more prerequisites. Later we might add docker/docker-compose to the list of those prerequisites, but that should really be it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk merged pull request #19737: Tests for Docker images in Python

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


   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/prod_image.py
##########
@@ -0,0 +1,202 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import json
+import subprocess
+import tempfile
+from pathlib import Path
+
+import pytest
+
+from docker_tests.docker_tests_utils import (
+    SOURCE_ROOT,
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash,
+    run_command,
+    run_python,
+)
+
+
+class TestCommands:
+    def test_without_command(self):
+        """Checking the image without a command. It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_command(self):
+        """Checking 'airflow' command  It should return non-zero exit code."""
+        with pytest.raises(subprocess.CalledProcessError) as ctx:
+            run_command(["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow"])
+        assert 2 == ctx.value.returncode
+
+    def test_airflow_version(self):
+        """Checking 'airflow version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "airflow", "version"]
+        )
+        assert "2." in output
+
+    def test_python_version(self):
+        """Checking 'python --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "python", "--version"]
+        )
+        assert "Python 3." in output
+
+    def test_bash_version(self):
+        """Checking 'bash --version' command  It should return zero exit code."""
+        output = run_command(
+            ["docker", "run", "--rm", "-e", "COLUMNS=180", docker_image, "bash", "--version"]
+        )
+        assert "GNU bash," in output
+
+
+class TestPythonPackages:
+    def test_required_providers_are_installed(self):
+        lines = (
+            d.strip()
+            for d in (SOURCE_ROOT / "scripts" / "ci" / "installed_providers.txt").read_text().splitlines()
+        )
+        lines = (d for d in lines)
+        packages_to_install = {f"apache-airflow-providers-{d.replace('.', '-')}" for d in lines}
+        assert len(packages_to_install) != 0
+
+        output = run_bash("airflow providers list --output json", stderr=subprocess.DEVNULL)
+        providers = json.loads(output)
+        packages_installed = {d['package_name'] for d in providers}
+        assert len(packages_installed) != 0
+
+        assert packages_to_install == packages_installed

Review comment:
       Just to explain "why" - we have quite a number of tests that are showing "diff" but without extra comment what to do with the diff, especially new users find it confusing what to do with the information. I think in all such cases where we just expect listst to match we should explain with a short comment how to reconcile the lists.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       And you have a chance to establish a new "pattern" here.

##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       And you have a chance to establish a new "pattern" here :D




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       Why the feeling ?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -0,0 +1,115 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import os
+import subprocess
+from pathlib import Path
+from typing import List
+
+docker_image = os.environ.get('DOCKER_IMAGE')

Review comment:
       This is a variable is used by the test so we can't use `click`. I mimic the other tests that pass parameters for tests using environment variables.  For example, https://github.com/apache/airflow/blob/8505d2f0a4524313e3eff7a4f16b9a9439c7a79f/tests/providers/apache/hive/sensors/test_metastore_partition.py#L28  and any system tests for Google operator.
   
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       This file is based on [ci_run_kubernetes_tests.sh](https://github.com/apache/airflow/blob/dddfedfdb1463f9e263dcc4570ff0fcee56b7c51/scripts/ci/kubernetes/ci_run_kubernetes_tests.sh), so rewriting it to Python is an extra effort which I agree we should make, but would prefer to do it in a separate contribution.
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       My main concern is that this script imports file `scripts/ci/libraries/_script_init.sh`, which initializes a lot of environment variables. If I wanted to write this file in Python I would have to find a way to access the following variables without using Bash: `BUILD_CACHE_DIR`, `AIRFLOW_PIP_VERSION`, `WHEEL_VERSION`, `CONSTRAINTS_GITHUB_REPOSITORY`, `DEFAULT_CONSTRAINTS_BRANCH`, `GITHUB_REGISTRY_PULL_IMAGE_TAG`.  I agree that duplicating code is not always the best idea, but generalizing this file and file `ci_run_kubernetes_tests.sh` in Bash would make it very complicated.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: .pre-commit-config.yaml
##########
@@ -238,6 +238,7 @@ repos:
           ^scripts/.*\.py$|
           ^dev|
           ^provider_packages|
+          ^docker_tests|

Review comment:
       When I turned it on, we got the following error messages. This is not what we expect.
   ```
   docker_tests/ci_image.py:28 in public class `TestFiles`:
           D101: Missing docstring in public class
   docker_tests/ci_image.py:33 in public class `TestPythonPackages`:
           D101: Missing docstring in public class
   docker_tests/prod_image.py:37 in public class `TestCommands`:
           D101: Missing docstring in public class
   docker_tests/prod_image.py:72 in public class `TestPythonPackages`:
           D101: Missing docstring in public class
   docker_tests/prod_image.py:169 in public class `TestExecuteAsRoot`:
           D101: Missing docstring in public class
   ```




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: .pre-commit-config.yaml
##########
@@ -238,6 +238,7 @@ repos:
           ^scripts/.*\.py$|
           ^dev|
           ^provider_packages|
+          ^docker_tests|

Review comment:
       Ah yeah. I miseed the first line wiht `^tests/` - it was hidden in GH interface :)
   




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]

Review comment:
       Just 3 levels up in directory tree.. I agree it's probably better than (abspath(join('..', .... but a comment might be useful




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Just to add a little more context on that one as well:
   
   And to add on that - we do not know what python version the user has. The idea of this script (similarly as previous BASH scripts) is that it should run pretty much everywhere without any "extra" assumptions. Just python3 should be enough to run the script - I think we can - for now assume safely have `python3.6+` as the "big assumption".  
   
   I would also love this to make sure it also works on Windows without WSL2 - this was the thing that was missing with `bash` approach - but ideally the workflow should be exactly the same for pure-windows developers too.
   
   So we have to be extra careful this one also works for Windows (this is something that we will likely add during the Internship - and some of our interns are using Windows primarily - so this is something they will be able to check and likely we can even add tests for it in GitHub on Windows runners (but that's a bit later stage).

##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Just to add a little more context on that one as well:
   
   And to add on that - we do not know what python version the user has. The idea of this script (similarly as previous BASH scripts) is that it should run pretty much everywhere without any "extra" assumptions. Just python3 should be enough to run the script - I think we can - for now - assume safely have `python3.6+` as the "big assumption".  
   
   I would also love this to make sure it also works on Windows without WSL2 - this was the thing that was missing with `bash` approach - but ideally the workflow should be exactly the same for pure-windows developers too.
   
   So we have to be extra careful this one also works for Windows (this is something that we will likely add during the Internship - and some of our interns are using Windows primarily - so this is something they will be able to check and likely we can even add tests for it in GitHub on Windows runners (but that's a bit later stage).




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Basically: with all those BashToPython changes we should assume that they are used by someone who has no dev env setup, nothing installed, just python (3.6+) installed on their OS (Modern Linux, MacOS, Windows). There should be no more prerequisites. Later we might add docker/docker-compose to the list of those prerequisites, but that should reall be it.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] github-actions[bot] commented on pull request #19737: Tests for Docker images in Python

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


   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 main 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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       Not really:
   
   1) BUILD_CACHE_DIR is always `AIRFLOW_SOURCES/.build` - the ENV var is only defined once to use it in many scripts and do not make a typo, but this could be easily hardcoded in the python script and we can generalize it later when we change the approach in similar way to "Chart unit tests" and "Helm tests' (whcih we will be able to next)
   
   2) pip install --upgrade pytest pytest-xdist - does not neeed constraints (that leaves most of the variables out). There is no reason why "environment" for "docker_tests" should be the same as Airifow one - we do not even install (and have no intentions to install) airflow here
   
   3) similarly there is no reason to use the same pip and wheel versions - those are really needed to install airflow (which actually happens when we run Helm tests because we are using imports from airflow there). Just to install pytest and pytest-xdist we can use "current PIP and no wheel" - we do not even need to reinstall them
   
   So what I am saying - this case and environment is far simpler, so it could be initialized in way simpler way - that's also one reason we should not carry all the "Breeze" baggage here - it's simply not needed here.
   
   
   
    

##########
File path: scripts/ci/images/ci_run_docker_tests.sh
##########
@@ -0,0 +1,121 @@
+#!/usr/bin/env bash

Review comment:
       And BTW. 
   
   I really like the idea of small, dedicated venvs for different tasks - separate venv for "dev", separate for "docker_tests", separate for "chart/tests" - especially if we add some scripting that will create them when needed automtically. So we should reapply this to other cases in the future.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])

Review comment:
       The problem is also with the interpreter. This script is run using the system interpreter and `pytest` uses the virtual environment. This means a big difference in `sys.path`. You can probably change that, but that adds extra complexity to the code area where we don't have too many contributors to troubleshoot, so I prefer to take the safe path.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])

Review comment:
       The problem is also with the interpreter. This script is run using the system interpreter and `pytest` uses the virtual environment. This means a big difference in `sys.path`. We can probably change that, but that adds extra complexity to the code area where we don't have too many contributors to troubleshoot, so I prefer to take the safe path.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")
+    subprocess.run(cmd, **kwargs)
+
+
+def create_virtualenv():
+    virtualenv_path = (
+        BUILD_CACHE_DIR / ".docker_venv" / f"host_python_{sys.version_info[0]}.{sys.version_info[1]}"
+    )
+    virtualenv_path.parent.mkdir(parents=True, exist_ok=True)
+    if not virtualenv_path.exists():
+        print("Creating virtualenv environment")
+        run_verbose([sys.executable, "-m", "venv", str(virtualenv_path)])
+
+    python_bin = virtualenv_path / "bin" / "python"
+    run_verbose([str(python_bin), "-m", "pip", "install", "pytest", "pytest-xdist"])
+    return python_bin
+
+
+def main():
+    parser = get_parser()
+    args = parser.parse_args()
+
+    python_bin = create_virtualenv()
+
+    if args.initialize:
+        return
+    if args.interactive:
+        activate_bin = python_bin.parent / "activate"
+        bash_trampoline = f"source {shlex.quote(str(activate_bin))}"
+        print("To enter virtual environment, run:")
+        print(f"    {bash_trampoline}")
+        return
+
+    extra_pytest_args = (
+        args.pytestopts[1:] if args.pytestopts and args.pytestopts[0] == "--" else args.pytestopts
+    )
+    if not extra_pytest_args:
+        raise SystemExit("You must select the tests to run.")
+
+    pytest_args = (
+        "--pythonwarnings=ignore::DeprecationWarning",
+        "--pythonwarnings=ignore::PendingDeprecationWarning",
+        "-n",
+        "auto",
+    )
+
+    run_verbose([str(python_bin), "-m", "pytest", *pytest_args, *extra_pytest_args])

Review comment:
       Oh yeah! 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: .pre-commit-config.yaml
##########
@@ -238,6 +238,7 @@ repos:
           ^scripts/.*\.py$|
           ^dev|
           ^provider_packages|
+          ^docker_tests|

Review comment:
       pydocstyle is disabled for all tests.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Let's add a term color here (blue?) - this will make the command stands out among other logs (better visibility in CI output).




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/images/ci_run_docker_tests.py
##########
@@ -0,0 +1,98 @@
+#!/usr/bin/env python
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import argparse
+import shlex
+import subprocess
+import sys
+from pathlib import Path
+from typing import List
+
+AIRFLOW_SOURCE = Path(__file__).resolve().parents[3]
+BUILD_CACHE_DIR = AIRFLOW_SOURCE / ".build"
+
+
+def get_parser():
+    parser = argparse.ArgumentParser(
+        prog="ci_run_docker_tests",
+        description="Running Docker tests using pytest",
+        epilog="Unknown arguments are passed unchanged to Pytest.",
+    )
+    parser.add_argument(
+        "--interactive",
+        "-i",
+        action='store_true',
+        help="Activates virtual environment ready to run tests and drops you in",
+    )
+    parser.add_argument("--initialize", action="store_true", help="Initialize virtual environment and exit")
+    parser.add_argument("pytestopts", nargs=argparse.REMAINDER, help="Tests to run")
+    return parser
+
+
+def run_verbose(cmd: List[str], **kwargs):
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Let's add a term color here (blue?) - this will make the command stands out among other logs.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] mik-laj commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/ci_image.py
##########
@@ -0,0 +1,42 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import subprocess
+import unittest
+
+from docker_tests.docker_tests_utils import (
+    display_dependency_conflict_message,
+    docker_image,
+    run_bash,
+    run_command,
+)
+
+
+class TestFiles(unittest.TestCase):

Review comment:
       Ahh. I forgot about this file. 




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] uranusjr commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: scripts/ci/libraries/_verify_image.sh
##########
@@ -19,11 +19,11 @@
 function verify_image::verify_prod_image {
     DOCKER_IMAGE="${1}"
     export DOCKER_IMAGE
-    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.sh" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"
+    "${SCRIPTS_CI_DIR}/images/ci_run_docker_tests.py" "${AIRFLOW_SOURCES}/docker_tests/prod_image.py"

Review comment:
       Because shebang requires an additional trip to the kernel, and the `env` context is sometimes finnicky to debug if somethings goes wrong. It’s not exactly wrong (and definitely useful), but if we can be explicit, we should.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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



[GitHub] [airflow] potiuk commented on a change in pull request #19737: Tests for Docker images in Python

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



##########
File path: docker_tests/docker_tests_utils.py
##########
@@ -28,7 +29,7 @@
 
 
 def run_command(cmd: List[str], print_output_on_error: bool = True, **kwargs):
-    print(f"$ {' '.join(c for c in cmd)}")
+    print(f"$ {' '.join(shlex.quote(c) for c in cmd)}")

Review comment:
       Side - comment: Following the learnings from breeze implementation I am planning (this is also part of the internship work that we planned) to document all such assumptions in the format of ADRs https://adr.github.io/  so that we avoid the problem that a lot of decisions with Breeze did not have enough "context" explained.




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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