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 2022/04/29 07:21:00 UTC

[GitHub] [airflow] bhirsz opened a new pull request, #23351: Tests for provider code structure

bhirsz opened a new pull request, #23351:
URL: https://github.com/apache/airflow/pull/23351

   We had test class in test_project_structure.py that checked coverage of system tests/examples (and other misc stuff in provider code structure). Unfortunately there were some bugs (not all operators were found, some tests always passed, the test class worked only for Google).
   
   I have refactored the tests in the test_project_structure.py so it could be reused by other providers. For Google we're now using two test class bases:
   ```
   ExampleCoverageTest
   AssetsCoverageTest
   ```
   
   See the docker and elasticsearch providers test class (that uses ExampleCoverageTest base) to see how those test bases can be used or optionally extended with custom behaviour.


-- 
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 diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r867532755


##########
tests/always/test_project_structure.py:
##########
@@ -135,34 +135,143 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+def print_sorted(container: Set, indent: str = "    ") -> None:
+    sorted_container = sorted(container)

Review Comment:
   Indeed. Sorting is important for predictability!



-- 
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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r862318784


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache

Review Comment:
   Right, I copied that part of the code fron previous implementation - I will update 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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r862319175


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"
+        )
+        assert set() == covered_but_omitted, "Operator listed in missing examples but is used in example dag"
+
+
+class AssetsCoverageTest(ProjectStructureTest):
+    """Checks that every operator have operator_extra_links attribute"""
+
+    # These operators should not have assets
+    ASSETS_NOT_REQUIRED: Set = set()
+
+    def test_missing_assets_for_operator(self):
+        all_operators = self.list_of_operators()
+        assets, no_assets = set(), set()
+        for name, operator in all_operators.items():
+            for attr in operator.body:
+                if (
+                    isinstance(attr, ast.Assign)
+                    and attr.targets
+                    and getattr(attr.targets[0], "id", "") == "operator_extra_links"
+                ):
+                    assets.add(name)
+                    break
+            else:
+                no_assets.add(name)
+
+        asset_should_be_missing = self.ASSETS_NOT_REQUIRED - no_assets
+        no_assets -= self.ASSETS_NOT_REQUIRED
+        # TODO: (bhirsz): uncomment when we reach full coverage
+        # assert set() == no_assets, "Operator is missing assets"

Review Comment:
   That's actually my mistake - MISSING_ASSETS_FOR_OPERATORS should be named ASSETS_NOT_REQUIRED in Google test class. I can additionaly create MISSING_ASSETS_FOR_OPERATORS and do it the same way it's done for example coverage - I was hesistant since we have hundreds of those in Google :D Though I can update the check there and then override test method in Google with expected fail decorator and call it through the super statement. 



-- 
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] ferruzzi commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863226630


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"

Review Comment:
   Nitpick:  They were actively removing the term "dummy" under the inclusive language campaign, please consider a different default value.  Maybe "none" or "blank" or an empty string.
   
   For example,. "DummyOperator" got renamed to "EmptyOperator"



##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"

Review Comment:
   Nitpick:  They were actively removing the term "dummy" under the inclusive language campaign, please consider a different default value.  Maybe "none" or "blank" or an empty string.
   
   For example, "DummyOperator" got renamed to "EmptyOperator"



-- 
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 #23351: Tests for provider code structure

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


-- 
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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r862319421


##########
tests/always/test_project_structure.py:
##########
@@ -213,76 +299,110 @@ class TestGoogleProviderProjectStructure(unittest.TestCase):
         'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPListDLPJobsOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.transfers.cassandra_to_gcs.CassandraToGCSOperator',
+        'airflow.providers.google.cloud.transfers.adls_to_gcs.ADLSToGCSOperator',
+        'airflow.providers.google.cloud.transfers.bigquery_to_mysql.BigQueryToMySqlOperator',
+        'airflow.providers.google.cloud.transfers.sql_to_gcs.BaseSQLToGCSOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.GetEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.auto_ml.AutoMLTrainingJobBaseOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.UpdateEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.batch_prediction_job.'
+        'GetBatchPredictionJobOperator',
     }
 
