You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2022/10/22 08:20:30 UTC

[airflow] branch main updated: Stop using deprecated set-output (#27193)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 5e542e7532 Stop using deprecated set-output (#27193)
5e542e7532 is described below

commit 5e542e7532521665dadc4d428c53cbc8e7e57c66
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Sat Oct 22 10:20:06 2022 +0200

    Stop using deprecated set-output (#27193)
    
    On October 11, 2022 GitHub deprecated "set-output" due to security
    concerns and replaced it with writing output name=value to
    GITHUB_OUTPUT file. The `set-ouptput` started to generate
    deprecation warnings.
    
    This PR replaces set-output with file redirection and removes the
    warnings.
    
    Additionally outputs that can be generated by selective checks
    are documented in the SELECTIVE_CHECKS.md documentation.
---
 .github/workflows/build-images.yml                 | 12 +++---
 .github/workflows/ci.yml                           | 24 ++++++-----
 .github/workflows/codeql-analysis.yml              |  2 +-
 .github/workflows/release_dockerhub_image.yml      |  2 +-
 BREEZE.rst                                         |  5 ++-
 dev/breeze/SELECTIVE_CHECKS.md                     | 48 ++++++++++++++++++++++
 .../src/airflow_breeze/commands/ci_commands.py     | 32 ++++++++-------
 .../src/airflow_breeze/utils/github_actions.py     |  2 +-
 dev/breeze/tests/test_selective_checks.py          | 33 ++++++++-------
 scripts/ci/constraints/ci_branch_constraints.sh    | 13 +++---
 10 files changed, 115 insertions(+), 58 deletions(-)

diff --git a/.github/workflows/build-images.yml b/.github/workflows/build-images.yml
index 7dd05fc53e..09a07fc91f 100644
--- a/.github/workflows/build-images.yml
+++ b/.github/workflows/build-images.yml
@@ -75,15 +75,15 @@ jobs:
         id: discover-pr-merge-commit
         run: |
           TARGET_COMMIT_SHA="$(gh api '${{ github.event.pull_request.url }}' --jq .merge_commit_sha)"
-          echo "TARGET_COMMIT_SHA=$TARGET_COMMIT_SHA" >> $GITHUB_ENV
-          echo "::set-output name=target-commit-sha::${TARGET_COMMIT_SHA}"
+          echo "TARGET_COMMIT_SHA=$TARGET_COMMIT_SHA" >> ${GITHUB_ENV}
+          echo "target-commit-sha=${TARGET_COMMIT_SHA}" >> ${GITHUB_OUTPUT}
         if: github.event_name == 'pull_request_target'
       # The labels in the event aren't updated when re-triggering the job, So lets hit the API to get
       # up-to-date values
       - name: Get latest PR labels
         id: get-latest-pr-labels
         run: |
-          echo -n "::set-output name=pull-request-labels::"
+          echo -n "pull-request-labels=" >> ${GITHUB_OUTPUT}
           gh api graphql --paginate -F node_id=${{github.event.pull_request.node_id}} -f query='
             query($node_id: ID!, $endCursor: String) {
               node(id:$node_id) {
@@ -94,7 +94,7 @@ jobs:
                   }
                 }
               }
-            }' --jq '.data.node.labels.nodes[]' | jq --slurp -c '[.[].name]'
+            }' --jq '.data.node.labels.nodes[]' | jq --slurp -c '[.[].name]' >> ${GITHUB_OUTPUT}
         if: github.event_name == 'pull_request_target'
       # Retrieve it to be able to determine which files has changed in the incoming commit of the PR
       # we checkout the target commit and it's parent to be able to compare them
@@ -120,7 +120,7 @@ jobs:
         # the PR, because the PR defaults have to be retrieved here.
         id: defaults
         run: |
-          python - <<EOF >>$GITHUB_ENV
+          python - <<EOF >> ${GITHUB_ENV}
           from pathlib import Path
           import re
           import sys
@@ -153,7 +153,7 @@ jobs:
         env:
           PR_LABELS: "${{ steps.get-latest-pr-labels.outputs.pull-request-labels }}"
           COMMIT_REF: "${{ env.TARGET_COMMIT_SHA }}"
