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

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

commit 9e836034eea7dcdd629db46528156743e74b32ae
Author: Jarek Potiuk <[email protected]>
AuthorDate: Wed Apr 19 17:23:11 2023 +0200

    Add multiple exit code handling in skip logic for BashOperator (#30739)
    
    Follow-up after #30734
    
    (cherry picked from commit 719e5a921f82b31b7e845db1d0297eaa6edf5b6c)
---
 airflow/operators/bash.py    | 17 +++++++++++------
 tests/operators/test_bash.py | 10 +++++++++-
 2 files changed, 20 insertions(+), 7 deletions(-)

diff --git a/airflow/operators/bash.py b/airflow/operators/bash.py
index da5e3bdf09..e474bd8d68 100644
--- a/airflow/operators/bash.py
+++ b/airflow/operators/bash.py
@@ -20,7 +20,7 @@ from __future__ import annotations
 import os
 import shutil
 import warnings
-from typing import Sequence
+from typing import Container, Sequence
 
 from airflow.compat.functools import cached_property
 from airflow.exceptions import AirflowException, AirflowSkipException
@@ -143,7 +143,7 @@ class BashOperator(BaseOperator):
         append_env: bool = False,
         output_encoding: str = "utf-8",
         skip_exit_code: int | None = None,
-        skip_on_exit_code: int = 99,
+        skip_on_exit_code: int | Container[int] | None = 99,
         cwd: str | None = None,
         **kwargs,
     ) -> None:
@@ -155,9 +155,14 @@ class BashOperator(BaseOperator):
             warnings.warn(
                 "skip_exit_code is deprecated. Please use skip_on_exit_code", 
DeprecationWarning, stacklevel=2
             )
-            self.skip_on_exit_code = skip_exit_code
-        else:
-            self.skip_on_exit_code = skip_on_exit_code
+            skip_on_exit_code = skip_exit_code
+        self.skip_on_exit_code = (
+            skip_on_exit_code
+            if isinstance(skip_on_exit_code, Container)
+            else [skip_on_exit_code]
+            if skip_on_exit_code
+            else []
+        )
         self.cwd = cwd
         self.append_env = append_env
 
@@ -199,7 +204,7 @@ class BashOperator(BaseOperator):
             output_encoding=self.output_encoding,
             cwd=self.cwd,
         )
-        if self.skip_on_exit_code is not None and result.exit_code == 
self.skip_on_exit_code:
+        if self.skip_on_exit_code is not None and result.exit_code in 
self.skip_on_exit_code:
             raise AirflowSkipException(f"Bash command returned exit code 
{self.skip_on_exit_code}. Skipping.")
         elif result.exit_code != 0:
             raise AirflowException(
diff --git a/tests/operators/test_bash.py b/tests/operators/test_bash.py
index 3c8320e1b2..43d6375409 100644
--- a/tests/operators/test_bash.py
+++ b/tests/operators/test_bash.py
@@ -176,14 +176,22 @@ class TestBashOperator:
             ({"skip_on_exit_code": 100}, 100, AirflowSkipException),
             ({"skip_on_exit_code": 100}, 101, AirflowException),
             ({"skip_on_exit_code": None}, 99, AirflowException),
+            ({"skip_on_exit_code": [100]}, 100, AirflowSkipException),
+            ({"skip_on_exit_code": (100, 101)}, 100, AirflowSkipException),
+            ({"skip_on_exit_code": 100}, 101, AirflowException),
+            ({"skip_on_exit_code": [100, 102]}, 101, AirflowException),
+            ({"skip_on_exit_code": None}, 0, None),
         ],
     )
     def test_skip(self, extra_kwargs, actual_exit_code, expected_exc):
         kwargs = dict(task_id="abc", bash_command=f'set -e; echo "hello 
world"; exit {actual_exit_code};')
         if extra_kwargs:
             kwargs.update(**extra_kwargs)
-        with pytest.raises(expected_exc):
+        if expected_exc is None:
             BashOperator(**kwargs).execute({})
+        else:
+            with pytest.raises(expected_exc):
+                BashOperator(**kwargs).execute({})
 
     def test_bash_operator_multi_byte_output(self):
         op = BashOperator(

Reply via email to