-    def test_missing_example_for_operator(self):
-        """
-        Assert that all operators defined under operators, sensors and transfers directories
-        are used in any of the example dags
-        """
-        all_operators = set()
-        services = set()
-        for resource_type in ["operators", "sensors", "transfers"]:
-            operator_files = set(
-                self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-            )
-            for filepath in operator_files:
-                service_name = os.path.basename(filepath)[: -(len(".py"))]
-                if service_name in self.MISSING_EXAMPLE_DAGS:
-                    continue
-                services.add(service_name)
-                operators_paths = set(get_classes_from_file(f"{ROOT_FOLDER}/{filepath}"))
-                all_operators.update(operators_paths)
-
-        for service in services:
-            example_dags = self.examples_for_service(service)
-            example_paths = {
-                path for example_dag in example_dags for path in get_imports_from_file(example_dag)
-            }
-            all_operators -= example_paths
+    # These operators should not have assets
+    MISSING_ASSETS_FOR_OPERATORS = {
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryIntervalCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryValueCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery_dts.BigQueryDeleteDataTransferConfigOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.cloud_build.CloudBuildDeleteBuildTriggerOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.CloudMemorystoreDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.'
+        'CloudMemorystoreMemcachedDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLBaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceDatabaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceDeleteJobOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceGetOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceListOperationsOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServicePauseOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceResumeOperationOperator',
+        'airflow.providers.google.cloud.operators.compute.ComputeEngineBaseOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryGroupOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateFieldOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeletePipelineOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteBatchOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteBackupOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteServiceOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreBeginTransactionOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreDeleteOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreGetOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRollbackOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRunQueryOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDLPJobOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteInspectTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteJobTriggerOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteStoredInfoTypeOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPInspectContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPReidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.functions.CloudFunctionDeleteFunctionOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteBucketOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteObjectsOperator',
+        'airflow.providers.google.cloud.operators.kubernetes_engine.GKEDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteVersionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteSubscriptionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteTopicOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteDatabaseInstanceOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteAlertOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteNotificationChannelOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksQueueDeleteOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksTaskDeleteOperator',
+        'airflow.providers.google.cloud.operators.translate.CloudTranslateTextOperator',
+        'airflow.providers.google.cloud.operators.translate_speech.CloudTranslateSpeechOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductSetOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteReferenceImageOperator',
+        'airflow.providers.google.cloud.operators.workflows.WorkflowsDeleteWorkflowOperator',
+        'airflow.providers.google.marketing_platform.sensors.campaign_manager.'
+        'GoogleCampaignManagerReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360GetSDFDownloadOperationSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360ReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.search_ads.GoogleSearchAdsReportSensor',
+    }
 
-        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
-        all_operators -= self.DEPRECATED_OPERATORS
-        all_operators -= self.BASE_OPERATORS
-        assert set() == all_operators
-
-    @parameterized.expand(
-        itertools.product(["_system.py", "_system_helper.py"], ["operators", "sensors", "transfers"])
-    )
-    def test_detect_invalid_system_tests(self, resource_type, filename_suffix):
-        operators_tests = self.find_resource_files(top_level_directory="tests", resource_type=resource_type)
-        operators_files = self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-
-        files = {f for f in operators_tests if f.endswith(filename_suffix)}
-
-        expected_files = (f"tests/{f[8:]}" for f in operators_files)
-        expected_files = (f.replace(".py", filename_suffix).replace("/test_", "/") for f in expected_files)
-        expected_files = {f'{f.rpartition("/")[0]}/test_{f.rpartition("/")[2]}' for f in expected_files}
-
-        assert set() == files - expected_files
-
-    @staticmethod
-    def find_resource_files(
-        top_level_directory: str = "airflow",
-        department: str = "*",
-        resource_type: str = "*",
-        service: str = "*",
-    ):
-        python_files = glob.glob(
-            f"{ROOT_FOLDER}/{top_level_directory}/providers/google/{department}/{resource_type}/{service}.py"
-        )
-        # Make path relative
-        resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
-        # Exclude __init__.py and pycache
-        resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
-        return resource_files
 