-        run: breeze ci selective-check
+        run: breeze ci selective-check 2>> ${GITHUB_OUTPUT}
       - name: env
         run: printenv
         env:
diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 8a52b139b4..be14a568d4 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -195,7 +195,7 @@ jobs:
         # the build-images.yml (there we cannot import python code coming from the PR - we need to
         # treat the python code as text and extract the variables from there.
         run: |
-          python - <<EOF >>$GITHUB_ENV
+          python - <<EOF >> ${GITHUB_ENV}
           from pathlib import Path
           import re
           import sys
@@ -219,13 +219,13 @@ jobs:
       - run: ./scripts/ci/install_breeze.sh
       - name: "Get information about the Workflow"
         id: source-run-info
-        run: breeze ci get-workflow-info
+        run: breeze ci get-workflow-info 2>> ${GITHUB_OUTPUT}
       - name: Selective checks
         id: selective-checks
         env:
           PR_LABELS: "${{ steps.source-run-info.outputs.pr-labels }}"
           COMMIT_REF: "${{ github.sha }}"
-        run: breeze ci selective-check
+        run: breeze ci selective-check 2>> ${GITHUB_OUTPUT}
       - name: env
         run: printenv
         env:
@@ -696,8 +696,9 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
         env:
           IMAGE_TAG: ${{ env.IMAGE_TAG_FOR_THE_BUILD }}
       - name: "Get Python version"
-        run: "echo \"::set-output name=host-python-version::$(python -c
- 'import platform; print(platform.python_version())')\""
+        run: >
+         echo "host-python-version=$(python -c 'import platform; print(platform.python_version())')"
+         >> ${GITHUB_OUTPUT}
         id: host-python-version
       - name: "Static checks"
         run: breeze static-checks --all-files --show-diff-on-failure --color always
@@ -747,8 +748,9 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - name: "Free space"
         run: breeze ci free-space
       - name: "Get Python version"
-        run: "echo \"::set-output name=host-python-version::$(python -c
- 'import platform; print(platform.python_version())')\""
+        run: >
+          echo "host-python-version=$(python -c 'import platform; print(platform.python_version())')"
+          >> ${GITHUB_OUTPUT}
         id: host-python-version
       - name: "Static checks: basic checks only"
         run: >
@@ -1398,15 +1400,15 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
       - name: "Set issue id for main"
         if: github.ref == 'refs/heads/main'
         run: |
-          echo "ISSUE_ID=10118" >> $GITHUB_ENV
+          echo "ISSUE_ID=10118" >> ${GITHUB_ENV}
       - name: "Set issue id for v1-10-stable"
         if: github.ref == 'refs/heads/v1-10-stable'
         run: |
-          echo "ISSUE_ID=10127" >> $GITHUB_ENV
+          echo "ISSUE_ID=10127" >> ${GITHUB_ENV}
       - name: "Set issue id for v1-10-test"
         if: github.ref == 'refs/heads/v1-10-test'
         run: |
-          echo "ISSUE_ID=10128" >> $GITHUB_ENV
+          echo "ISSUE_ID=10128" >> ${GITHUB_ENV}
       - run: ./scripts/ci/install_breeze.sh
       - name: "Free space"
         run: breeze ci free-space
@@ -1689,7 +1691,7 @@ ${{ hashFiles('.pre-commit-config.yaml') }}"
           PYTHON_VERSIONS: ${{ needs.build-info.outputs.python-versions-list-as-string }}
       - name: "Set constraints branch name"
         id: constraints-branch
-        run: ./scripts/ci/constraints/ci_branch_constraints.sh
+        run: ./scripts/ci/constraints/ci_branch_constraints.sh >> ${GITHUB_OUTPUT}
         if: needs.build-info.outputs.canary-run == 'true'
       - name: Checkout ${{ steps.constraints-branch.outputs.branch }}
         uses: actions/checkout@v3
diff --git a/.github/workflows/codeql-analysis.yml b/.github/workflows/codeql-analysis.yml
index 7e6e2d0f28..8ba837cd94 100644
--- a/.github/workflows/codeql-analysis.yml
+++ b/.github/workflows/codeql-analysis.yml
@@ -55,7 +55,7 @@ jobs:
         id: selective-checks
         env:
           COMMIT_REF: "${{ github.sha }}"
