You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ep...@apache.org on 2023/03/08 13:17:59 UTC

[airflow] 04/12: Fix discoverability of tests for ARM in Breeze (#28432)

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

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

commit 7787fdb10f7d6de369648108232ae79fba8c963c
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Sun Dec 18 18:03:46 2022 +0100

    Fix discoverability of tests for ARM in Breeze (#28432)
    
    Breeze in case of ARM processor lacks support for several components
    (because they do not have supported ARM binaries available):
    
    * MySQL
    * MSSQL
    * LevelDB
    * Azure Service Bus
    
    When you try to attempt to run pytest on a group of tests that import
    one of those, the collection failed and none of the tests could run
    even if some of them could.
    
    This change uses pytest's skip on a module level and local imports
    in case the tests are inter-mixed with other tests in the same module
    to avoid import errors during collection.
    
    The try/except pattern over pytest.importorskip is preferred because
    we are using try/except in a number of other cases and we are pretty
    familiar with similar pattern and importorskipi has a bit unexpected
    behaviour (it returns imported module and you do not see the usual
    `import nnnn`. Also in our case we often wrap more than one
    import in one try/except (and it would lead to a duplicating messages
    to print really.
    
    We also add a separate command in ci to just perform a collection of
    tests and see if all tests are collectable after uninstalling all
    those libraries. This would prevent the problems from reapparing.
    
    Isort fixes are implemented for recently relesed isort version
    
    (cherry picked from commit 2a78f50b36eb7d0e4589633d12458eabbf82418d)
---
 .github/workflows/ci.yml                           |  6 ++-
 scripts/in_container/test_arm_pytest_collection.py | 53 ++++++++++++++++++++++
 tests/operators/test_generic_transfer.py           |  6 ++-
 .../apache/hive/transfers/test_mssql_to_hive.py    | 12 +++--
 .../apache/hive/transfers/test_mysql_to_hive.py    |  8 +++-
 .../cloud/transfers/test_bigquery_to_mssql.py      |  9 +++-
 .../google/cloud/transfers/test_mssql_to_gcs.py    |  8 +++-
 .../google/cloud/transfers/test_mysql_to_gcs.py    | 11 +++--
 .../google/cloud/triggers/test_cloud_build.py      |  2 -
 .../providers/google/leveldb/hooks/test_leveldb.py |  7 ++-
 .../google/leveldb/operators/test_leveldb.py       | 11 ++++-
 tests/providers/microsoft/azure/hooks/test_asb.py  |  8 +++-
 .../microsoft/azure/operators/test_asb.py          |  6 ++-
 .../providers/microsoft/mssql/hooks/test_mssql.py  |  6 ++-
 .../microsoft/mssql/operators/test_mssql.py        | 10 +++-
 tests/providers/mysql/hooks/test_mysql.py          | 11 ++++-
 .../mysql/transfers/test_vertica_to_mysql.py       |  8 +++-
 .../cloud/bigquery/example_bigquery_to_mssql.py    |  8 +++-
 .../google/cloud/gcs/example_mssql_to_gcs.py       |  9 +++-
 .../google/cloud/gcs/example_mysql_to_gcs.py       |  9 +++-
 .../providers/google/leveldb/example_leveldb.py    | 10 +++-
 .../microsoft/azure/example_azure_service_bus.py   | 30 +++++++-----
 .../providers/microsoft/mssql/example_mssql.py     | 10 +++-
 23 files changed, 212 insertions(+), 46 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index 63138de93f..4c8bc96151 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -864,8 +864,10 @@ jobs:
         uses: ./.github/actions/prepare_breeze_and_image
       - name: "Migration Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}"
         uses: ./.github/actions/migration_tests
-      - name: "Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}} (w/Kerberos)"
+      - name: "Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}"
         run: breeze testing tests --run-in-parallel
+      - name: "Tests ARM Pytest collection: ${{matrix.python-version}}"
+        run: breeze shell "python /opt/airflow/scripts/in_container/test_arm_pytest_collection.py"
       - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}"
         uses: ./.github/actions/post_tests
 
@@ -989,6 +991,8 @@ jobs:
         uses: ./.github/actions/migration_tests
       - name: "Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}"
         run: breeze testing tests --run-in-parallel
+      - name: "Tests ARM Pytest collection: ${{matrix.python-version}}"
+        run: breeze shell "python /opt/airflow/scripts/in_container/test_arm_pytest_collection.py"
       - name: "Post Tests: ${{matrix.python-version}}:${{needs.build-info.outputs.test-types}}"
         uses: ./.github/actions/post_tests
 
diff --git a/scripts/in_container/test_arm_pytest_collection.py b/scripts/in_container/test_arm_pytest_collection.py
new file mode 100755
index 0000000000..43277c5562
--- /dev/null
+++ b/scripts/in_container/test_arm_pytest_collection.py
@@ -0,0 +1,53 @@
+#!/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.
+from __future__ import annotations
+
+import json
+import re
+import subprocess
+from pathlib import Path
+
+from rich.console import Console
+
+AIRFLOW_SOURCES_ROOT = Path(__file__).parents[2].resolve()
+
+if __name__ == "__main__":
+    console = Console(width=400, color_system="standard")
+
+    provider_dependencies = json.loads(
+        (AIRFLOW_SOURCES_ROOT / "generated" / "provider_dependencies.json").read_text()
+    )
+    all_dependencies_to_remove = []
+    for provider in provider_dependencies:
+        for dependency in provider_dependencies[provider]["deps"]:
+            if 'platform_machine != "aarch64"' in dependency:
+                all_dependencies_to_remove.append(re.split(r"[~<>=;]", dependency)[0])
+    console.print(
+        "\n[bright_blue]Uninstalling ARM-incompatible libraries "
+        + " ".join(all_dependencies_to_remove)
+        + "\n"
+    )
+    subprocess.run(["pip", "uninstall", "-y"] + all_dependencies_to_remove)
+    result = subprocess.run(["pytest", "--collect-only", "-qqqq", "--disable-warnings", "tests"], check=False)
+    if result.returncode != 0:
+        console.print("\n[red]Test collection in ARM environment failed.")
+        console.print(
+            "[yellow]You should wrap the failing imports in try/except/skip clauses\n"
+            "See similar examples as skipped tests right above.\n"
+        )
+        exit(result.returncode)
diff --git a/tests/operators/test_generic_transfer.py b/tests/operators/test_generic_transfer.py
index 202478fe48..aaf7c07262 100644
--- a/tests/operators/test_generic_transfer.py
+++ b/tests/operators/test_generic_transfer.py
@@ -26,10 +26,8 @@ from parameterized import parameterized
 
 from airflow.models.dag import DAG
 from airflow.operators.generic_transfer import GenericTransfer
-from airflow.providers.mysql.hooks.mysql import MySqlHook
 from airflow.providers.postgres.hooks.postgres import PostgresHook
 from airflow.utils import timezone
-from tests.providers.mysql.hooks.test_mysql import MySqlContext
 
 DEFAULT_DATE = timezone.datetime(2015, 1, 1)
 DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
@@ -45,6 +43,8 @@ class TestMySql(unittest.TestCase):
         self.dag = dag
 
     def tearDown(self):
+        from airflow.providers.mysql.hooks.mysql import MySqlHook
+
         drop_tables = {"test_mysql_to_mysql", "test_airflow"}
         with closing(MySqlHook().get_conn()) as conn:
             for table in drop_tables:
@@ -59,6 +59,8 @@ class TestMySql(unittest.TestCase):
         ]
     )
     def test_mysql_to_mysql(self, client):
+        from tests.providers.mysql.hooks.test_mysql import MySqlContext
+
         with MySqlContext(client):
             sql = "SELECT * FROM connection;"
             op = GenericTransfer(
diff --git a/tests/providers/apache/hive/transfers/test_mssql_to_hive.py b/tests/providers/apache/hive/transfers/test_mssql_to_hive.py
index 53e49207e5..5ad8b329b6 100644
--- a/tests/providers/apache/hive/transfers/test_mssql_to_hive.py
+++ b/tests/providers/apache/hive/transfers/test_mssql_to_hive.py
@@ -20,11 +20,17 @@ from __future__ import annotations
 from collections import OrderedDict
 from unittest.mock import Mock, PropertyMock, patch
 
-import pymssql
+import pytest
 
-from airflow.providers.apache.hive.transfers.mssql_to_hive import MsSqlToHiveOperator
+try:
+    import pymssql
 
+    from airflow.providers.apache.hive.transfers.mssql_to_hive import MsSqlToHiveOperator
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
 
+
+@pytest.mark.backend("mssql")
 class TestMsSqlToHiveTransfer:
     def setup_method(self):
         self.kwargs = dict(sql="sql", hive_table="table", task_id="test_mssql_to_hive", dag=None)
@@ -36,13 +42,11 @@ class TestMsSqlToHiveTransfer:
         assert mapped_type == "INT"
 
     def test_type_map_decimal(self):
-
         mapped_type = MsSqlToHiveOperator(**self.kwargs).type_map(pymssql.DECIMAL.value)
 
         assert mapped_type == "FLOAT"
 
     def test_type_map_number(self):
-
         mapped_type = MsSqlToHiveOperator(**self.kwargs).type_map(pymssql.NUMBER.value)
 
         assert mapped_type == "INT"
diff --git a/tests/providers/apache/hive/transfers/test_mysql_to_hive.py b/tests/providers/apache/hive/transfers/test_mysql_to_hive.py
index 06f3259afc..0e601263ab 100644
--- a/tests/providers/apache/hive/transfers/test_mysql_to_hive.py
+++ b/tests/providers/apache/hive/transfers/test_mysql_to_hive.py
@@ -25,10 +25,14 @@ from unittest import mock
 import pytest
 
 from airflow.providers.apache.hive.hooks.hive import HiveCliHook
-from airflow.providers.apache.hive.transfers.mysql_to_hive import MySqlToHiveOperator
-from airflow.providers.mysql.hooks.mysql import MySqlHook
 from airflow.utils import timezone
 
+try:
+    from airflow.providers.apache.hive.transfers.mysql_to_hive import MySqlToHiveOperator
+    from airflow.providers.mysql.hooks.mysql import MySqlHook
+except ImportError:
+    pytest.skip("MysQL and/or hive not available", allow_module_level=True)
+
 DEFAULT_DATE = timezone.datetime(2015, 1, 1)
 DEFAULT_DATE_ISO = DEFAULT_DATE.isoformat()
 DEFAULT_DATE_DS = DEFAULT_DATE_ISO[:10]
diff --git a/tests/providers/google/cloud/transfers/test_bigquery_to_mssql.py b/tests/providers/google/cloud/transfers/test_bigquery_to_mssql.py
index 9554749622..3c4e8f0ecc 100644
--- a/tests/providers/google/cloud/transfers/test_bigquery_to_mssql.py
+++ b/tests/providers/google/cloud/transfers/test_bigquery_to_mssql.py
@@ -20,7 +20,13 @@ from __future__ import annotations
 import unittest
 from unittest import mock
 
-from airflow.providers.google.cloud.transfers.bigquery_to_mssql import BigQueryToMsSqlOperator
+import pytest
+
+try:
+    from airflow.providers.google.cloud.transfers.bigquery_to_mssql import BigQueryToMsSqlOperator
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
+
 
 TASK_ID = "test-bq-create-table-operator"
 TEST_PROJECT_ID = "test-project"
@@ -29,6 +35,7 @@ TEST_TABLE_ID = "test-table-id"
 TEST_DAG_ID = "test-bigquery-operators"
 
 
+@pytest.mark.backend("mssql")
 class TestBigQueryToMsSqlOperator(unittest.TestCase):
     @mock.patch("airflow.providers.google.cloud.transfers.bigquery_to_mssql.BigQueryHook")
     def test_execute_good_request_to_bq(self, mock_hook):
diff --git a/tests/providers/google/cloud/transfers/test_mssql_to_gcs.py b/tests/providers/google/cloud/transfers/test_mssql_to_gcs.py
index 9bd9e43b12..f2aeb218e3 100644
--- a/tests/providers/google/cloud/transfers/test_mssql_to_gcs.py
+++ b/tests/providers/google/cloud/transfers/test_mssql_to_gcs.py
@@ -21,9 +21,14 @@ import datetime
 import unittest
 from unittest import mock
 
+import pytest
 from parameterized import parameterized
 
-from airflow.providers.google.cloud.transfers.mssql_to_gcs import MSSQLToGCSOperator
+try:
+    from airflow.providers.google.cloud.transfers.mssql_to_gcs import MSSQLToGCSOperator
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
+
 
 TASK_ID = "test-mssql-to-gcs"
 MSSQL_CONN_ID = "mssql_conn_test"
@@ -49,6 +54,7 @@ SCHEMA_JSON = [
 ]
 
 
+@pytest.mark.backend("mssql")
 class TestMsSqlToGoogleCloudStorageOperator(unittest.TestCase):
     @parameterized.expand(
         [
diff --git a/tests/providers/google/cloud/transfers/test_mysql_to_gcs.py b/tests/providers/google/cloud/transfers/test_mysql_to_gcs.py
index 0101548843..4688b9bf77 100644
--- a/tests/providers/google/cloud/transfers/test_mysql_to_gcs.py
+++ b/tests/providers/google/cloud/transfers/test_mysql_to_gcs.py
@@ -23,11 +23,8 @@ import unittest
 from unittest import mock
 
 import pytest
-from MySQLdb import ProgrammingError
 from parameterized import parameterized
 
-from airflow.providers.google.cloud.transfers.mysql_to_gcs import MySQLToGCSOperator
-
 TASK_ID = "test-mysql-to-gcs"
 MYSQL_CONN_ID = "mysql_conn_test"
 TZ_QUERY = "SET time_zone = '+00:00'"
@@ -69,7 +66,15 @@ CUSTOM_SCHEMA_JSON = [
     b'{"mode": "REQUIRED", "name": "some_num", "type": "TIMESTAMP"}]',
 ]
 
+try:
+    from MySQLdb import ProgrammingError
+
+    from airflow.providers.google.cloud.transfers.mysql_to_gcs import MySQLToGCSOperator
+except ImportError:
+    pytest.skip("MySQL not available", allow_module_level=True)
+
 
+@pytest.mark.backend("mysql")
 class TestMySqlToGoogleCloudStorageOperator(unittest.TestCase):
     def test_init(self):
         """Test MySqlToGoogleCloudStorageOperator instance is properly initialized."""
diff --git a/tests/providers/google/cloud/triggers/test_cloud_build.py b/tests/providers/google/cloud/triggers/test_cloud_build.py
index 62203ddacb..f3658e37ba 100644
--- a/tests/providers/google/cloud/triggers/test_cloud_build.py
+++ b/tests/providers/google/cloud/triggers/test_cloud_build.py
@@ -83,8 +83,6 @@ TEST_BUILD_INSTANCE = dict(
     warnings=[],
 )
 
-pytest.hook = CloudBuildAsyncHook(gcp_conn_id="google_cloud_default")
-
 
 @pytest.fixture
 def hook():
diff --git a/tests/providers/google/leveldb/hooks/test_leveldb.py b/tests/providers/google/leveldb/hooks/test_leveldb.py
index dd15b66fb5..a0fd5b6ddf 100644
--- a/tests/providers/google/leveldb/hooks/test_leveldb.py
+++ b/tests/providers/google/leveldb/hooks/test_leveldb.py
@@ -22,7 +22,12 @@ from unittest import mock
 
 import pytest
 
-from airflow.providers.google.leveldb.hooks.leveldb import LevelDBHook, LevelDBHookException
+from airflow.exceptions import AirflowOptionalProviderFeatureException
+
+try:
+    from airflow.providers.google.leveldb.hooks.leveldb import LevelDBHook, LevelDBHookException
+except AirflowOptionalProviderFeatureException:
+    pytest.skip("LevelDB not available", allow_module_level=True)
 
 
 class TestLevelDBHook(unittest.TestCase):
diff --git a/tests/providers/google/leveldb/operators/test_leveldb.py b/tests/providers/google/leveldb/operators/test_leveldb.py
index b2993f7ffd..3becdf9a86 100644
--- a/tests/providers/google/leveldb/operators/test_leveldb.py
+++ b/tests/providers/google/leveldb/operators/test_leveldb.py
@@ -36,8 +36,15 @@ from __future__ import annotations
 import unittest
 from unittest import mock
 
-from airflow.providers.google.leveldb.hooks.leveldb import LevelDBHook
-from airflow.providers.google.leveldb.operators.leveldb import LevelDBOperator
+import pytest
+
+from airflow.exceptions import AirflowOptionalProviderFeatureException
+
+try:
+    from airflow.providers.google.leveldb.hooks.leveldb import LevelDBHook
+    from airflow.providers.google.leveldb.operators.leveldb import LevelDBOperator
+except AirflowOptionalProviderFeatureException:
+    pytest.skip("LevelDB not available", allow_module_level=True)
 
 
 class TestLevelDBOperator(unittest.TestCase):
diff --git a/tests/providers/microsoft/azure/hooks/test_asb.py b/tests/providers/microsoft/azure/hooks/test_asb.py
index fc5c8dba21..61ce05a0b9 100644
--- a/tests/providers/microsoft/azure/hooks/test_asb.py
+++ b/tests/providers/microsoft/azure/hooks/test_asb.py
@@ -19,8 +19,12 @@ from __future__ import annotations
 from unittest import mock
 
 import pytest
-from azure.servicebus import ServiceBusClient, ServiceBusMessage, ServiceBusMessageBatch
-from azure.servicebus.management import ServiceBusAdministrationClient
+
+try:
+    from azure.servicebus import ServiceBusClient, ServiceBusMessage, ServiceBusMessageBatch
+    from azure.servicebus.management import ServiceBusAdministrationClient
+except ImportError:
+    pytest.skip("Azure Service Bus not available", allow_module_level=True)
 
 from airflow.models import Connection
 from airflow.providers.microsoft.azure.hooks.asb import AdminClientHook, MessageHook
diff --git a/tests/providers/microsoft/azure/operators/test_asb.py b/tests/providers/microsoft/azure/operators/test_asb.py
index d7b162ec32..e35f7f9cf7 100644
--- a/tests/providers/microsoft/azure/operators/test_asb.py
+++ b/tests/providers/microsoft/azure/operators/test_asb.py
@@ -19,7 +19,11 @@ from __future__ import annotations
 from unittest import mock
 
 import pytest
-from azure.servicebus import ServiceBusMessage
+
+try:
+    from azure.servicebus import ServiceBusMessage
+except ImportError:
+    pytest.skip("Azure Service Bus not available", allow_module_level=True)
 
 from airflow.providers.microsoft.azure.operators.asb import (
     ASBReceiveSubscriptionMessageOperator,
diff --git a/tests/providers/microsoft/mssql/hooks/test_mssql.py b/tests/providers/microsoft/mssql/hooks/test_mssql.py
index 0b899fe370..6bffab744b 100644
--- a/tests/providers/microsoft/mssql/hooks/test_mssql.py
+++ b/tests/providers/microsoft/mssql/hooks/test_mssql.py
@@ -23,7 +23,11 @@ from urllib.parse import quote_plus
 import pytest
 
 from airflow.models import Connection
-from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
+
+try:
+    from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
 
 PYMSSQL_CONN = Connection(
     conn_type="mssql", host="ip", schema="share", login="username", password="password", port=8081
diff --git a/tests/providers/microsoft/mssql/operators/test_mssql.py b/tests/providers/microsoft/mssql/operators/test_mssql.py
index 5f0955ac03..62865c85f5 100644
--- a/tests/providers/microsoft/mssql/operators/test_mssql.py
+++ b/tests/providers/microsoft/mssql/operators/test_mssql.py
@@ -20,9 +20,15 @@ from __future__ import annotations
 from unittest import mock
 from unittest.mock import MagicMock, Mock
 
+import pytest
+
 from airflow import AirflowException
-from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
-from airflow.providers.microsoft.mssql.operators.mssql import MsSqlOperator
+
+try:
+    from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
+    from airflow.providers.microsoft.mssql.operators.mssql import MsSqlOperator
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
 
 
 class TestMsSqlOperator:
diff --git a/tests/providers/mysql/hooks/test_mysql.py b/tests/providers/mysql/hooks/test_mysql.py
index 4c4d991864..eefc34c51b 100644
--- a/tests/providers/mysql/hooks/test_mysql.py
+++ b/tests/providers/mysql/hooks/test_mysql.py
@@ -23,12 +23,19 @@ import uuid
 from contextlib import closing
 from unittest import mock
 
-import MySQLdb.cursors
 import pytest
 
 from airflow.models import Connection
 from airflow.models.dag import DAG
-from airflow.providers.mysql.hooks.mysql import MySqlHook
+
+try:
+    import MySQLdb.cursors
+
+    from airflow.providers.mysql.hooks.mysql import MySqlHook
+except ImportError:
+    pytest.skip("MySQL not available", allow_module_level=True)
+
+
 from airflow.utils import timezone
 from tests.test_utils.asserts import assert_equal_ignore_multiple_spaces
 
diff --git a/tests/providers/mysql/transfers/test_vertica_to_mysql.py b/tests/providers/mysql/transfers/test_vertica_to_mysql.py
index e13c5d0578..82997a46f4 100644
--- a/tests/providers/mysql/transfers/test_vertica_to_mysql.py
+++ b/tests/providers/mysql/transfers/test_vertica_to_mysql.py
@@ -20,8 +20,14 @@ from __future__ import annotations
 import datetime
 from unittest import mock
 
+import pytest
+
 from airflow.models.dag import DAG
-from airflow.providers.mysql.transfers.vertica_to_mysql import VerticaToMySqlOperator
+
+try:
+    from airflow.providers.mysql.transfers.vertica_to_mysql import VerticaToMySqlOperator
+except ImportError:
+    pytest.skip("MySQL not available", allow_module_level=True)
 
 
 def mock_get_conn():
diff --git a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py
index d9a409791e..c3747ec589 100644
--- a/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py
+++ b/tests/system/providers/google/cloud/bigquery/example_bigquery_to_mssql.py
@@ -23,13 +23,19 @@ from __future__ import annotations
 import os
 from datetime import datetime
 
+import pytest
+
 from airflow import models
 from airflow.providers.google.cloud.operators.bigquery import (
     BigQueryCreateEmptyDatasetOperator,
     BigQueryCreateEmptyTableOperator,
     BigQueryDeleteDatasetOperator,
 )
-from airflow.providers.google.cloud.transfers.bigquery_to_mssql import BigQueryToMsSqlOperator
+
+try:
+    from airflow.providers.google.cloud.transfers.bigquery_to_mssql import BigQueryToMsSqlOperator
+except ImportError:
+    pytest.skip("MySQL not available", allow_module_level=True)
 
 ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
 PROJECT_ID = os.environ.get("GCP_PROJECT_ID", "example-project")
diff --git a/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py
index b762970e72..76b9a68095 100644
--- a/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py
+++ b/tests/system/providers/google/cloud/gcs/example_mssql_to_gcs.py
@@ -19,9 +19,16 @@ from __future__ import annotations
 import os
 from datetime import datetime
 
+import pytest
+
 from airflow import models
 from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
-from airflow.providers.google.cloud.transfers.mssql_to_gcs import MSSQLToGCSOperator
+
+try:
+    from airflow.providers.google.cloud.transfers.mssql_to_gcs import MSSQLToGCSOperator
+except ImportError:
+    pytest.skip("MSSQL not available", allow_module_level=True)
+
 from airflow.utils.trigger_rule import TriggerRule
 
 ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
diff --git a/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py b/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py
index 55b4dd1b57..97404f1a8a 100644
--- a/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py
+++ b/tests/system/providers/google/cloud/gcs/example_mysql_to_gcs.py
@@ -19,9 +19,16 @@ from __future__ import annotations
 import os
 from datetime import datetime
 
+import pytest
+
 from airflow import models
 from airflow.providers.google.cloud.operators.gcs import GCSCreateBucketOperator, GCSDeleteBucketOperator
-from airflow.providers.google.cloud.transfers.mysql_to_gcs import MySQLToGCSOperator
+
+try:
+    from airflow.providers.google.cloud.transfers.mysql_to_gcs import MySQLToGCSOperator
+except ImportError:
+    pytest.skip("MySQL not available", allow_module_level=True)
+
 from airflow.utils.trigger_rule import TriggerRule
 
 ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
diff --git a/tests/system/providers/google/leveldb/example_leveldb.py b/tests/system/providers/google/leveldb/example_leveldb.py
index e199ccbe7b..2662c38940 100644
--- a/tests/system/providers/google/leveldb/example_leveldb.py
+++ b/tests/system/providers/google/leveldb/example_leveldb.py
@@ -23,8 +23,16 @@ from __future__ import annotations
 import os
 from datetime import datetime
 
+import pytest
+
 from airflow import models
-from airflow.providers.google.leveldb.operators.leveldb import LevelDBOperator
+from airflow.exceptions import AirflowOptionalProviderFeatureException
+
+try:
+    from airflow.providers.google.leveldb.operators.leveldb import LevelDBOperator
+except AirflowOptionalProviderFeatureException:
+    pytest.skip("LevelDB not available", allow_module_level=True)
+
 from airflow.utils.trigger_rule import TriggerRule
 
 ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
diff --git a/tests/system/providers/microsoft/azure/example_azure_service_bus.py b/tests/system/providers/microsoft/azure/example_azure_service_bus.py
index a3c51bd2dc..7c4a99786d 100644
--- a/tests/system/providers/microsoft/azure/example_azure_service_bus.py
+++ b/tests/system/providers/microsoft/azure/example_azure_service_bus.py
@@ -19,20 +19,26 @@ from __future__ import annotations
 import os
 from datetime import datetime, timedelta
 
+import pytest
+
 from airflow import DAG
 from airflow.models.baseoperator import chain
-from airflow.providers.microsoft.azure.operators.asb import (
-    ASBReceiveSubscriptionMessageOperator,
-    AzureServiceBusCreateQueueOperator,
-    AzureServiceBusDeleteQueueOperator,
-    AzureServiceBusReceiveMessageOperator,
-    AzureServiceBusSendMessageOperator,
-    AzureServiceBusSubscriptionCreateOperator,
-    AzureServiceBusSubscriptionDeleteOperator,
-    AzureServiceBusTopicCreateOperator,
-    AzureServiceBusTopicDeleteOperator,
-    AzureServiceBusUpdateSubscriptionOperator,
-)
+
+try:
+    from airflow.providers.microsoft.azure.operators.asb import (
+        ASBReceiveSubscriptionMessageOperator,
+        AzureServiceBusCreateQueueOperator,
+        AzureServiceBusDeleteQueueOperator,
+        AzureServiceBusReceiveMessageOperator,
+        AzureServiceBusSendMessageOperator,
+        AzureServiceBusSubscriptionCreateOperator,
+        AzureServiceBusSubscriptionDeleteOperator,
+        AzureServiceBusTopicCreateOperator,
+        AzureServiceBusTopicDeleteOperator,
+        AzureServiceBusUpdateSubscriptionOperator,
+    )
+except ImportError:
+    pytest.skip("Azure Service Bus not available", allow_module_level=True)
 
 EXECUTION_TIMEOUT = int(os.getenv("EXECUTION_TIMEOUT", 6))
 
diff --git a/tests/system/providers/microsoft/mssql/example_mssql.py b/tests/system/providers/microsoft/mssql/example_mssql.py
index 4b59da30ed..8d7fad9f25 100644
--- a/tests/system/providers/microsoft/mssql/example_mssql.py
+++ b/tests/system/providers/microsoft/mssql/example_mssql.py
@@ -24,9 +24,15 @@ from __future__ import annotations
 import os
 from datetime import datetime
 
+import pytest
+
 from airflow import DAG
-from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
-from airflow.providers.microsoft.mssql.operators.mssql import MsSqlOperator
+
+try:
+    from airflow.providers.microsoft.mssql.hooks.mssql import MsSqlHook
+    from airflow.providers.microsoft.mssql.operators.mssql import MsSqlOperator
+except ImportError:
+    pytest.skip("MSSQL provider not available", allow_module_level=True)
 
 ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
 DAG_ID = "example_mssql"