-    @staticmethod
-    def examples_for_service(service_name):
-        yield from glob.glob(
-            f"{ROOT_FOLDER}/airflow/providers/google/*/example_dags/example_{service_name}*.py"
-        )
-        yield from glob.glob(f"{ROOT_FOLDER}/tests/system/providers/google/{service_name}/example_*.py")
+class TestElasticsearchProviderProjectStructure(ExampleCoverageTest):
+    PROVIDER = "elasticsearch"
+    OPERATOR_DIRS = {"hooks"}

Review Comment:
   I agree, I'm not satisfied with current naming (just took over the old naming). Same goes for method names such as list_of_operators etc. I will think about 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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863175568


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"

Review Comment:
   Ok, I will update it with the print. I was offline for a few days - I will update the PR tommorow with the changes from the other discussions. And thanks for the comments, it helps immensely to improve those 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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863810858


##########
tests/always/test_project_structure.py:
##########
@@ -213,76 +299,110 @@ class TestGoogleProviderProjectStructure(unittest.TestCase):
         'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPListDLPJobsOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.transfers.cassandra_to_gcs.CassandraToGCSOperator',
+        'airflow.providers.google.cloud.transfers.adls_to_gcs.ADLSToGCSOperator',
+        'airflow.providers.google.cloud.transfers.bigquery_to_mysql.BigQueryToMySqlOperator',
+        'airflow.providers.google.cloud.transfers.sql_to_gcs.BaseSQLToGCSOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.GetEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.auto_ml.AutoMLTrainingJobBaseOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.UpdateEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.batch_prediction_job.'
+        'GetBatchPredictionJobOperator',
     }
 
-    def test_missing_example_for_operator(self):
-        """
-        Assert that all operators defined under operators, sensors and transfers directories
-        are used in any of the example dags
-        """
-        all_operators = set()
-        services = set()
-        for resource_type in ["operators", "sensors", "transfers"]:
-            operator_files = set(
-                self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-            )
-            for filepath in operator_files:
-                service_name = os.path.basename(filepath)[: -(len(".py"))]
-                if service_name in self.MISSING_EXAMPLE_DAGS:
-                    continue
-                services.add(service_name)
-                operators_paths = set(get_classes_from_file(f"{ROOT_FOLDER}/{filepath}"))
-                all_operators.update(operators_paths)
-
-        for service in services:
-            example_dags = self.examples_for_service(service)
-            example_paths = {
-                path for example_dag in example_dags for path in get_imports_from_file(example_dag)
-            }
-            all_operators -= example_paths
+    # These operators should not have assets
+    MISSING_ASSETS_FOR_OPERATORS = {
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryIntervalCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryValueCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery_dts.BigQueryDeleteDataTransferConfigOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.cloud_build.CloudBuildDeleteBuildTriggerOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.CloudMemorystoreDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.'
+        'CloudMemorystoreMemcachedDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLBaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceDatabaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceDeleteJobOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceGetOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceListOperationsOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServicePauseOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceResumeOperationOperator',
+        'airflow.providers.google.cloud.operators.compute.ComputeEngineBaseOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryGroupOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateFieldOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeletePipelineOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteBatchOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteBackupOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteServiceOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreBeginTransactionOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreDeleteOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreGetOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRollbackOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRunQueryOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDLPJobOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteInspectTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteJobTriggerOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteStoredInfoTypeOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPInspectContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPReidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.functions.CloudFunctionDeleteFunctionOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteBucketOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteObjectsOperator',
+        'airflow.providers.google.cloud.operators.kubernetes_engine.GKEDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteVersionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteSubscriptionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteTopicOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteDatabaseInstanceOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteAlertOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteNotificationChannelOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksQueueDeleteOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksTaskDeleteOperator',
+        'airflow.providers.google.cloud.operators.translate.CloudTranslateTextOperator',
+        'airflow.providers.google.cloud.operators.translate_speech.CloudTranslateSpeechOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductSetOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteReferenceImageOperator',
+        'airflow.providers.google.cloud.operators.workflows.WorkflowsDeleteWorkflowOperator',
+        'airflow.providers.google.marketing_platform.sensors.campaign_manager.'
+        'GoogleCampaignManagerReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360GetSDFDownloadOperationSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360ReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.search_ads.GoogleSearchAdsReportSensor',
+    }
 