-        run: breeze ci selective-check
+        run: breeze ci selective-check 2>> ${GITHUB_OUTPUT}
 
   analyze:
     name: Analyze
diff --git a/.github/workflows/release_dockerhub_image.yml b/.github/workflows/release_dockerhub_image.yml
index 2b22a3a0c1..0a8b9936a3 100644
--- a/.github/workflows/release_dockerhub_image.yml
+++ b/.github/workflows/release_dockerhub_image.yml
@@ -63,7 +63,7 @@ jobs:
       - run: ./scripts/ci/install_breeze.sh
       - name: Selective checks
         id: selective-checks
-        run: breeze ci selective-check
+        run: breeze ci selective-check 2>> ${GITHUB_OUTPUT}
   release-images:
     timeout-minutes: 120
     name: "Release images: ${{ github.event.inputs.airflowVersion }}, ${{ matrix.python-version }}"
diff --git a/BREEZE.rst b/BREEZE.rst
index 42897171b9..0aac12ebed 100644
--- a/BREEZE.rst
+++ b/BREEZE.rst
@@ -1408,7 +1408,10 @@ should be run on multiple combinations of Python, Kubernetes, Backend versions.
 needed to run the CI Builds. You can also use the tool to test what tests will be run when you provide
 a specific commit that Breeze should run the tests on.
 
-More details about the algorithm used to pick the right tests can be
+The selective-check command will produce the set of ``name=value`` pairs of outputs derived
+from the context of the commit/PR to be merged via stderr output.
+
+More details about the algorithm used to pick the right tests and the available outputs can be
 found in `Selective Checks <dev/breeze/SELECTIVE_CHECKS.md>`_.
 
 Those are all available flags of ``selective-check`` command:
diff --git a/dev/breeze/SELECTIVE_CHECKS.md b/dev/breeze/SELECTIVE_CHECKS.md
index c99139e34a..33d364e4a2 100644
--- a/dev/breeze/SELECTIVE_CHECKS.md
+++ b/dev/breeze/SELECTIVE_CHECKS.md
@@ -104,3 +104,51 @@ The logic implements the following rules:
 * if `Image building` is disabled, only basic pre-commits are enabled - no 'image-depending` pre-commits
   are enabled.
 * If there are some setup files changed, `upgrade to newer dependencies` is enabled.
