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

eladkal 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 7d80b7d5e4 Refactor shorter defaults (#34350)
7d80b7d5e4 is described below

commit 7d80b7d5e48bf9f289337135f8721e01fc98d694
Author: Miroslav Šedivý <6774676+eum...@users.noreply.github.com>
AuthorDate: Sat Oct 21 19:06:25 2023 +0000

    Refactor shorter defaults (#34350)
---
 airflow/api/common/trigger_dag.py                        |  2 +-
 airflow/api_connexion/endpoints/extra_link_endpoint.py   |  4 +---
 airflow/cli/commands/info_command.py                     |  2 +-
 airflow/jobs/scheduler_job_runner.py                     |  2 +-
 airflow/models/dag.py                                    |  2 +-
 airflow/models/param.py                                  |  2 +-
 airflow/triggers/external_task.py                        |  2 +-
 airflow/utils/deprecation_tools.py                       |  2 +-
 airflow/www/views.py                                     | 16 ++++++++--------
 .../src/airflow_breeze/utils/docker_command_utils.py     |  2 +-
 helm_tests/airflow_core/test_dag_processor.py            |  2 +-
 helm_tests/airflow_core/test_scheduler.py                |  2 +-
 helm_tests/airflow_core/test_triggerer.py                |  2 +-
 helm_tests/airflow_core/test_worker.py                   |  2 +-
 helm_tests/other/test_flower.py                          |  2 +-
 helm_tests/other/test_pgbouncer.py                       |  2 +-
 helm_tests/other/test_statsd.py                          |  2 +-
 kubernetes_tests/test_base.py                            |  2 +-
 kubernetes_tests/test_kubernetes_pod_operator.py         |  2 +-
 .../pre_commit_update_providers_dependencies.py          |  4 +---
 tests/models/test_dagrun.py                              |  4 +---
 tests/operators/test_python.py                           |  6 ++----
 tests/providers/amazon/aws/system/utils/test_helpers.py  |  2 +-
 tests/providers/cncf/kubernetes/operators/test_pod.py    |  6 ++----
 tests/providers/dbt/cloud/operators/test_dbt_cloud.py    |  2 +-
 tests/providers/docker/decorators/test_docker.py         |  2 +-
 tests/providers/jenkins/sensors/test_jenkins.py          |  4 ++--
 .../microsoft/azure/operators/test_azure_data_factory.py |  4 ++--
 tests/system/providers/amazon/aws/utils/__init__.py      |  2 +-
 29 files changed, 40 insertions(+), 50 deletions(-)

diff --git a/airflow/api/common/trigger_dag.py 
b/airflow/api/common/trigger_dag.py
index a7b39cb5a6..231a300756 100644
--- a/airflow/api/common/trigger_dag.py
+++ b/airflow/api/common/trigger_dag.py
@@ -54,7 +54,7 @@ def _trigger_dag(
     if dag is None or dag_id not in dag_bag.dags:
         raise DagNotFound(f"Dag id {dag_id} not found")
 
-    execution_date = execution_date if execution_date else timezone.utcnow()
+    execution_date = execution_date or timezone.utcnow()
 
     if not timezone.is_localized(execution_date):
         raise ValueError("The execution_date should be localized")
diff --git a/airflow/api_connexion/endpoints/extra_link_endpoint.py 
b/airflow/api_connexion/endpoints/extra_link_endpoint.py
index 2e9954587c..ddf4b67028 100644
--- a/airflow/api_connexion/endpoints/extra_link_endpoint.py
+++ b/airflow/api_connexion/endpoints/extra_link_endpoint.py
@@ -71,7 +71,5 @@ def get_extra_links(
     all_extra_link_pairs = (
         (link_name, task.get_extra_links(ti, link_name)) for link_name in 
task.extra_links
     )
-    all_extra_links = {
-        link_name: link_url if link_url else None for link_name, link_url in 
sorted(all_extra_link_pairs)
-    }
+    all_extra_links = {link_name: link_url or None for link_name, link_url in 
sorted(all_extra_link_pairs)}
     return all_extra_links
diff --git a/airflow/cli/commands/info_command.py 
b/airflow/cli/commands/info_command.py
index 3a4ba396e4..3d72e4641e 100644
--- a/airflow/cli/commands/info_command.py
+++ b/airflow/cli/commands/info_command.py
@@ -162,7 +162,7 @@ class Architecture(Enum):
     def get_current() -> Architecture:
         """Get architecture."""
         current_architecture = 
_MACHINE_TO_ARCHITECTURE.get(platform.machine().lower())
-        return current_architecture if current_architecture else 
Architecture.UNKNOWN
+        return current_architecture or Architecture.UNKNOWN
 
 
 _MACHINE_TO_ARCHITECTURE: dict[str, Architecture] = {
diff --git a/airflow/jobs/scheduler_job_runner.py 
b/airflow/jobs/scheduler_job_runner.py
index 7d21cdff2e..41c8714b49 100644
--- a/airflow/jobs/scheduler_job_runner.py
+++ b/airflow/jobs/scheduler_job_runner.py
@@ -1002,7 +1002,7 @@ class SchedulerJobRunner(BaseJobRunner, LoggingMixin):
                 # If the scheduler is doing things, don't sleep. This means 
when there is work to do, the
                 # scheduler will run "as quick as possible", but when it's 
stopped, it can sleep, dropping CPU
                 # usage when "idle"
-                time.sleep(min(self._scheduler_idle_sleep_time, next_event if 
next_event else 0))
+                time.sleep(min(self._scheduler_idle_sleep_time, next_event or 
0))
 
             if loop_count >= self.num_runs > 0:
                 self.log.info(
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index c738f55e27..d2540a250b 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -470,7 +470,7 @@ class DAG(LoggingMixin):
         if tags and any(len(tag) > TAG_MAX_LEN for tag in tags):
             raise AirflowException(f"tag cannot be longer than {TAG_MAX_LEN} 
characters")
 
-        self.owner_links = owner_links if owner_links else {}
+        self.owner_links = owner_links or {}
         self.user_defined_macros = user_defined_macros
         self.user_defined_filters = user_defined_filters
         if default_args and not isinstance(default_args, dict):
diff --git a/airflow/models/param.py b/airflow/models/param.py
index 69b378c95c..d87b3721bb 100644
--- a/airflow/models/param.py
+++ b/airflow/models/param.py
@@ -118,7 +118,7 @@ class Param:
 
         if value is not NOTSET:
             self._warn_if_not_json(value)
-        final_val = value if value is not NOTSET else self.value
+        final_val = self.value if value is NOTSET else value
         if isinstance(final_val, ArgNotSet):
             if suppress_exception:
                 return None
diff --git a/airflow/triggers/external_task.py 
b/airflow/triggers/external_task.py
index c353c01159..269dbaebc8 100644
--- a/airflow/triggers/external_task.py
+++ b/airflow/triggers/external_task.py
@@ -70,7 +70,7 @@ class TaskStateTrigger(BaseTrigger):
         self.execution_dates = execution_dates
         self.poll_interval = poll_interval
         self.trigger_start_time = trigger_start_time
-        self.states = states if states else [TaskInstanceState.SUCCESS.value]
+        self.states = states or [TaskInstanceState.SUCCESS.value]
         self._timeout_sec = 60
 
     def serialize(self) -> tuple[str, dict[str, typing.Any]]:
diff --git a/airflow/utils/deprecation_tools.py 
b/airflow/utils/deprecation_tools.py
index 6be6650f09..e819e92b90 100644
--- a/airflow/utils/deprecation_tools.py
+++ b/airflow/utils/deprecation_tools.py
@@ -92,6 +92,6 @@ def add_deprecated_classes(
             imports,
             full_module_name,
             override_deprecated_classes_for_module,
-            extra_message if extra_message else "",
+            extra_message or "",
         )
         sys.modules.setdefault(full_module_name, module_type)
diff --git a/airflow/www/views.py b/airflow/www/views.py
index 88e46a14c2..b98bea370d 100644
--- a/airflow/www/views.py
+++ b/airflow/www/views.py
@@ -1006,7 +1006,7 @@ class Airflow(AirflowBaseView):
             dashboard_alerts=dashboard_alerts,
             
migration_moved_data_alerts=sorted(set(_iter_parsed_moved_data_table_names())),
             current_page=current_page,
-            search_query=arg_search_query if arg_search_query else "",
+            search_query=arg_search_query or "",
             page_title=Markup(page_title) if page_title_has_markup else 
page_title,
             page_size=dags_per_page,
             num_of_pages=num_of_pages,
@@ -1017,10 +1017,10 @@ class Airflow(AirflowBaseView):
                 current_page,
                 num_of_pages,
                 search=escape(arg_search_query) if arg_search_query else None,
-                status=arg_status_filter if arg_status_filter else None,
-                tags=arg_tags_filter if arg_tags_filter else None,
-                sorting_key=arg_sorting_key if arg_sorting_key else None,
-                sorting_direction=arg_sorting_direction if 
arg_sorting_direction else None,
+                status=arg_status_filter or None,
+                tags=arg_tags_filter or None,
+                sorting_key=arg_sorting_key or None,
+                sorting_direction=arg_sorting_direction or None,
             ),
             num_runs=num_runs,
             tags=tags,
@@ -2044,7 +2044,7 @@ class Airflow(AirflowBaseView):
                 dag=dag,
                 dag_id=dag_id,
                 origin=origin,
-                conf=request_conf if request_conf else {},
+                conf=request_conf or {},
                 form=form,
                 
is_dag_run_conf_overrides_params=is_dag_run_conf_overrides_params,
                 recent_confs=recent_confs,
@@ -3810,8 +3810,8 @@ class Airflow(AirflowBaseView):
             paging=wwwutils.generate_pages(
                 current_page,
                 num_of_pages,
-                sorting_key=arg_sorting_key if arg_sorting_key else None,
-                sorting_direction=arg_sorting_direction if 
arg_sorting_direction else None,
+                sorting_key=arg_sorting_key or None,
+                sorting_direction=arg_sorting_direction or None,
             ),
             sorting_key=arg_sorting_key,
             sorting_direction=arg_sorting_direction,
diff --git a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py 
b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
index e6960bd5a2..a9f9910c72 100644
--- a/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
+++ b/dev/breeze/src/airflow_breeze/utils/docker_command_utils.py
@@ -568,7 +568,7 @@ def update_expected_environment_variables(env: dict[str, 
str]) -> None:
     set_value_to_default_if_not_set(env, "AIRFLOW_CONSTRAINTS_REFERENCE", 
"constraints-source-providers")
     set_value_to_default_if_not_set(env, "AIRFLOW_EXTRAS", "")
     set_value_to_default_if_not_set(env, "AIRFLOW_ENABLE_AIP_44", "true")
-    set_value_to_default_if_not_set(env, "ANSWER", answer if answer is not 
None else "")
+    set_value_to_default_if_not_set(env, "ANSWER", answer or "")
     set_value_to_default_if_not_set(env, "BASE_BRANCH", "main")
     set_value_to_default_if_not_set(env, "BREEZE", "true")
     set_value_to_default_if_not_set(env, "BREEZE_INIT_COMMAND", "")
diff --git a/helm_tests/airflow_core/test_dag_processor.py 
b/helm_tests/airflow_core/test_dag_processor.py
index b9843c1594..ceb0267076 100644
--- a/helm_tests/airflow_core/test_dag_processor.py
+++ b/helm_tests/airflow_core/test_dag_processor.py
@@ -486,7 +486,7 @@ class TestDagProcessor:
             values=values,
             
show_only=["templates/dag-processor/dag-processor-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     @pytest.mark.parametrize("command", [None, ["custom", "command"]])
diff --git a/helm_tests/airflow_core/test_scheduler.py 
b/helm_tests/airflow_core/test_scheduler.py
index c7336bbc98..630e45ec4e 100644
--- a/helm_tests/airflow_core/test_scheduler.py
+++ b/helm_tests/airflow_core/test_scheduler.py
@@ -194,7 +194,7 @@ class TestScheduler:
             values=values,
             show_only=["templates/scheduler/scheduler-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     def test_should_create_valid_affinity_tolerations_and_node_selector(self):
diff --git a/helm_tests/airflow_core/test_triggerer.py 
b/helm_tests/airflow_core/test_triggerer.py
index 780f926196..615d68f9b3 100644
--- a/helm_tests/airflow_core/test_triggerer.py
+++ b/helm_tests/airflow_core/test_triggerer.py
@@ -73,7 +73,7 @@ class TestTriggerer:
             values=values,
             show_only=["templates/triggerer/triggerer-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     def test_disable_wait_for_migration(self):
diff --git a/helm_tests/airflow_core/test_worker.py 
b/helm_tests/airflow_core/test_worker.py
index 0bc8275c81..d58e9a72dc 100644
--- a/helm_tests/airflow_core/test_worker.py
+++ b/helm_tests/airflow_core/test_worker.py
@@ -61,7 +61,7 @@ class TestWorker:
             values=values,
             show_only=["templates/workers/worker-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     def test_should_add_extra_containers(self):
diff --git a/helm_tests/other/test_flower.py b/helm_tests/other/test_flower.py
index d018455a19..3977775414 100644
--- a/helm_tests/other/test_flower.py
+++ b/helm_tests/other/test_flower.py
@@ -65,7 +65,7 @@ class TestFlowerDeployment:
             values=values,
             show_only=["templates/flower/flower-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     @pytest.mark.parametrize(
diff --git a/helm_tests/other/test_pgbouncer.py 
b/helm_tests/other/test_pgbouncer.py
index f46e38e5fb..96b603da2a 100644
--- a/helm_tests/other/test_pgbouncer.py
+++ b/helm_tests/other/test_pgbouncer.py
@@ -116,7 +116,7 @@ class TestPgbouncer:
             values=values,
             show_only=["templates/pgbouncer/pgbouncer-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     def test_scheduler_name(self):
diff --git a/helm_tests/other/test_statsd.py b/helm_tests/other/test_statsd.py
index 820a59fe1e..399309702b 100644
--- a/helm_tests/other/test_statsd.py
+++ b/helm_tests/other/test_statsd.py
@@ -102,7 +102,7 @@ class TestStatsd:
             values=values,
             show_only=["templates/statsd/statsd-deployment.yaml"],
         )
-        expected_result = revision_history_limit if revision_history_limit 
else global_revision_history_limit
+        expected_result = revision_history_limit or 
global_revision_history_limit
         assert jmespath.search("spec.revisionHistoryLimit", docs[0]) == 
expected_result
 
     def test_scheduler_name(self):
diff --git a/kubernetes_tests/test_base.py b/kubernetes_tests/test_base.py
index 98c2e978c7..3507848079 100644
--- a/kubernetes_tests/test_base.py
+++ b/kubernetes_tests/test_base.py
@@ -108,7 +108,7 @@ class BaseK8STest:
 
     @staticmethod
     def _delete_airflow_pod(name=""):
-        suffix = "-" + name if name else ""
+        suffix = f"-{name}" if name else ""
         air_pod = check_output(["kubectl", "get", "pods"]).decode()
         air_pod = air_pod.splitlines()
         names = [re2.compile(r"\s+").split(x)[0] for x in air_pod if "airflow" 
+ suffix in x]
diff --git a/kubernetes_tests/test_kubernetes_pod_operator.py 
b/kubernetes_tests/test_kubernetes_pod_operator.py
index 86bf78bd29..44b4b3df80 100644
--- a/kubernetes_tests/test_kubernetes_pod_operator.py
+++ b/kubernetes_tests/test_kubernetes_pod_operator.py
@@ -75,7 +75,7 @@ def create_context(task) -> Context:
 @pytest.fixture(scope="session")
 def kubeconfig_path():
     kubeconfig_path = os.environ.get("KUBECONFIG")
-    return kubeconfig_path if kubeconfig_path else 
os.path.expanduser("~/.kube/config")
+    return kubeconfig_path or os.path.expanduser("~/.kube/config")
 
 
 @pytest.fixture
diff --git a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py 
b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py
index 34393f22ba..ada9f884be 100755
--- a/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py
+++ b/scripts/ci/pre_commit/pre_commit_update_providers_dependencies.py
@@ -201,9 +201,7 @@ if __name__ == "__main__":
             set(ALL_DEPENDENCIES[key]["cross-providers-deps"])
         )
         excluded_versions = ALL_PROVIDERS[key].get("excluded-python-versions")
-        unique_sorted_dependencies[key]["excluded-python-versions"] = (
-            excluded_versions if excluded_versions else []
-        )
+        unique_sorted_dependencies[key]["excluded-python-versions"] = 
excluded_versions or []
     if errors:
         console.print()
         console.print("[red]Errors found during verification. Exiting!")
diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py
index 9fc2ea86d4..50a54783ff 100644
--- a/tests/models/test_dagrun.py
+++ b/tests/models/test_dagrun.py
@@ -87,9 +87,7 @@ class TestDagRun:
         session: Session,
     ):
         now = timezone.utcnow()
-        if execution_date is None:
-            execution_date = now
-        execution_date = pendulum.instance(execution_date)
+        execution_date = pendulum.instance(execution_date or now)
         if is_backfill:
             run_type = DagRunType.BACKFILL_JOB
             data_interval = dag.infer_automated_data_interval(execution_date)
diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py
index 700b04a8ff..bfb1180353 100644
--- a/tests/operators/test_python.py
+++ b/tests/operators/test_python.py
@@ -832,15 +832,13 @@ class BaseTestPythonVirtualenvOperator(BasePythonTest):
 
         if expected_state == TaskInstanceState.FAILED:
             with pytest.raises(CalledProcessError):
-                self.run_as_task(
-                    f, op_kwargs={"exit_code": actual_exit_code}, 
**(extra_kwargs if extra_kwargs else {})
-                )
+                self.run_as_task(f, op_kwargs={"exit_code": actual_exit_code}, 
**(extra_kwargs or {}))
         else:
             ti = self.run_as_task(
                 f,
                 return_ti=True,
                 op_kwargs={"exit_code": actual_exit_code},
-                **(extra_kwargs if extra_kwargs else {}),
+                **(extra_kwargs or {}),
             )
             assert ti.state == expected_state
 
diff --git a/tests/providers/amazon/aws/system/utils/test_helpers.py 
b/tests/providers/amazon/aws/system/utils/test_helpers.py
index 3db17158da..8860a8d0b9 100644
--- a/tests/providers/amazon/aws/system/utils/test_helpers.py
+++ b/tests/providers/amazon/aws/system/utils/test_helpers.py
@@ -76,7 +76,7 @@ class TestAmazonSystemTestHelpers:
     def test_fetch_variable_success(
         self, mock_getenv, env_value, ssm_value, default_value, expected_result
     ) -> None:
-        mock_getenv.return_value = env_value if env_value else ssm_value
+        mock_getenv.return_value = env_value or ssm_value
 
         result = utils.fetch_variable(ANY, default_value) if default_value 
else utils.fetch_variable(ANY_STR)
 
diff --git a/tests/providers/cncf/kubernetes/operators/test_pod.py 
b/tests/providers/cncf/kubernetes/operators/test_pod.py
index d804f51469..42fd7ecfe4 100644
--- a/tests/providers/cncf/kubernetes/operators/test_pod.py
+++ b/tests/providers/cncf/kubernetes/operators/test_pod.py
@@ -1299,9 +1299,7 @@ class TestKubernetesPodOperator:
         self, remote_pod, extra_kwargs, actual_exit_code, expected_exc
     ):
         """Tests that an AirflowSkipException is raised when the container 
exits with the skip_on_exit_code"""
-        k = KubernetesPodOperator(
-            task_id="task", on_finish_action="delete_pod", **(extra_kwargs if 
extra_kwargs else {})
-        )
+        k = KubernetesPodOperator(task_id="task", 
on_finish_action="delete_pod", **(extra_kwargs or {}))
 
         base_container = MagicMock()
         base_container.name = k.base_container_name
@@ -1576,7 +1574,7 @@ class TestKubernetesPodOperatorAsync:
             in_cluster=True,
             get_logs=True,
             deferrable=True,
-            **(extra_kwargs if extra_kwargs else {}),
+            **(extra_kwargs or {}),
         )
 
         base_container = MagicMock()
diff --git a/tests/providers/dbt/cloud/operators/test_dbt_cloud.py 
b/tests/providers/dbt/cloud/operators/test_dbt_cloud.py
index 17a929c66f..471bd1f81c 100644
--- a/tests/providers/dbt/cloud/operators/test_dbt_cloud.py
+++ b/tests/providers/dbt/cloud/operators/test_dbt_cloud.py
@@ -367,7 +367,7 @@ class TestDbtCloudRunJobOperator:
 
         assert url == (
             EXPECTED_JOB_RUN_OP_EXTRA_LINK.format(
-                account_id=account_id if account_id else DEFAULT_ACCOUNT_ID,
+                account_id=account_id or DEFAULT_ACCOUNT_ID,
                 project_id=PROJECT_ID,
                 run_id=_run_response["data"]["id"],
             )
diff --git a/tests/providers/docker/decorators/test_docker.py 
b/tests/providers/docker/decorators/test_docker.py
index 49892c7aa4..8f1691d16f 100644
--- a/tests/providers/docker/decorators/test_docker.py
+++ b/tests/providers/docker/decorators/test_docker.py
@@ -129,7 +129,7 @@ class TestDockerDecorator:
         ],
     )
     def test_skip_docker_operator(self, extra_kwargs, actual_exit_code, 
expected_state, dag_maker):
-        @task.docker(image="python:3.9-slim", auto_remove="force", 
**(extra_kwargs if extra_kwargs else {}))
+        @task.docker(image="python:3.9-slim", auto_remove="force", 
**(extra_kwargs or {}))
         def f(exit_code):
             raise SystemExit(exit_code)
 
diff --git a/tests/providers/jenkins/sensors/test_jenkins.py 
b/tests/providers/jenkins/sensors/test_jenkins.py
index 56178e4b5d..c63275105a 100644
--- a/tests/providers/jenkins/sensors/test_jenkins.py
+++ b/tests/providers/jenkins/sensors/test_jenkins.py
@@ -44,7 +44,7 @@ class TestJenkinsBuildSensor:
     )
     @patch("jenkins.Jenkins")
     def test_poke_buliding(self, mock_jenkins, build_number, build_state, 
result):
-        target_build_number = build_number if build_number else 10
+        target_build_number = build_number or 10
 
         jenkins_mock = MagicMock()
         jenkins_mock.get_job_info.return_value = {"lastBuild": {"number": 
target_build_number}}
@@ -91,7 +91,7 @@ class TestJenkinsBuildSensor:
     def test_poke_finish_building(
         self, mock_jenkins, build_number, build_state, result, soft_fail, 
expected_exception
     ):
-        target_build_number = build_number if build_number else 10
+        target_build_number = build_number or 10
 
         jenkins_mock = MagicMock()
         jenkins_mock.get_job_info.return_value = {"lastBuild": {"number": 
target_build_number}}
diff --git 
a/tests/providers/microsoft/azure/operators/test_azure_data_factory.py 
b/tests/providers/microsoft/azure/operators/test_azure_data_factory.py
index 599a7b54e9..8baa3235fb 100644
--- a/tests/providers/microsoft/azure/operators/test_azure_data_factory.py
+++ b/tests/providers/microsoft/azure/operators/test_azure_data_factory.py
@@ -260,8 +260,8 @@ class TestAzureDataFactoryRunPipelineOperator:
             EXPECTED_PIPELINE_RUN_OP_EXTRA_LINK.format(
                 run_id=PIPELINE_RUN_RESPONSE["run_id"],
                 subscription_id=SUBSCRIPTION_ID,
-                resource_group_name=resource_group if resource_group else 
conn_resource_group_name,
-                factory_name=factory if factory else conn_factory_name,
+                resource_group_name=resource_group or conn_resource_group_name,
+                factory_name=factory or conn_factory_name,
             )
         )
 
diff --git a/tests/system/providers/amazon/aws/utils/__init__.py 
b/tests/system/providers/amazon/aws/utils/__init__.py
index 65afc5e3c6..513025fc87 100644
--- a/tests/system/providers/amazon/aws/utils/__init__.py
+++ b/tests/system/providers/amazon/aws/utils/__init__.py
@@ -97,7 +97,7 @@ def _fetch_from_ssm(key: str, test_name: str | None = None) 
-> str:
     :param key: The key to search for within the returned Parameter Value.
     :return: The value of the provided key from SSM
     """
-    _test_name: str = test_name if test_name else _get_test_name()
+    _test_name: str = test_name or _get_test_name()
     hook = SsmHook(aws_conn_id=None)
     value: str = ""
 

Reply via email to