-        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
-        all_operators -= self.DEPRECATED_OPERATORS
-        all_operators -= self.BASE_OPERATORS
-        assert set() == all_operators
-
-    @parameterized.expand(
-        itertools.product(["_system.py", "_system_helper.py"], ["operators", "sensors", "transfers"])
-    )
-    def test_detect_invalid_system_tests(self, resource_type, filename_suffix):
-        operators_tests = self.find_resource_files(top_level_directory="tests", resource_type=resource_type)
-        operators_files = self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-
-        files = {f for f in operators_tests if f.endswith(filename_suffix)}
-
-        expected_files = (f"tests/{f[8:]}" for f in operators_files)
-        expected_files = (f.replace(".py", filename_suffix).replace("/test_", "/") for f in expected_files)
-        expected_files = {f'{f.rpartition("/")[0]}/test_{f.rpartition("/")[2]}' for f in expected_files}
-
-        assert set() == files - expected_files
-
-    @staticmethod
-    def find_resource_files(
-        top_level_directory: str = "airflow",
-        department: str = "*",
-        resource_type: str = "*",
-        service: str = "*",
-    ):
-        python_files = glob.glob(
-            f"{ROOT_FOLDER}/{top_level_directory}/providers/google/{department}/{resource_type}/{service}.py"
-        )
-        # Make path relative
-        resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
-        # Exclude __init__.py and pycache
-        resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
-        return resource_files
 
-    @staticmethod
-    def examples_for_service(service_name):
-        yield from glob.glob(
-            f"{ROOT_FOLDER}/airflow/providers/google/*/example_dags/example_{service_name}*.py"
-        )
-        yield from glob.glob(f"{ROOT_FOLDER}/tests/system/providers/google/{service_name}/example_*.py")
+class TestElasticsearchProviderProjectStructure(ExampleCoverageTest):
+    PROVIDER = "elasticsearch"
+    OPERATOR_DIRS = {"hooks"}

Review Comment:
   Used "class" generic name. CLASS_DIRS, list_of_classes, class_paths, DEPRECATED_CLASSES, MISSING_EXAMPLES_FOR_CLASSES etc



-- 
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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863806713


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"
+        )
+        assert set() == covered_but_omitted, "Operator listed in missing examples but is used in example dag"
+
+
+class AssetsCoverageTest(ProjectStructureTest):
+    """Checks that every operator have operator_extra_links attribute"""
+
+    # These operators should not have assets
+    ASSETS_NOT_REQUIRED: Set = set()
+
+    def test_missing_assets_for_operator(self):
+        all_operators = self.list_of_operators()
+        assets, no_assets = set(), set()
+        for name, operator in all_operators.items():
+            for attr in operator.body:
+                if (
+                    isinstance(attr, ast.Assign)
+                    and attr.targets
+                    and getattr(attr.targets[0], "id", "") == "operator_extra_links"
+                ):
+                    assets.add(name)
+                    break
+            else:
+                no_assets.add(name)
+
+        asset_should_be_missing = self.ASSETS_NOT_REQUIRED - no_assets
+        no_assets -= self.ASSETS_NOT_REQUIRED
+        # TODO: (bhirsz): uncomment when we reach full coverage
+        # assert set() == no_assets, "Operator is missing assets"