+
+The selective check outputs available are described below:
+
+| Output                             | Meaning of the output                                                                                  | Example value                                                 |
+|------------------------------------|--------------------------------------------------------------------------------------------------------|---------------------------------------------------------------|
+| all-python-versions                | List of all python versions there are available in the form of JSON array                              | ['3.7', '3.8', '3.9', '3.10']                                 |
+| all-python-versions-list-as-string | List of all python versions there are available in the form of space separated string                  | 3.7 3.8 3.9 3.10                                              |
+| basic-checks-only                  | Whether to run all static checks ("false") or only basic set of static checks ("true")                 | false                                                         |
+| cache-directive                    | Which cache should be be used for images ("registry", "local" , "disabled")                            | registry                                                      |
+| debug-resources                    | Whether resources usage should be printed during parallel job execution ("true"/ "false")              | false                                                         |
+| default-branch                     | Which branch is default for the the build ("main" for main branch, "v2-4-test" for 2.4 line etc.)      | main                                                          |
+| default-constraints-branch         | Which branch is default for the the build ("constraints-main" for main branch, "constraints-2-4" etc.) | constraints-main                                              |
+| default-helm-version               | Which Helm version to use as default                                                                   | v3.9.4                                                        |
+| default-kind-version               | Which Kind version to use as default                                                                   | v0.16.0                                                       |
+| default-kubernetes-version         | Which Kubernetes version to use as default                                                             | v1.25.2                                                       |
+| default-mssql-version              | Which MsSQL version to use as default                                                                  | 2017-latest                                                   |
+| default-mysql-version              | Which MySQL version to use as default                                                                  | 5.7                                                           |
+| default-postgres-version           | Which Postgres version to use as default                                                               | 10                                                            |
+| default-python-version             | Which Python version to use as default                                                                 | 3.7                                                           |
+| docs-build                         | Whether to build documentation ("true"/"false")                                                        | true                                                          |
+| docs-filter                        | What filter to apply to docs building - used in non-main branches to skip provider and chart docs.     | --package-filter apache-airflow --package-filter docker-stack |
+| helm-version                       | Which Helm version to use for tests                                                                    | v3.9.4                                                        |
+| image-build                        | Whether CI image build is needed                                                                       | true                                                          |
+| kind-version                       | Which Kind version to use for tests                                                                    | v0.16.0                                                       |
+| kubernetes-combos                  | All combinations of Python version and Kubernetes version to use for tests as space-separated string   | 3.7-v1.25.2 3.8-v1.26.4                                       |
+| kubernetes-versions                | All Kubernetes versions to use for tests as JSON array                                                 | ['v1.25.2']                                                   |
+| kubernetes-versions-list-as-string | All Kubernetes versions to use for tests as space-separated string                                     | v1.25.2                                                       |
+| mssql-exclude                      | Which versions of MsSQL to exclude for tests as JSON array                                             | []                                                            |
+| mssql-versions                     | Which versions of MsSQL to use for tests as JSON array                                                 | ['2017-latest']                                               |
+| mysql-exclude                      | Which versions of MySQL to exclude for tests as JSON array                                             | []                                                            |
+| mysql-versions                     | Which versions of MySQL to use for tests as JSON array                                                 | ['5.7']                                                       |
+| needs-api-codegen                  | Whether "api-codegen" are needed to run ("true"/"false")                                               | true                                                          |
+| needs-api-tests                    | Whether "api-tests" are needed to run ("true"/"false")                                                 | true                                                          |
+| needs-helm-tests                   | Whether Helm tests are needed to run ("true"/"false")                                                  | true                                                          |
+| needs-javascript-scans             | Whether javascript CodeQL scans should be run ("true"/"false")                                         | true                                                          |
+| needs-python-scans                 | Whether Python CodeQL scans should be run ("true"/"false")                                             | true                                                          |
+| postgres-exclude                   | Which versions of Postgres to exclude for tests as JSON array                                          | []                                                            |
+| postgres-versions                  | Which versions of Postgres to use for tests as JSON array                                              | ['10']                                                        |
+| python-versions                    | Which versions of Python to use for tests as JSON array                                                | ['3.7']                                                       |
+| python-versions-list-as-string     | Which versions of MySQL to use for tests as space-separated string                                     | 3.7                                                           |
+| run-kubernetes-tests               | Whether Kubernetes tests should be run ("true"/"false")                                                | true                                                          |
+| run-tests                          | Whether unit tests should be run ("true"/"false")                                                      | true                                                          |
+| run-ui-tests                       | Whether UI tests should be run ("true"/"false")                                                        | true                                                          |
+| run-www-tests                      | Whether WWW tests should be run ("true"/"false")                                                       | true                                                          |
+| skip-pre-commits                   | Which pre-commits should be skipped during the static-checks run                                       | identity                                                      |
+| sqlite-exclude                     | Which versions of Sqlite to exclude for tests as JSON array                                            | []                                                            |
+| test-types                         | Which test types should be run for unit tests                                                          | API Always CLI Core Integration Other Providers WWW           |
+| upgrade-to-newer-dependencies      | Whether the image build should attempt to upgrade all dependencies                                     | false                                                         |
diff --git a/dev/breeze/src/airflow_breeze/commands/ci_commands.py b/dev/breeze/src/airflow_breeze/commands/ci_commands.py
index 111ee3e65f..bff3407c3c 100644
--- a/dev/breeze/src/airflow_breeze/commands/ci_commands.py
+++ b/dev/breeze/src/airflow_breeze/commands/ci_commands.py
@@ -26,7 +26,7 @@ import sys
 import tempfile
 from io import StringIO
 from pathlib import Path
-from typing import Any, NamedTuple
+from typing import Any, Iterable, NamedTuple
 
 import click
 
@@ -262,7 +262,7 @@ def selective_check(
         pr_labels=tuple(ast.literal_eval(pr_labels)) if pr_labels else (),
         github_event=github_event,
     )
-    print(str(sc))
+    print(str(sc), file=sys.stderr)
 
 
 @ci_group.command(name="find-newer-dependencies", help="Finds which dependencies are being upgraded.")
@@ -295,17 +295,21 @@ class WorkflowInfo(NamedTuple):
     ref_name: str | None
     pr_number: int | None
 
-    def print_ga_outputs(self):
-        print(get_ga_output(name="pr_labels", value=str(self.pull_request_labels)))
-        print(get_ga_output(name="target_repo", value=self.target_repo))
-        print(get_ga_output(name="head_repo", value=self.head_repo))
-        print(get_ga_output(name="pr_number", value=str(self.pr_number) if self.pr_number else ""))
-        print(get_ga_output(name="event_name", value=str(self.event_name)))
-        print(get_ga_output(name="runs-on", value=self.get_runs_on()))
-        print(get_ga_output(name='in-workflow-build', value=self.in_workflow_build()))
-        print(get_ga_output(name="build-job-description", value=self.get_build_job_description()))
-        print(get_ga_output(name="canary-run", value=self.is_canary_run()))
-        print(get_ga_output(name="run-coverage", value=self.run_coverage()))
+    def get_all_ga_outputs(self) -> Iterable[str]:
+        yield get_ga_output(name="pr_labels", value=str(self.pull_request_labels))
+        yield get_ga_output(name="target_repo", value=self.target_repo)
+        yield get_ga_output(name="head_repo", value=self.head_repo)
+        yield get_ga_output(name="pr_number", value=str(self.pr_number) if self.pr_number else "")
+        yield get_ga_output(name="event_name", value=str(self.event_name))
+        yield get_ga_output(name="runs-on", value=self.get_runs_on())
+        yield get_ga_output(name='in-workflow-build', value=self.in_workflow_build())
+        yield get_ga_output(name="build-job-description", value=self.get_build_job_description())
+        yield get_ga_output(name="canary-run", value=self.is_canary_run())
+        yield get_ga_output(name="run-coverage", value=self.run_coverage())
+
+    def print_all_ga_outputs(self):
+        for output in self.get_all_ga_outputs():
+            print(output, file=sys.stderr)
 
     def get_runs_on(self) -> str:
         for label in self.pull_request_labels:
@@ -420,4 +424,4 @@ def get_workflow_info(github_context: str, github_context_input: StringIO):
         )
         sys.exit(1)
     wi = workflow_info(context=context)
-    wi.print_ga_outputs()
+    wi.print_all_ga_outputs()
diff --git a/dev/breeze/src/airflow_breeze/utils/github_actions.py b/dev/breeze/src/airflow_breeze/utils/github_actions.py
index 271ed1e2c3..e96937b100 100644
--- a/dev/breeze/src/airflow_breeze/utils/github_actions.py
+++ b/dev/breeze/src/airflow_breeze/utils/github_actions.py
@@ -27,4 +27,4 @@ def get_ga_output(name: str, value: Any) -> str:
     output_name = name.replace('_', '-')
     printed_value = str(value).lower() if isinstance(value, bool) else value
     get_console().print(f"[info]{output_name}[/] = [green]{escape(str(printed_value))}[/]")
-    return f"::set-output name={output_name}::{printed_value}"
+    return f"{output_name}={printed_value}"
diff --git a/dev/breeze/tests/test_selective_checks.py b/dev/breeze/tests/test_selective_checks.py
index d200bef367..e502bb627c 100644
--- a/dev/breeze/tests/test_selective_checks.py
+++ b/dev/breeze/tests/test_selective_checks.py
@@ -22,9 +22,9 @@ from airflow_breeze.global_constants import GithubEvents
 from airflow_breeze.utils.selective_checks import SelectiveChecks
 
 
-def assert_outputs_are_printed(expected_outputs: dict[str, str], output: str):
+def assert_outputs_are_printed(expected_outputs: dict[str, str], stderr: str):
     for name, value in expected_outputs.items():
-        assert f"::set-output name={name}::{value}" in output
+        assert f"{name}={value}" in stderr
 
 
 @pytest.mark.parametrize(
@@ -241,14 +241,14 @@ def test_expected_output_pull_request_main(
     files: tuple[str, ...],
     expected_outputs: dict[str, str],
 ):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PULL_REQUEST,
         pr_labels=(),
         default_branch="main",
     )