Review Comment:
   I've added ``MISSING_ASSETS_FOR_OPERATORS`` so it's possible to define list of missing assets but it's also possible to temporarily expect test to fail:
   
   ```
       @pytest.mark.xfail(reason="We did not reach full coverage yet")
       def test_missing_assets_for_operator(self):
           super().test_missing_assets_for_operator()
   ```
   (example from the Google provider). When we will decrease number of missing assets I will override ``MISSING_ASSETS_FOR_OPERATORS`` with missing assets and expect test to pass.



-- 
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] ferruzzi commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
ferruzzi commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863226630


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"

Review Comment:
   Nitpick:  They were actively removing the term "dummy" under the inclusive language campaign, please consider a different default value.  Maybe "none" or "blank" or an empty string.



-- 
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] o-nikolas commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863208495


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"

Review Comment:
   > And thanks for the comments, it helps immensely to improve those tests :)
   
   And thank you for this PR! We've been doing a project to update all the docs and example dags for AWS operators, so something like this will help a lot to ensure there are no future regressions :smile: 



-- 
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 #23351: Tests for provider code structure

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

   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] o-nikolas commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r862251923


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache

Review Comment:
   I only see the former being excluded



##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"
+        )
+        assert set() == covered_but_omitted, "Operator listed in missing examples but is used in example dag"
+
+
+class AssetsCoverageTest(ProjectStructureTest):
+    """Checks that every operator have operator_extra_links attribute"""
+
+    # These operators should not have assets
+    ASSETS_NOT_REQUIRED: Set = set()
+
+    def test_missing_assets_for_operator(self):
+        all_operators = self.list_of_operators()
+        assets, no_assets = set(), set()
+        for name, operator in all_operators.items():
+            for attr in operator.body:
+                if (
+                    isinstance(attr, ast.Assign)
+                    and attr.targets
+                    and getattr(attr.targets[0], "id", "") == "operator_extra_links"
+                ):
+                    assets.add(name)
+                    break
+            else:
+                no_assets.add(name)
+
+        asset_should_be_missing = self.ASSETS_NOT_REQUIRED - no_assets
+        no_assets -= self.ASSETS_NOT_REQUIRED
+        # TODO: (bhirsz): uncomment when we reach full coverage
+        # assert set() == no_assets, "Operator is missing assets"

Review Comment:
   Why not add a field akin to `MISSING_EXAMPLES_FOR_OPERATORS` so that this can be uncommitted while the work is being done to patch up the operators that are failing?
   
   In fact, it looks like you're actually setting `MISSING_ASSETS_FOR_OPERATORS` below in the `TestGoogleProviderProjectStructure` class, but not using it here.



##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"

Review Comment:
   Should we log the operators that are not covered here?



##########
tests/always/test_project_structure.py:
##########
@@ -213,76 +299,110 @@ class TestGoogleProviderProjectStructure(unittest.TestCase):
         'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPListDLPJobsOperator',
         'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.transfers.cassandra_to_gcs.CassandraToGCSOperator',
+        'airflow.providers.google.cloud.transfers.adls_to_gcs.ADLSToGCSOperator',
+        'airflow.providers.google.cloud.transfers.bigquery_to_mysql.BigQueryToMySqlOperator',
+        'airflow.providers.google.cloud.transfers.sql_to_gcs.BaseSQLToGCSOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.GetEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.auto_ml.AutoMLTrainingJobBaseOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.endpoint_service.UpdateEndpointOperator',
+        'airflow.providers.google.cloud.operators.vertex_ai.batch_prediction_job.'
+        'GetBatchPredictionJobOperator',
     }
 
-    def test_missing_example_for_operator(self):
-        """
-        Assert that all operators defined under operators, sensors and transfers directories
-        are used in any of the example dags
-        """
-        all_operators = set()
-        services = set()
-        for resource_type in ["operators", "sensors", "transfers"]:
-            operator_files = set(
-                self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-            )
-            for filepath in operator_files:
-                service_name = os.path.basename(filepath)[: -(len(".py"))]
-                if service_name in self.MISSING_EXAMPLE_DAGS:
-                    continue
-                services.add(service_name)
-                operators_paths = set(get_classes_from_file(f"{ROOT_FOLDER}/{filepath}"))
-                all_operators.update(operators_paths)
-
-        for service in services:
-            example_dags = self.examples_for_service(service)
-            example_paths = {
-                path for example_dag in example_dags for path in get_imports_from_file(example_dag)
-            }
-            all_operators -= example_paths
+    # These operators should not have assets
+    MISSING_ASSETS_FOR_OPERATORS = {
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.automl.AutoMLDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteDatasetOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryIntervalCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery.BigQueryValueCheckOperator',
+        'airflow.providers.google.cloud.operators.bigquery_dts.BigQueryDeleteDataTransferConfigOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.bigtable.BigtableDeleteTableOperator',
+        'airflow.providers.google.cloud.operators.cloud_build.CloudBuildDeleteBuildTriggerOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.CloudMemorystoreDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_memorystore.'
+        'CloudMemorystoreMemcachedDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLBaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceDatabaseOperator',
+        'airflow.providers.google.cloud.operators.cloud_sql.CloudSQLDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceDeleteJobOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceGetOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceListOperationsOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServicePauseOperationOperator',
+        'airflow.providers.google.cloud.operators.cloud_storage_transfer_service.'
+        'CloudDataTransferServiceResumeOperationOperator',
+        'airflow.providers.google.cloud.operators.compute.ComputeEngineBaseOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryGroupOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteEntryOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateFieldOperator',
+        'airflow.providers.google.cloud.operators.datacatalog.CloudDataCatalogDeleteTagTemplateOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.datafusion.CloudDataFusionDeletePipelineOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteBatchOperator',
+        'airflow.providers.google.cloud.operators.dataproc.DataprocDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteBackupOperator',
+        'airflow.providers.google.cloud.operators.dataproc_metastore.DataprocMetastoreDeleteServiceOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreBeginTransactionOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreDeleteOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreGetOperationOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRollbackOperator',
+        'airflow.providers.google.cloud.operators.datastore.CloudDatastoreRunQueryOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDLPJobOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteDeidentifyTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteInspectTemplateOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteJobTriggerOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPDeleteStoredInfoTypeOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPInspectContentOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPRedactImageOperator',
+        'airflow.providers.google.cloud.operators.dlp.CloudDLPReidentifyContentOperator',
+        'airflow.providers.google.cloud.operators.functions.CloudFunctionDeleteFunctionOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteBucketOperator',
+        'airflow.providers.google.cloud.operators.gcs.GCSDeleteObjectsOperator',
+        'airflow.providers.google.cloud.operators.kubernetes_engine.GKEDeleteClusterOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteModelOperator',
+        'airflow.providers.google.cloud.operators.mlengine.MLEngineDeleteVersionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteSubscriptionOperator',
+        'airflow.providers.google.cloud.operators.pubsub.PubSubDeleteTopicOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteDatabaseInstanceOperator',
+        'airflow.providers.google.cloud.operators.spanner.SpannerDeleteInstanceOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteAlertOperator',
+        'airflow.providers.google.cloud.operators.stackdriver.StackdriverDeleteNotificationChannelOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksQueueDeleteOperator',
+        'airflow.providers.google.cloud.operators.tasks.CloudTasksTaskDeleteOperator',
+        'airflow.providers.google.cloud.operators.translate.CloudTranslateTextOperator',
+        'airflow.providers.google.cloud.operators.translate_speech.CloudTranslateSpeechOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteProductSetOperator',
+        'airflow.providers.google.cloud.operators.vision.CloudVisionDeleteReferenceImageOperator',
+        'airflow.providers.google.cloud.operators.workflows.WorkflowsDeleteWorkflowOperator',
+        'airflow.providers.google.marketing_platform.sensors.campaign_manager.'
+        'GoogleCampaignManagerReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360GetSDFDownloadOperationSensor',
+        'airflow.providers.google.marketing_platform.sensors.display_video.'
+        'GoogleDisplayVideo360ReportSensor',
+        'airflow.providers.google.marketing_platform.sensors.search_ads.GoogleSearchAdsReportSensor',
+    }
 