-    assert_outputs_are_printed(expected_outputs, str(sc))
+    assert_outputs_are_printed(expected_outputs, str(stderr))
 
 
 @pytest.mark.parametrize(
@@ -336,15 +336,14 @@ def test_expected_output_full_tests_needed(
     default_branch: str,
     expected_outputs: dict[str, str],
 ):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PULL_REQUEST,
         pr_labels=pr_labels,
         default_branch=default_branch,
     )
-    output = str(sc)
-    assert_outputs_are_printed(expected_outputs, output)
+    assert_outputs_are_printed(expected_outputs, str(stderr))
 
 
 @pytest.mark.parametrize(
@@ -426,14 +425,14 @@ def test_expected_output_pull_request_v2_3(
     files: tuple[str, ...],
     expected_outputs: dict[str, str],
 ):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PULL_REQUEST,
         pr_labels=(),
         default_branch="v2-3-stable",
     )
-    assert_outputs_are_printed(expected_outputs, str(sc))
+    assert_outputs_are_printed(expected_outputs, str(stderr))
 
 
 @pytest.mark.parametrize(
@@ -496,14 +495,14 @@ def test_expected_output_pull_request_target(
     files: tuple[str, ...],
     expected_outputs: dict[str, str],
 ):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PULL_REQUEST_TARGET,
         pr_labels=(),
         default_branch="main",
     )
-    assert_outputs_are_printed(expected_outputs, str(sc))
+    assert_outputs_are_printed(expected_outputs, str(stderr))
 
 
 @pytest.mark.parametrize(
@@ -566,14 +565,14 @@ def test_expected_output_push(
     default_branch: str,
     expected_outputs: dict[str, str],
 ):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PUSH,
         pr_labels=pr_labels,
         default_branch=default_branch,
     )
-    assert_outputs_are_printed(expected_outputs, str(sc))
+    assert_outputs_are_printed(expected_outputs, str(stderr))
 
 
 @pytest.mark.parametrize(
@@ -587,7 +586,7 @@ def test_expected_output_push(
     ],
 )
 def test_no_commit_provided_trigger_full_build_for_any_event_type(github_event):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=(),
         commit_ref="",
         github_event=github_event,
@@ -607,7 +606,7 @@ def test_no_commit_provided_trigger_full_build_for_any_event_type(github_event):
             else "false",
             "test-types": "API Always CLI Core Integration Other Providers WWW",
         },
-        str(sc),
+        str(stderr),
     )
 
 
@@ -652,11 +651,11 @@ def test_no_commit_provided_trigger_full_build_for_any_event_type(github_event):
     ],
 )
 def test_upgrade_to_newer_dependencies(files: tuple[str, ...], expected_outputs: dict[str, str]):
-    sc = SelectiveChecks(
+    stderr = SelectiveChecks(
         files=files,
         commit_ref="HEAD",
         github_event=GithubEvents.PULL_REQUEST,
         pr_labels=(),
         default_branch="main",
     )
-    assert_outputs_are_printed(expected_outputs, str(sc))
+    assert_outputs_are_printed(expected_outputs, str(stderr))
diff --git a/scripts/ci/constraints/ci_branch_constraints.sh b/scripts/ci/constraints/ci_branch_constraints.sh
index 4883da4d28..d4f73e62e5 100755
--- a/scripts/ci/constraints/ci_branch_constraints.sh
+++ b/scripts/ci/constraints/ci_branch_constraints.sh
@@ -15,14 +15,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+# shellcheck disable=SC2086
 if [[ ${GITHUB_REF} == 'refs/heads/main' ]]; then
-  echo "::set-output name=branch::constraints-main"
+  echo "branch=constraints-main"
 elif [[ ${GITHUB_REF} =~ refs/heads/v([0-9\-]*)\-(test|stable) ]]; then
-  echo "::set-output name=branch::constraints-${BASH_REMATCH[1]}"
+  echo "branch=constraints-${BASH_REMATCH[1]}"
 else
   # Assume PR to constraints-main here
-  echo
-  echo "[${COLOR_YELLOW}Assuming that the PR is to 'main' branch!${COLOR_RESET}"
-  echo
-  echo "::set-output name=branch::constraints-main"
+  echo >&2
+  echo "[${COLOR_YELLOW}Assuming that the PR is to 'main' branch!${COLOR_RESET}" >&2
+  echo >&2
+  echo "branch=constraints-main"
 fi