-        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
-        all_operators -= self.DEPRECATED_OPERATORS
-        all_operators -= self.BASE_OPERATORS
-        assert set() == all_operators
-
-    @parameterized.expand(
-        itertools.product(["_system.py", "_system_helper.py"], ["operators", "sensors", "transfers"])
-    )
-    def test_detect_invalid_system_tests(self, resource_type, filename_suffix):
-        operators_tests = self.find_resource_files(top_level_directory="tests", resource_type=resource_type)
-        operators_files = self.find_resource_files(top_level_directory="airflow", resource_type=resource_type)
-
-        files = {f for f in operators_tests if f.endswith(filename_suffix)}
-
-        expected_files = (f"tests/{f[8:]}" for f in operators_files)
-        expected_files = (f.replace(".py", filename_suffix).replace("/test_", "/") for f in expected_files)
-        expected_files = {f'{f.rpartition("/")[0]}/test_{f.rpartition("/")[2]}' for f in expected_files}
-
-        assert set() == files - expected_files
-
-    @staticmethod
-    def find_resource_files(
-        top_level_directory: str = "airflow",
-        department: str = "*",
-        resource_type: str = "*",
-        service: str = "*",
-    ):
-        python_files = glob.glob(
-            f"{ROOT_FOLDER}/{top_level_directory}/providers/google/{department}/{resource_type}/{service}.py"
-        )
-        # Make path relative
-        resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
-        # Exclude __init__.py and pycache
-        resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
-        return resource_files
 
-    @staticmethod
-    def examples_for_service(service_name):
-        yield from glob.glob(
-            f"{ROOT_FOLDER}/airflow/providers/google/*/example_dags/example_{service_name}*.py"
-        )
-        yield from glob.glob(f"{ROOT_FOLDER}/tests/system/providers/google/{service_name}/example_*.py")
+class TestElasticsearchProviderProjectStructure(ExampleCoverageTest):
+    PROVIDER = "elasticsearch"
+    OPERATOR_DIRS = {"hooks"}

Review Comment:
   The verbiage of the new code above is very Operator-centric, but this provider already breaks that naming paradigm (as well as Sensors, and Transfers). I wonder if something more generic makes sense? Maybe just `DIRS`? And similar changes to genericize the new classes above.  



-- 
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] bhirsz commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
bhirsz commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r862318865


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"

Review Comment:
   If the assertion fails, pytest will print the difference between sets - essentially what wer'e missing



-- 
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] o-nikolas commented on a diff in pull request #23351: Tests for provider code structure

Posted by GitBox <gi...@apache.org>.
o-nikolas commented on code in PR #23351:
URL: https://github.com/apache/airflow/pull/23351#discussion_r863170221


##########
tests/always/test_project_structure.py:
##########
@@ -135,33 +133,123 @@ def filepath_to_module(filepath: str):
     return filepath.replace("/", ".")[: -(len('.py'))]
 
 
-def get_classes_from_file(filepath: str):
-    with open(filepath) as py_file:
-        content = py_file.read()
-    doc_node = ast.parse(content, filepath)
-    module = filepath_to_module(filepath)
-    results: List[str] = []
-    for current_node in ast.walk(doc_node):
-        if not isinstance(current_node, ast.ClassDef):
-            continue
-        name = current_node.name
-        if not name.endswith("Operator") and not name.endswith("Sensor") and not name.endswith("Operator"):
-            continue
-        results.append(f"{module}.{name}")
-    return results
-
-
-class TestGoogleProviderProjectStructure(unittest.TestCase):
-    MISSING_EXAMPLE_DAGS = {
-        'adls_to_gcs',
-        'sql_to_gcs',
-        'bigquery_to_mysql',
-        'cassandra_to_gcs',
-        'drive',
-        'ads_to_gcs',
-    }
+class ProjectStructureTest:
+    PROVIDER = "dummy"
+    OPERATOR_DIRS = {"operators", "sensors", "transfers"}
+    CLASS_SUFFIXES = ["Operator", "Sensor"]
+
+    def operator_paths(self):
+        """Override this method if your operators are located under different paths"""
+        for resource_type in self.OPERATOR_DIRS:
+            python_files = glob.glob(
+                f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/{resource_type}/**.py", recursive=True
+            )
+            # Make path relative
+            resource_files = (os.path.relpath(f, ROOT_FOLDER) for f in python_files)
+            # Exclude __init__.py and pycache
+            resource_files = (f for f in resource_files if not f.endswith("__init__.py"))
+            yield from resource_files
+
+    def list_of_operators(self):
+        all_operators = {}
+        for operator_file in self.operator_paths():
+            operators_paths = self.get_classes_from_file(f"{ROOT_FOLDER}/{operator_file}")
+            all_operators.update(operators_paths)
+        return all_operators
+
+    def get_classes_from_file(self, filepath: str):
+        with open(filepath) as py_file:
+            content = py_file.read()
+        doc_node = ast.parse(content, filepath)
+        module = filepath_to_module(filepath)
+        results: Dict = {}
+        for current_node in ast.walk(doc_node):
+            if not isinstance(current_node, ast.ClassDef):
+                continue
+            name = current_node.name
+            if not any(name.endswith(suffix) for suffix in self.CLASS_SUFFIXES):
+                continue
+            results[f"{module}.{name}"] = current_node
+        return results
+
+
+class ExampleCoverageTest(ProjectStructureTest):
+    """Checks that every operator is covered by example"""
+
+    # Those operators are deprecated, so we do not need examples for them
+    DEPRECATED_OPERATORS: Set = set()
+
+    # Those operators should not have examples as they are never used standalone (they are abstract)
+    BASE_OPERATORS: Set = set()
+
+    # Please add the examples to those operators at the earliest convenience :)
+    MISSING_EXAMPLES_FOR_OPERATORS: Set = set()
+
+    def example_paths(self):
+        """Override this method if your example dags are located elsewhere"""
+        # old_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/airflow/providers/{self.PROVIDER}/**/example_dags/example_*.py", recursive=True
+        )
+        # new_design:
+        yield from glob.glob(
+            f"{ROOT_FOLDER}/tests/system/providers/{self.PROVIDER}/**/example_*.py", recursive=True
+        )
+
+    def test_missing_example_for_operator(self):
+        """
+        Assert that all operators defined under operators, sensors and transfers directories
+        are used in any of the example dags
+        """
+        all_operators = self.list_of_operators()
+        assert 0 != len(all_operators), "Failed to retrieve operators, override operator_paths if needed"
+        all_operators = set(all_operators.keys())
+        for example in self.example_paths():
+            all_operators -= get_imports_from_file(example)
+
+        covered_but_omitted = self.MISSING_EXAMPLES_FOR_OPERATORS - all_operators
+        all_operators -= self.MISSING_EXAMPLES_FOR_OPERATORS
+        all_operators -= self.DEPRECATED_OPERATORS
+        all_operators -= self.BASE_OPERATORS
+        assert set() == all_operators, (
+            "Not all operators are covered with example dags. "
+            "Update self.MISSING_EXAMPLES_FOR_OPERATORS if you want to skip this error"

Review Comment:
   Yeah, I pulled the CR code and ran it for AWS (I plan to put in a CR for that after this merges) and the set diff shown is really hard to read, especially when the diff is large. There are no new lines and you have to review carefully to be sure you're looking at the right side of the set subtraction. I think a pretty-formatted message that prints exactly the items which are missing would be really helpful for users.



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