From d4437deaf0789ec6de2cd8d6185aa56d4ff3e5d8 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 11 Sep 2023 13:28:04 -0700 Subject: [PATCH 01/60] fix(airflow): fix provider loading exception Would previously see this from the plugin manager. ``` Exception: The package 'acryl-datahub' from setuptools and acryl-datahub-airflow-plugin do not match. Please make sure they are aligned ``` Also fixes + registers hook types --- .../airflow-plugin/setup.py | 3 ++- .../src/datahub_airflow_plugin/__init__.py | 16 +++++++++++++ .../datahub_airflow_plugin/hooks/datahub.py | 23 +++++++++++++++---- metadata-ingestion/setup.py | 1 - 4 files changed, 37 insertions(+), 6 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 18e605ae76ebd..47069f59c314d 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -80,7 +80,8 @@ def get_long_description(): entry_points = { - "airflow.plugins": "acryl-datahub-airflow-plugin = datahub_airflow_plugin.datahub_plugin:DatahubPlugin" + "airflow.plugins": "acryl-datahub-airflow-plugin = datahub_airflow_plugin.datahub_plugin:DatahubPlugin", + "apache_airflow_provider": ["provider_info=datahub_provider:get_provider_info"], } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/__init__.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/__init__.py index b2c45d3a1e75d..e4040e3a17dfd 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/__init__.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/__init__.py @@ -18,4 +18,20 @@ def get_provider_info(): "package-name": f"{__package_name__}", "name": f"{__package_name__}", "description": "Datahub metadata collector plugin", + "connection-types": [ + { + "hook-class-name": "datahub_airflow_plugin.hooks.datahub.DatahubRestHook", + "connection-type": "datahub-rest", + }, + { + "hook-class-name": "datahub_airflow_plugin.hooks.datahub.DatahubKafkaHook", + "connection-type": "datahub-kafka", + }, + ], + # Deprecated method of providing connection types, kept for backwards compatibility. + # We can remove with Airflow 3. + "hook-class-names": [ + "datahub_airflow_plugin.hooks.datahub.DatahubRestHook", + "datahub_airflow_plugin.hooks.datahub.DatahubKafkaHook", + ], } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index aed858c6c4df0..8fb7363f8cad1 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -29,7 +29,7 @@ class DatahubRestHook(BaseHook): conn_name_attr = "datahub_rest_conn_id" default_conn_name = "datahub_rest_default" - conn_type = "datahub_rest" + conn_type = "datahub-rest" hook_name = "DataHub REST Server" def __init__(self, datahub_rest_conn_id: str = default_conn_name) -> None: @@ -50,6 +50,15 @@ def get_ui_field_behaviour() -> Dict: }, } + def test_connection(self) -> Tuple[bool, str]: + try: + emitter = self.make_emitter() + emitter.test_connection() + except Exception as e: + return False, str(e) + + return True, "Successfully connected to DataHub." + def _get_config(self) -> Tuple[str, Optional[str], Optional[int]]: conn: "Connection" = self.get_connection(self.datahub_rest_conn_id) @@ -99,7 +108,7 @@ class DatahubKafkaHook(BaseHook): conn_name_attr = "datahub_kafka_conn_id" default_conn_name = "datahub_kafka_default" - conn_type = "datahub_kafka" + conn_type = "datahub-kafka" hook_name = "DataHub Kafka Sink" def __init__(self, datahub_kafka_conn_id: str = default_conn_name) -> None: @@ -194,9 +203,15 @@ def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: # We need to figure out the underlying hook type. First check the # conn_type. If that fails, attempt to guess using the conn id name. - if conn.conn_type == DatahubRestHook.conn_type: + if ( + conn.conn_type == DatahubRestHook.conn_type + or conn.conn_type == DatahubRestHook.conn_type.replace("-", "_") + ): return DatahubRestHook(self.datahub_conn_id) - elif conn.conn_type == DatahubKafkaHook.conn_type: + elif ( + conn.conn_type == DatahubKafkaHook.conn_type + or conn.conn_type == DatahubKafkaHook.conn_type.replace("-", "_") + ): return DatahubKafkaHook(self.datahub_conn_id) elif "rest" in self.datahub_conn_id: return DatahubRestHook(self.datahub_conn_id) diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index a119eba25be2a..b9169186174fa 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -643,7 +643,6 @@ def get_long_description(): "datahub = datahub.ingestion.reporting.datahub_ingestion_run_summary_provider:DatahubIngestionRunSummaryProvider", "file = datahub.ingestion.reporting.file_reporter:FileReporter", ], - "apache_airflow_provider": ["provider_info=datahub_provider:get_provider_info"], } From 72378e1617798211e1f69cb7494ae358586d5f18 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 11 Sep 2023 22:50:42 -0700 Subject: [PATCH 02/60] start refactoring backend continue making tweaks refactor ti + config refactor get_lineage_config refactoring add in special case handling from backport listener-based plugin fix hooks + add log_level config make logging work more sql fixes start working on combining lineage work with old datahub lineage entities support fine grained lineage support enable_extractors tweak stuff preserve datajob info fix bugs in emitter support threaded exec fix rendering refactoring more refactoring tweak comment review comments mypy fix mypy errors drop python 3.7 fix lint fix lint --- .github/workflows/airflow-plugin.yml | 4 +- .../airflow-plugin/setup.py | 3 +- .../datahub_airflow_plugin/_airflow_shims.py | 30 ++ .../src/datahub_airflow_plugin/_config.py | 80 ++++ .../_datahub_ol_adapter.py | 23 + .../src/datahub_airflow_plugin/_extractors.py | 180 ++++++++ .../client/airflow_generator.py | 15 +- .../datahub_listener.py | 434 ++++++++++++++++++ .../datahub_airflow_plugin/datahub_plugin.py | 364 +-------------- .../datahub_plugin_v22.py | 322 +++++++++++++ .../{ => lineage}/_lineage_core.py | 30 +- .../datahub_airflow_plugin/lineage/datahub.py | 28 +- .../datahub/api/entities/datajob/datajob.py | 30 +- .../src/datahub/emitter/rest_emitter.py | 10 +- .../src/datahub/ingestion/graph/client.py | 17 + .../src/datahub/ingestion/source/tableau.py | 11 +- .../src/datahub/utilities/sqlglot_lineage.py | 17 +- 17 files changed, 1153 insertions(+), 445 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_config.py create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_ol_adapter.py create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py rename metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/{ => lineage}/_lineage_core.py (72%) diff --git a/.github/workflows/airflow-plugin.yml b/.github/workflows/airflow-plugin.yml index 63bab821cc398..92097911fb067 100644 --- a/.github/workflows/airflow-plugin.yml +++ b/.github/workflows/airflow-plugin.yml @@ -32,9 +32,9 @@ jobs: strategy: matrix: include: - - python-version: "3.7" + - python-version: "3.8" extraPythonRequirement: "apache-airflow~=2.1.0" - - python-version: "3.7" + - python-version: "3.8" extraPythonRequirement: "apache-airflow~=2.2.0" - python-version: "3.10" extraPythonRequirement: "apache-airflow~=2.4.0" diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 47069f59c314d..4ac18b525675b 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -27,7 +27,8 @@ def get_long_description(): "pydantic>=1.5.1", "apache-airflow >= 2.0.2", *rest_common, - f"acryl-datahub == {package_metadata['__version__']}", + f"acryl-datahub[sql-parser] == {package_metadata['__version__']}", + "openlineage-airflow==1.2.0", } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py index 5ad20e1f72551..e2e85d6f7eb6e 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py @@ -1,7 +1,15 @@ +from typing import List + +import airflow.version +import packaging.version from airflow.models.baseoperator import BaseOperator from datahub_airflow_plugin._airflow_compat import AIRFLOW_PATCHED +AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) + +IS_AIRFLOW_V23_PLUS = AIRFLOW_VERSION >= packaging.version.parse("2.3.0.dev0") + try: from airflow.models.mappedoperator import MappedOperator from airflow.models.operator import Operator @@ -21,7 +29,29 @@ assert AIRFLOW_PATCHED + +def get_task_inlets(operator: "Operator") -> List: + # From Airflow 2.4 _inlets is dropped and inlets used consistently. Earlier it was not the case, so we have to stick there to _inlets + if hasattr(operator, "_inlets"): + return operator._inlets # type: ignore[attr-defined, union-attr] + if hasattr(operator, "get_inlet_defs"): + return operator.get_inlet_defs() # type: ignore[attr-defined] + return operator.inlets + + +def get_task_outlets(operator: "Operator") -> List: + # From Airflow 2.4 _outlets is dropped and inlets used consistently. Earlier it was not the case, so we have to stick there to _outlets + # We have to use _outlets because outlets is empty in Airflow < 2.4.0 + if hasattr(operator, "_outlets"): + return operator._outlets # type: ignore[attr-defined, union-attr] + if hasattr(operator, "get_outlet_defs"): + return operator.get_outlet_defs() + return operator.outlets + + __all__ = [ + "AIRFLOW_VERSION", + "IS_AIRFLOW_V23_PLUS", "Operator", "MappedOperator", "EmptyOperator", diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_config.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_config.py new file mode 100644 index 0000000000000..67843da2ba995 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_config.py @@ -0,0 +1,80 @@ +from typing import TYPE_CHECKING, Optional + +import datahub.emitter.mce_builder as builder +from airflow.configuration import conf +from datahub.configuration.common import ConfigModel + +if TYPE_CHECKING: + from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook + + +class DatahubLineageConfig(ConfigModel): + # This class is shared between the lineage backend and the Airflow plugin. + # The defaults listed here are only relevant for the lineage backend. + # The Airflow plugin's default values come from the fallback values in + # the get_lineage_config() function below. + + enabled: bool = True + + # DataHub hook connection ID. + datahub_conn_id: str + + # Cluster to associate with the pipelines and tasks. Defaults to "prod". + cluster: str = builder.DEFAULT_FLOW_CLUSTER + + # If true, the owners field of the DAG will be capture as a DataHub corpuser. + capture_ownership_info: bool = True + + # If true, the tags field of the DAG will be captured as DataHub tags. + capture_tags_info: bool = True + + capture_executions: bool = False + + enable_extractors: bool = True + + log_level: Optional[str] = None + debug_emitter: bool = False + + disable_openlineage_plugin: bool = True + + # Note that this field is only respected by the lineage backend. + # The Airflow plugin behaves as if it were set to True. + graceful_exceptions: bool = True + + def make_emitter_hook(self) -> "DatahubGenericHook": + # This is necessary to avoid issues with circular imports. + from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook + + return DatahubGenericHook(self.datahub_conn_id) + + +def get_lineage_config() -> DatahubLineageConfig: + """Load the DataHub plugin config from airflow.cfg.""" + + enabled = conf.get("datahub", "enabled", fallback=True) + datahub_conn_id = conf.get("datahub", "conn_id", fallback="datahub_rest_default") + cluster = conf.get("datahub", "cluster", fallback=builder.DEFAULT_FLOW_CLUSTER) + capture_tags_info = conf.get("datahub", "capture_tags_info", fallback=True) + capture_ownership_info = conf.get( + "datahub", "capture_ownership_info", fallback=True + ) + capture_executions = conf.get("datahub", "capture_executions", fallback=True) + enable_extractors = conf.get("datahub", "enable_extractors", fallback=True) + log_level = conf.get("datahub", "log_level", fallback=None) + debug_emitter = conf.get("datahub", "debug_emitter", fallback=False) + disable_openlineage_plugin = conf.get( + "datahub", "disable_openlineage_plugin", fallback=True + ) + + return DatahubLineageConfig( + enabled=enabled, + datahub_conn_id=datahub_conn_id, + cluster=cluster, + capture_ownership_info=capture_ownership_info, + capture_tags_info=capture_tags_info, + capture_executions=capture_executions, + enable_extractors=enable_extractors, + log_level=log_level, + debug_emitter=debug_emitter, + disable_openlineage_plugin=disable_openlineage_plugin, + ) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_ol_adapter.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_ol_adapter.py new file mode 100644 index 0000000000000..7d35791bf1db4 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_ol_adapter.py @@ -0,0 +1,23 @@ +import logging + +import datahub.emitter.mce_builder as builder +from openlineage.client.run import Dataset as OpenLineageDataset + +logger = logging.getLogger(__name__) + + +OL_SCHEME_TWEAKS = { + "sqlserver": "mssql", + "trino": "presto", + "awsathena": "athena", +} + + +def translate_ol_to_datahub_urn(ol_uri: OpenLineageDataset) -> str: + namespace = ol_uri.namespace + name = ol_uri.name + + scheme, *rest = namespace.split("://", maxsplit=1) + + platform = OL_SCHEME_TWEAKS.get(scheme, scheme) + return builder.make_dataset_urn(platform=platform, name=name) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py new file mode 100644 index 0000000000000..40758ad1909e1 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py @@ -0,0 +1,180 @@ +import contextlib +import logging +import unittest.mock +from typing import TYPE_CHECKING, Optional + +import datahub.emitter.mce_builder as builder +from datahub.utilities.sqlglot_lineage import ( + SqlParsingResult, + create_lineage_sql_parsed_result, +) +from openlineage.airflow.extractors import BaseExtractor +from openlineage.airflow.extractors import ExtractorManager as OLExtractorManager +from openlineage.airflow.extractors import TaskMetadata +from openlineage.airflow.extractors.snowflake_extractor import SnowflakeExtractor +from openlineage.airflow.extractors.sql_extractor import SqlExtractor +from openlineage.client.facet import ( + ExtractionError, + ExtractionErrorRunFacet, + SqlJobFacet, +) + +from datahub_airflow_plugin._airflow_shims import Operator +from datahub_airflow_plugin._datahub_ol_adapter import OL_SCHEME_TWEAKS + +if TYPE_CHECKING: + from airflow.models import DagRun, TaskInstance + from datahub.ingestion.graph.client import DataHubGraph + +logger = logging.getLogger(__name__) +_DATAHUB_GRAPH_CONTEXT_KEY = "datahub_graph" +SQL_PARSING_RESULT_KEY = "datahub_sql" + + +class ExtractorManager(OLExtractorManager): + # TODO: On Airflow 2.7, the OLExtractorManager is part of the built-in Airflow API. + # When available, we should use that instead. The same goe for most of the OL + # extractors. + + def __init__(self): + super().__init__() + + self._graph: Optional["DataHubGraph"] = None + + @contextlib.contextmanager + def _patch_extractors(self): + with contextlib.ExitStack() as stack: + # Patch the SqlExtractor.extract() method. + # TODO: Make this work for Airflow 2.7+. + stack.enter_context( + unittest.mock.patch.object( + SqlExtractor, + "extract", + _sql_extractor_extract, + ) + ) + + # Patch the SnowflakeExtractor.default_schema property. + stack.enter_context( + unittest.mock.patch.object( + SnowflakeExtractor, + "default_schema", + property(snowflake_default_schema), + ) + ) + + # TODO: Override the BigQuery extractor to use the DataHub SQL parser. + # self.extractor_manager.add_extractor() + + yield + + def extract_metadata( + self, + dagrun: "DagRun", + task: "Operator", + complete: bool = False, + task_instance: Optional["TaskInstance"] = None, + task_uuid: Optional[str] = None, + graph: Optional["DataHubGraph"] = None, + ) -> TaskMetadata: + self._graph = graph + with self._patch_extractors(): + return super().extract_metadata( + dagrun, task, complete, task_instance, task_uuid + ) + + def _get_extractor(self, task: "Operator") -> Optional[BaseExtractor]: + extractor = super()._get_extractor(task) + if extractor: + extractor.set_context(_DATAHUB_GRAPH_CONTEXT_KEY, self._graph) + return extractor + + +def _sql_extractor_extract(self: "SqlExtractor") -> TaskMetadata: + # Why not override the OL sql_parse method directly, instead of overriding + # extract()? A few reasons: + # + # 1. We would want to pass the default_db and graph instance into our sql parser + # method. The OL code doesn't pass the default_db (despite having it available), + # and it's not clear how to get the graph instance into that method. + # 2. OL has some janky logic to fetch table schemas as part of the sql extractor. + # We don't want that behavior and this lets us disable it. + # 3. Our SqlParsingResult already has DataHub urns, whereas using SqlMeta would + # require us to convert those urns to OL uris, just for them to get converted + # back to urns later on in our processing. + + task_name = f"{self.operator.dag_id}.{self.operator.task_id}" + sql = self.operator.sql + + run_facets = {} + job_facets = {"sql": SqlJobFacet(query=self._normalize_sql(sql))} + + # Prepare to run the SQL parser. + graph = self.context.get(_DATAHUB_GRAPH_CONTEXT_KEY, None) + + database = getattr(self.operator, "database", None) + if not database: + database = self._get_database() + + # TODO: Add better handling for sql being a list of statements. + if isinstance(sql, list): + logger.info(f"Got list of SQL statements for {task_name}. Using first one.") + sql = sql[0] + + # Run the SQL parser. + self.log.debug( + "Running the SQL parser (%s): %s", + "with graph client" if graph else "in offline mode", + sql, + ) + scheme = self.scheme + platform = OL_SCHEME_TWEAKS.get(scheme, scheme) + sql_parsing_result: SqlParsingResult = create_lineage_sql_parsed_result( + query=sql, + graph=graph, + platform=platform, + platform_instance=None, + env=builder.DEFAULT_ENV, + database=database, + schema=self.default_schema, + ) + self.log.debug(f"Got sql lineage {sql_parsing_result}") + + if sql_parsing_result.debug_info.error: + error = sql_parsing_result.debug_info.error + run_facets["extractionError"] = ExtractionErrorRunFacet( + totalTasks=1, + failedTasks=1, + errors=[ + ExtractionError( + errorMessage=str(error), + stackTrace=None, + task="datahub_sql_parser", + taskNumber=None, + ) + ], + ) + + # Save sql_parsing_result to the facets dict. It is removed from the + # facet dict in the extractor's processing logic. + run_facets[SQL_PARSING_RESULT_KEY] = sql_parsing_result # type: ignore + + return TaskMetadata( + name=task_name, + inputs=[], + outputs=[], + run_facets=run_facets, + job_facets=job_facets, + ) + + +def snowflake_default_schema(self: "SnowflakeExtractor") -> Optional[str]: + if hasattr(self.operator, "schema") and self.operator.schema is not None: + return self.operator.schema + return ( + self.conn.extra_dejson.get("extra__snowflake__schema", "") + or self.conn.extra_dejson.get("schema", "") + or self.conn.schema + ) + # TODO: Should we try a fallback of: + # execute_query_on_hook(self.hook, "SELECT current_schema();")[0][0] diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index b5e86e14d85d0..8dc84f9d4e46f 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -1,3 +1,4 @@ +from datetime import datetime from typing import TYPE_CHECKING, Dict, List, Optional, Set, Union, cast from airflow.configuration import conf @@ -171,7 +172,7 @@ def generate_dataflow( data_flow.url = f"{base_url}/tree?dag_id={dag.dag_id}" if capture_owner and dag.owner: - data_flow.owners.add(dag.owner) + data_flow.owners.update(owner.strip() for owner in dag.owner.split(",")) if capture_tags and dag.tags: data_flow.tags.update(dag.tags) @@ -442,8 +443,10 @@ def run_datajob( dpi.type = DataProcessTypeClass.BATCH_AD_HOC if start_timestamp_millis is None: - assert ti.start_date - start_timestamp_millis = int(ti.start_date.timestamp() * 1000) + if ti.start_date: + start_timestamp_millis = int(ti.start_date.timestamp() * 1000) + else: + start_timestamp_millis = int(datetime.now().timestamp() * 1000) if attempt is None: attempt = ti.try_number @@ -483,8 +486,10 @@ def complete_datajob( datajob = AirflowGenerator.generate_datajob(cluster, ti.task, dag) if end_timestamp_millis is None: - assert ti.end_date - end_timestamp_millis = int(ti.end_date.timestamp() * 1000) + if ti.end_date: + end_timestamp_millis = int(ti.end_date.timestamp() * 1000) + else: + end_timestamp_millis = int(datetime.now().timestamp() * 1000) if result is None: # We should use TaskInstanceState but it is not available in Airflow 1 diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py new file mode 100644 index 0000000000000..55ade19a63371 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -0,0 +1,434 @@ +import copy +import functools +import logging +import threading +from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast + +import datahub.emitter.mce_builder as builder +from airflow.listeners import hookimpl +from datahub.api.entities.datajob import DataJob +from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult +from datahub.emitter.rest_emitter import DatahubRestEmitter +from datahub.ingestion.graph.client import DataHubGraph +from datahub.metadata.schema_classes import ( + FineGrainedLineageClass, + FineGrainedLineageDownstreamTypeClass, + FineGrainedLineageUpstreamTypeClass, +) +from datahub.utilities.sqlglot_lineage import SqlParsingResult +from datahub.utilities.urns.dataset_urn import DatasetUrn +from openlineage.airflow.listener import TaskHolder +from openlineage.airflow.utils import redact_with_exclusions +from openlineage.client.serde import Serde + +from datahub_airflow_plugin._airflow_shims import ( + Operator, + get_task_inlets, + get_task_outlets, +) +from datahub_airflow_plugin._config import DatahubLineageConfig, get_lineage_config +from datahub_airflow_plugin._datahub_ol_adapter import translate_ol_to_datahub_urn +from datahub_airflow_plugin._extractors import SQL_PARSING_RESULT_KEY, ExtractorManager +from datahub_airflow_plugin.client.airflow_generator import AirflowGenerator +from datahub_airflow_plugin.entities import _Entity + +if TYPE_CHECKING: + from airflow.models import DAG, DagRun, TaskInstance + from sqlalchemy.orm import Session + +logger = logging.getLogger(__name__) + +_airflow_listener_initialized = False +_airflow_listener: Optional["DataHubListener"] = None +_RUN_IN_THREAD = True + + +def get_airflow_plugin_listener() -> Optional["DataHubListener"]: + # Using globals instead of functools.lru_cache to make testing easier. + global _airflow_listener_initialized + global _airflow_listener + + if not _airflow_listener_initialized: + _airflow_listener_initialized = True + + plugin_config = get_lineage_config() + + if plugin_config.enabled: + _airflow_listener = DataHubListener(config=plugin_config) + + if plugin_config.disable_openlineage_plugin: + # Deactivate the OpenLineagePlugin listener to avoid conflicts. + from openlineage.airflow.plugin import OpenLineagePlugin + + OpenLineagePlugin.listeners = [] + + return _airflow_listener + + +_F = TypeVar("_F", bound=Callable[..., None]) + + +def run_in_thread(f: _F) -> _F: + @functools.wraps(f) + def wrapper(*args, **kwargs): + try: + if _RUN_IN_THREAD: + # A poor-man's timeout mechanism. + # This ensures that we don't hang the task if the extractors + # are slow or the DataHub API is slow to respond. + + thread = threading.Thread( + target=f, args=args, kwargs=kwargs, daemon=True + ) + thread.start() + + thread.join(timeout=15) + else: + f(*args, **kwargs) + except Exception as e: + logger.exception(e) + + return cast(_F, wrapper) + + +class DataHubListener: + __name__ = "DataHubListener" + + def __init__(self, config: DatahubLineageConfig): + self.config = config + self._set_log_level() + + self._emitter = config.make_emitter_hook().make_emitter() + self._graph: Optional[DataHubGraph] = None + logger.info(f"DataHub plugin using {repr(self._emitter)}") + + # See discussion here https://github.com/OpenLineage/OpenLineage/pull/508 for + # why we need to keep track of tasks ourselves. + self._task_holder = TaskHolder() + + # In our case, we also want to cache the initial datajob object + # so that we can add to it when the task completes. + self._datajob_holder: Dict[str, DataJob] = {} + + self.extractor_manager = ExtractorManager() + + @property + def emitter(self): + return self._emitter + + @property + def graph(self) -> Optional[DataHubGraph]: + if self._graph: + return self._graph + + if isinstance(self._emitter, DatahubRestEmitter) and not isinstance( + self._emitter, DataHubGraph + ): + # This is lazy initialized to avoid throwing errors on plugin load. + self._graph = self._emitter.to_graph() + self._emitter = self._graph + + return self._graph + + def _set_log_level(self) -> None: + """Set the log level for the plugin and its dependencies. + + This may need to be called multiple times, since Airflow sometimes + messes with the logging configuration after the plugin is loaded. + In particular, the loggers may get changed when the worker starts + executing a task. + """ + + if self.config.log_level: + logging.getLogger(__name__.split(".")[0]).setLevel(self.config.log_level) + if self.config.debug_emitter: + logging.getLogger("datahub.emitter").setLevel(logging.DEBUG) + + def _make_emit_callback(self) -> Callable[[Optional[Exception], str], None]: + def emit_callback(err: Optional[Exception], msg: str) -> None: + if err: + logger.error(f"Error sending metadata to datahub: {msg}", exc_info=err) + + return emit_callback + + def _extract_lineage( + self, + datajob: DataJob, + dagrun: "DagRun", + task: "Operator", + task_instance: "TaskInstance", + complete: bool = False, + ) -> None: + """ + Combine lineage (including column lineage) from task inlets/outlets and + extractor-generated task_metadata and write it to the datajob. This + routine is also responsible for converting the lineage to DataHub URNs. + """ + + input_urns: List[str] = [] + output_urns: List[str] = [] + fine_grained_lineages: List[FineGrainedLineageClass] = [] + + task_metadata = None + if self.config.enable_extractors: + task_metadata = self.extractor_manager.extract_metadata( + dagrun, + task, + complete=complete, + task_instance=task_instance, + task_uuid=str(datajob.urn), + graph=self.graph, + ) + logger.debug(f"Got task metadata: {task_metadata}") + + # Translate task_metadata.inputs/outputs to DataHub URNs. + input_urns.extend( + translate_ol_to_datahub_urn(dataset) for dataset in task_metadata.inputs + ) + output_urns.extend( + translate_ol_to_datahub_urn(dataset) + for dataset in task_metadata.outputs + ) + + # Add DataHub-native SQL parser results. + sql_parsing_result: Optional[SqlParsingResult] = None + if task_metadata: + sql_parsing_result = task_metadata.run_facets.pop( + SQL_PARSING_RESULT_KEY, None + ) + if sql_parsing_result: + if sql_parsing_result.debug_info.error: + datajob.properties["datahub_sql_parser_error"] = str( + sql_parsing_result.debug_info.error + ) + else: + input_urns.extend(sql_parsing_result.in_tables) + output_urns.extend(sql_parsing_result.out_tables) + + if sql_parsing_result.column_lineage: + fine_grained_lineages.extend( + FineGrainedLineageClass( + upstreamType=FineGrainedLineageUpstreamTypeClass.FIELD_SET, + downstreamType=FineGrainedLineageDownstreamTypeClass.FIELD, + upstreams=[ + builder.make_schema_field_urn( + upstream.table, upstream.column + ) + for upstream in column_lineage.upstreams + ], + downstreams=[ + builder.make_schema_field_urn( + downstream.table, downstream.column + ) + for downstream in [column_lineage.downstream] + if downstream.table + ], + ) + for column_lineage in sql_parsing_result.column_lineage + ) + + # Add DataHub-native inlets/outlets. + # These are filtered out by the extractor, so we need to add them manually. + input_urns.extend( + iolet.urn for iolet in get_task_inlets(task) if isinstance(iolet, _Entity) + ) + output_urns.extend( + iolet.urn for iolet in get_task_outlets(task) if isinstance(iolet, _Entity) + ) + + # Write the lineage to the datajob object. + datajob.inlets.extend(DatasetUrn.create_from_string(urn) for urn in input_urns) + datajob.outlets.extend( + DatasetUrn.create_from_string(urn) for urn in output_urns + ) + datajob.fine_grained_lineages.extend(fine_grained_lineages) + + # Merge in extra stuff that was present in the DataJob we constructed + # at the start of the task. + if complete: + original_datajob = self._datajob_holder.get(str(datajob.urn), None) + else: + self._datajob_holder[str(datajob.urn)] = datajob + original_datajob = None + + if original_datajob: + logger.debug("Merging start datajob into finish datajob") + datajob.inlets.extend(original_datajob.inlets) + datajob.outlets.extend(original_datajob.outlets) + datajob.fine_grained_lineages.extend(original_datajob.fine_grained_lineages) + + for k, v in original_datajob.properties.items(): + datajob.properties.setdefault(k, v) + + # TODO: Deduplicate inlets/outlets. + + # Write all other OL facets as DataHub properties. + if task_metadata: + for k, v in task_metadata.job_facets.items(): + datajob.properties[f"openlineage_job_facet_{k}"] = Serde.to_json( + redact_with_exclusions(v) + ) + + for k, v in task_metadata.run_facets.items(): + datajob.properties[f"openlineage_run_facet_{k}"] = Serde.to_json( + redact_with_exclusions(v) + ) + + @hookimpl + @run_in_thread + def on_task_instance_running( + self, + previous_state: None, + task_instance: "TaskInstance", + session: "Session", # This will always be QUEUED + ) -> None: + self._set_log_level() + + # This if statement mirrors the logic in https://github.com/OpenLineage/OpenLineage/pull/508. + if not hasattr(task_instance, "task"): + logger.warning( + f"No task set for task_id: {task_instance.task_id} - " + f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}" + ) + return + + logger.debug( + f"DataHub listener got notification about task instance start for {task_instance.task_id}" + ) + + # Render templates in a copy of the task instance. + # This is necessary to get the correct operator args in the extractors. + task_instance = copy.deepcopy(task_instance) + task_instance.render_templates() + + dagrun: "DagRun" = task_instance.dag_run + task = task_instance.task + dag: "DAG" = task.dag # type: ignore[assignment] + + self._task_holder.set_task(task_instance) + + # Handle async operators in Airflow 2.3 by skipping deferred state. + # Inspired by https://github.com/OpenLineage/OpenLineage/pull/1601 + if task_instance.next_method is not None: + return + + datajob = AirflowGenerator.generate_datajob( + cluster=self.config.cluster, + task=task, + dag=dag, + capture_tags=self.config.capture_tags_info, + capture_owner=self.config.capture_ownership_info, + ) + + # TODO: Make use of get_task_location to extract github urls. + + # Add lineage info. + self._extract_lineage(datajob, dagrun, task, task_instance) + + # TODO: Add handling for Airflow mapped tasks using task_instance.map_index + + datajob.emit(self.emitter, callback=self._make_emit_callback()) + logger.debug(f"Emitted DataHub Datajob start: {datajob}") + + if self.config.capture_executions: + dpi = AirflowGenerator.run_datajob( + emitter=self.emitter, + cluster=self.config.cluster, + ti=task_instance, + dag=dag, + dag_run=dagrun, + datajob=datajob, + emit_templates=False, + ) + logger.debug(f"Emitted DataHub DataProcess Instance start: {dpi}") + + self.emitter.flush() + + def on_task_instance_finish( + self, task_instance: "TaskInstance", status: InstanceRunResult + ) -> None: + dagrun: "DagRun" = task_instance.dag_run + task = self._task_holder.get_task(task_instance) or task_instance.task + dag: "DAG" = task.dag # type: ignore[assignment] + + datajob = AirflowGenerator.generate_datajob( + cluster=self.config.cluster, + task=task, + dag=dag, + capture_tags=self.config.capture_tags_info, + capture_owner=self.config.capture_ownership_info, + ) + + # Add lineage info. + self._extract_lineage(datajob, dagrun, task, task_instance, complete=True) + + datajob.emit(self.emitter, callback=self._make_emit_callback()) + logger.debug(f"Emitted DataHub Datajob finish w/ status {status}: {datajob}") + + if self.config.capture_executions: + dpi = AirflowGenerator.complete_datajob( + emitter=self.emitter, + cluster=self.config.cluster, + ti=task_instance, + dag=dag, + dag_run=dagrun, + datajob=datajob, + result=status, + ) + logger.debug( + f"Emitted DataHub DataProcess Instance with status {status}: {dpi}" + ) + + self.emitter.flush() + + @hookimpl + @run_in_thread + def on_task_instance_success( + self, previous_state: None, task_instance: "TaskInstance", session: "Session" + ) -> None: + self._set_log_level() + + logger.debug( + f"DataHub listener got notification about task instance success for {task_instance.task_id}" + ) + + self.on_task_instance_finish(task_instance, status=InstanceRunResult.SUCCESS) + + @hookimpl + @run_in_thread + def on_task_instance_failed( + self, previous_state: None, task_instance: "TaskInstance", session: "Session" + ) -> None: + self._set_log_level() + + logger.debug( + f"DataHub listener got notification about task instance failure for {task_instance.task_id}" + ) + + # TODO: Handle UP_FOR_RETRY state. + self.on_task_instance_finish(task_instance, status=InstanceRunResult.FAILURE) + + @hookimpl + @run_in_thread + def on_dag_run_running(self, dag_run: "DagRun", msg: str) -> None: + self._set_log_level() + + logger.debug( + f"DataHub listener got notification about dag run start for {dag_run.dag_id}" + ) + + dag = dag_run.dag + if not dag: + return + + dataflow = AirflowGenerator.generate_dataflow( + cluster=self.config.cluster, + dag=dag, + capture_tags=self.config.capture_tags_info, + capture_owner=self.config.capture_ownership_info, + ) + dataflow.emit(self.emitter, callback=self._make_emit_callback()) + + self.emitter.flush() + + # TODO: Add hooks for on_dag_run_success, on_dag_run_failed -> call AirflowGenerator.complete_dataflow diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index d1cec9e5c1b54..c3db85c3a127d 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -1,367 +1,31 @@ -import contextlib import logging -import traceback -from typing import Any, Callable, Iterable, List, Optional, Union +import os -from airflow.configuration import conf -from airflow.lineage import PIPELINE_OUTLETS -from airflow.models.baseoperator import BaseOperator from airflow.plugins_manager import AirflowPlugin -from airflow.utils.module_loading import import_string -from cattr import structure -from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult from datahub_airflow_plugin._airflow_compat import AIRFLOW_PATCHED -from datahub_airflow_plugin._airflow_shims import MappedOperator, Operator -from datahub_airflow_plugin.client.airflow_generator import AirflowGenerator -from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook -from datahub_airflow_plugin.lineage.datahub import DatahubLineageConfig +from datahub_airflow_plugin._airflow_shims import IS_AIRFLOW_V23_PLUS assert AIRFLOW_PATCHED logger = logging.getLogger(__name__) -TASK_ON_FAILURE_CALLBACK = "on_failure_callback" -TASK_ON_SUCCESS_CALLBACK = "on_success_callback" +_USE_AIRFLOW_LISTENER_INTERFACE = IS_AIRFLOW_V23_PLUS and not os.getenv( + "DATAHUB_AIRFLOW_PLUGIN_USE_LEGACY_PLUGIN_POLICY", "false" +).lower() in ("true", "1") -def get_lineage_config() -> DatahubLineageConfig: - """Load the lineage config from airflow.cfg.""" - enabled = conf.get("datahub", "enabled", fallback=True) - datahub_conn_id = conf.get("datahub", "conn_id", fallback="datahub_rest_default") - cluster = conf.get("datahub", "cluster", fallback="prod") - graceful_exceptions = conf.get("datahub", "graceful_exceptions", fallback=True) - capture_tags_info = conf.get("datahub", "capture_tags_info", fallback=True) - capture_ownership_info = conf.get( - "datahub", "capture_ownership_info", fallback=True - ) - capture_executions = conf.get("datahub", "capture_executions", fallback=True) - return DatahubLineageConfig( - enabled=enabled, - datahub_conn_id=datahub_conn_id, - cluster=cluster, - graceful_exceptions=graceful_exceptions, - capture_ownership_info=capture_ownership_info, - capture_tags_info=capture_tags_info, - capture_executions=capture_executions, - ) - - -def _task_inlets(operator: "Operator") -> List: - # From Airflow 2.4 _inlets is dropped and inlets used consistently. Earlier it was not the case, so we have to stick there to _inlets - if hasattr(operator, "_inlets"): - return operator._inlets # type: ignore[attr-defined, union-attr] - return operator.inlets - - -def _task_outlets(operator: "Operator") -> List: - # From Airflow 2.4 _outlets is dropped and inlets used consistently. Earlier it was not the case, so we have to stick there to _outlets - # We have to use _outlets because outlets is empty in Airflow < 2.4.0 - if hasattr(operator, "_outlets"): - return operator._outlets # type: ignore[attr-defined, union-attr] - return operator.outlets - - -def get_inlets_from_task(task: BaseOperator, context: Any) -> Iterable[Any]: - # TODO: Fix for https://github.com/apache/airflow/commit/1b1f3fabc5909a447a6277cafef3a0d4ef1f01ae - # in Airflow 2.4. - # TODO: ignore/handle airflow's dataset type in our lineage - - inlets: List[Any] = [] - task_inlets = _task_inlets(task) - # From Airflow 2.3 this should be AbstractOperator but due to compatibility reason lets use BaseOperator - if isinstance(task_inlets, (str, BaseOperator)): - inlets = [ - task_inlets, - ] - - if task_inlets and isinstance(task_inlets, list): - inlets = [] - task_ids = ( - {o for o in task_inlets if isinstance(o, str)} - .union(op.task_id for op in task_inlets if isinstance(op, BaseOperator)) - .intersection(task.get_flat_relative_ids(upstream=True)) - ) - - from airflow.lineage import AUTO - - # pick up unique direct upstream task_ids if AUTO is specified - if AUTO.upper() in task_inlets or AUTO.lower() in task_inlets: - print("Picking up unique direct upstream task_ids as AUTO is specified") - task_ids = task_ids.union( - task_ids.symmetric_difference(task.upstream_task_ids) - ) - - inlets = task.xcom_pull( - context, task_ids=list(task_ids), dag_id=task.dag_id, key=PIPELINE_OUTLETS - ) - - # re-instantiate the obtained inlets - inlets = [ - structure(item["data"], import_string(item["type_name"])) - # _get_instance(structure(item, Metadata)) - for sublist in inlets - if sublist - for item in sublist - ] - - for inlet in task_inlets: - if not isinstance(inlet, str): - inlets.append(inlet) - - return inlets - - -def _make_emit_callback( - logger: logging.Logger, -) -> Callable[[Optional[Exception], str], None]: - def emit_callback(err: Optional[Exception], msg: str) -> None: - if err: - logger.error(f"Error sending metadata to datahub: {msg}", exc_info=err) - - return emit_callback - - -def datahub_task_status_callback(context, status): - ti = context["ti"] - task: "BaseOperator" = ti.task - dag = context["dag"] - - # This code is from the original airflow lineage code -> - # https://github.com/apache/airflow/blob/main/airflow/lineage/__init__.py - inlets = get_inlets_from_task(task, context) - - emitter = ( - DatahubGenericHook(context["_datahub_config"].datahub_conn_id) - .get_underlying_hook() - .make_emitter() - ) - - dataflow = AirflowGenerator.generate_dataflow( - cluster=context["_datahub_config"].cluster, - dag=dag, - capture_tags=context["_datahub_config"].capture_tags_info, - capture_owner=context["_datahub_config"].capture_ownership_info, - ) - task.log.info(f"Emitting Datahub Dataflow: {dataflow}") - dataflow.emit(emitter, callback=_make_emit_callback(task.log)) - - datajob = AirflowGenerator.generate_datajob( - cluster=context["_datahub_config"].cluster, - task=task, - dag=dag, - capture_tags=context["_datahub_config"].capture_tags_info, - capture_owner=context["_datahub_config"].capture_ownership_info, - ) - - for inlet in inlets: - datajob.inlets.append(inlet.urn) - - task_outlets = _task_outlets(task) - for outlet in task_outlets: - datajob.outlets.append(outlet.urn) - - task.log.info(f"Emitting Datahub Datajob: {datajob}") - datajob.emit(emitter, callback=_make_emit_callback(task.log)) - - if context["_datahub_config"].capture_executions: - dpi = AirflowGenerator.run_datajob( - emitter=emitter, - cluster=context["_datahub_config"].cluster, - ti=context["ti"], - dag=dag, - dag_run=context["dag_run"], - datajob=datajob, - start_timestamp_millis=int(ti.start_date.timestamp() * 1000), - ) - - task.log.info(f"Emitted Start Datahub Dataprocess Instance: {dpi}") - - dpi = AirflowGenerator.complete_datajob( - emitter=emitter, - cluster=context["_datahub_config"].cluster, - ti=context["ti"], - dag_run=context["dag_run"], - result=status, - dag=dag, - datajob=datajob, - end_timestamp_millis=int(ti.end_date.timestamp() * 1000), - ) - task.log.info(f"Emitted Completed Data Process Instance: {dpi}") - - emitter.flush() - - -def datahub_pre_execution(context): - ti = context["ti"] - task: "BaseOperator" = ti.task - dag = context["dag"] - - task.log.info("Running Datahub pre_execute method") - - emitter = ( - DatahubGenericHook(context["_datahub_config"].datahub_conn_id) - .get_underlying_hook() - .make_emitter() - ) - - # This code is from the original airflow lineage code -> - # https://github.com/apache/airflow/blob/main/airflow/lineage/__init__.py - inlets = get_inlets_from_task(task, context) - - datajob = AirflowGenerator.generate_datajob( - cluster=context["_datahub_config"].cluster, - task=context["ti"].task, - dag=dag, - capture_tags=context["_datahub_config"].capture_tags_info, - capture_owner=context["_datahub_config"].capture_ownership_info, - ) - - for inlet in inlets: - datajob.inlets.append(inlet.urn) - - task_outlets = _task_outlets(task) - - for outlet in task_outlets: - datajob.outlets.append(outlet.urn) - - task.log.info(f"Emitting Datahub dataJob {datajob}") - datajob.emit(emitter, callback=_make_emit_callback(task.log)) - - if context["_datahub_config"].capture_executions: - dpi = AirflowGenerator.run_datajob( - emitter=emitter, - cluster=context["_datahub_config"].cluster, - ti=context["ti"], - dag=dag, - dag_run=context["dag_run"], - datajob=datajob, - start_timestamp_millis=int(ti.start_date.timestamp() * 1000), - ) - - task.log.info(f"Emitting Datahub Dataprocess Instance: {dpi}") - - emitter.flush() - - -def _wrap_pre_execution(pre_execution): - def custom_pre_execution(context): - config = get_lineage_config() - if config.enabled: - context["_datahub_config"] = config - datahub_pre_execution(context) - - # Call original policy - if pre_execution: - pre_execution(context) - - return custom_pre_execution - - -def _wrap_on_failure_callback(on_failure_callback): - def custom_on_failure_callback(context): - config = get_lineage_config() - if config.enabled: - context["_datahub_config"] = config - try: - datahub_task_status_callback(context, status=InstanceRunResult.FAILURE) - except Exception as e: - if not config.graceful_exceptions: - raise e - else: - print(f"Exception: {traceback.format_exc()}") - - # Call original policy - if on_failure_callback: - on_failure_callback(context) - - return custom_on_failure_callback - - -def _wrap_on_success_callback(on_success_callback): - def custom_on_success_callback(context): - config = get_lineage_config() - if config.enabled: - context["_datahub_config"] = config - try: - datahub_task_status_callback(context, status=InstanceRunResult.SUCCESS) - except Exception as e: - if not config.graceful_exceptions: - raise e - else: - print(f"Exception: {traceback.format_exc()}") - - # Call original policy - if on_success_callback: - on_success_callback(context) - - return custom_on_success_callback - - -def task_policy(task: Union[BaseOperator, MappedOperator]) -> None: - task.log.debug(f"Setting task policy for Dag: {task.dag_id} Task: {task.task_id}") - # task.add_inlets(["auto"]) - # task.pre_execute = _wrap_pre_execution(task.pre_execute) +class DatahubPlugin(AirflowPlugin): + name = "datahub_plugin" - # MappedOperator's callbacks don't have setters until Airflow 2.X.X - # https://github.com/apache/airflow/issues/24547 - # We can bypass this by going through partial_kwargs for now - if MappedOperator and isinstance(task, MappedOperator): # type: ignore - on_failure_callback_prop: property = getattr( - MappedOperator, TASK_ON_FAILURE_CALLBACK - ) - on_success_callback_prop: property = getattr( - MappedOperator, TASK_ON_SUCCESS_CALLBACK + if _USE_AIRFLOW_LISTENER_INTERFACE: + from datahub_airflow_plugin.datahub_listener import ( # type: ignore[misc] + get_airflow_plugin_listener, ) - if not on_failure_callback_prop.fset or not on_success_callback_prop.fset: - task.log.debug( - "Using MappedOperator's partial_kwargs instead of callback properties" - ) - task.partial_kwargs[TASK_ON_FAILURE_CALLBACK] = _wrap_on_failure_callback( - task.on_failure_callback - ) - task.partial_kwargs[TASK_ON_SUCCESS_CALLBACK] = _wrap_on_success_callback( - task.on_success_callback - ) - return - - task.on_failure_callback = _wrap_on_failure_callback(task.on_failure_callback) # type: ignore - task.on_success_callback = _wrap_on_success_callback(task.on_success_callback) # type: ignore - # task.pre_execute = _wrap_pre_execution(task.pre_execute) - - -def _wrap_task_policy(policy): - if policy and hasattr(policy, "_task_policy_patched_by"): - return policy - - def custom_task_policy(task): - policy(task) - task_policy(task) - - # Add a flag to the policy to indicate that we've patched it. - custom_task_policy._task_policy_patched_by = "datahub_plugin" # type: ignore[attr-defined] - return custom_task_policy + listeners = list(filter(None, [get_airflow_plugin_listener()])) -def _patch_policy(settings): - if hasattr(settings, "task_policy"): - datahub_task_policy = _wrap_task_policy(settings.task_policy) - settings.task_policy = datahub_task_policy - -def _patch_datahub_policy(): - with contextlib.suppress(ImportError): - import airflow_local_settings - - _patch_policy(airflow_local_settings) - - from airflow.models.dagbag import settings - - _patch_policy(settings) - - -_patch_datahub_policy() - - -class DatahubPlugin(AirflowPlugin): - name = "datahub_plugin" +if not _USE_AIRFLOW_LISTENER_INTERFACE: + # Use the policy patcher mechanism on Airflow 2.2 and below. + import datahub_airflow_plugin.datahub_plugin_v22 # noqa: F401 diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py new file mode 100644 index 0000000000000..e8ff02d48a651 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py @@ -0,0 +1,322 @@ +import contextlib +import logging +import traceback +from typing import Any, Callable, Iterable, List, Optional, Union + +from airflow.lineage import PIPELINE_OUTLETS +from airflow.models.baseoperator import BaseOperator +from airflow.utils.module_loading import import_string +from cattr import structure +from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult + +from datahub_airflow_plugin._airflow_shims import ( + MappedOperator, + get_task_inlets, + get_task_outlets, +) +from datahub_airflow_plugin._config import get_lineage_config +from datahub_airflow_plugin.client.airflow_generator import AirflowGenerator +from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook +from datahub_airflow_plugin.lineage.datahub import DatahubLineageConfig + +TASK_ON_FAILURE_CALLBACK = "on_failure_callback" +TASK_ON_SUCCESS_CALLBACK = "on_success_callback" + + +def get_task_inlets_advanced(task: BaseOperator, context: Any) -> Iterable[Any]: + # TODO: Fix for https://github.com/apache/airflow/commit/1b1f3fabc5909a447a6277cafef3a0d4ef1f01ae + # in Airflow 2.4. + # TODO: ignore/handle airflow's dataset type in our lineage + + inlets: List[Any] = [] + task_inlets = get_task_inlets(task) + # From Airflow 2.3 this should be AbstractOperator but due to compatibility reason lets use BaseOperator + if isinstance(task_inlets, (str, BaseOperator)): + inlets = [ + task_inlets, + ] + + if task_inlets and isinstance(task_inlets, list): + inlets = [] + task_ids = ( + {o for o in task_inlets if isinstance(o, str)} + .union(op.task_id for op in task_inlets if isinstance(op, BaseOperator)) + .intersection(task.get_flat_relative_ids(upstream=True)) + ) + + from airflow.lineage import AUTO + + # pick up unique direct upstream task_ids if AUTO is specified + if AUTO.upper() in task_inlets or AUTO.lower() in task_inlets: + print("Picking up unique direct upstream task_ids as AUTO is specified") + task_ids = task_ids.union( + task_ids.symmetric_difference(task.upstream_task_ids) + ) + + inlets = task.xcom_pull( + context, task_ids=list(task_ids), dag_id=task.dag_id, key=PIPELINE_OUTLETS + ) + + # re-instantiate the obtained inlets + inlets = [ + structure(item["data"], import_string(item["type_name"])) + # _get_instance(structure(item, Metadata)) + for sublist in inlets + if sublist + for item in sublist + ] + + for inlet in task_inlets: + if not isinstance(inlet, str): + inlets.append(inlet) + + return inlets + + +def _make_emit_callback( + logger: logging.Logger, +) -> Callable[[Optional[Exception], str], None]: + def emit_callback(err: Optional[Exception], msg: str) -> None: + if err: + logger.error(f"Error sending metadata to datahub: {msg}", exc_info=err) + + return emit_callback + + +def datahub_task_status_callback(context, status): + ti = context["ti"] + task: "BaseOperator" = ti.task + dag = context["dag"] + config: DatahubLineageConfig = context["_datahub_config"] + + # This code is from the original airflow lineage code -> + # https://github.com/apache/airflow/blob/main/airflow/lineage/__init__.py + inlets = get_task_inlets_advanced(task, context) + + emitter = ( + DatahubGenericHook(config.datahub_conn_id).get_underlying_hook().make_emitter() + ) + + dataflow = AirflowGenerator.generate_dataflow( + cluster=config.cluster, + dag=dag, + capture_tags=config.capture_tags_info, + capture_owner=config.capture_ownership_info, + ) + task.log.info(f"Emitting Datahub Dataflow: {dataflow}") + dataflow.emit(emitter, callback=_make_emit_callback(task.log)) + + datajob = AirflowGenerator.generate_datajob( + cluster=config.cluster, + task=task, + dag=dag, + capture_tags=config.capture_tags_info, + capture_owner=config.capture_ownership_info, + ) + + for inlet in inlets: + datajob.inlets.append(inlet.urn) + + task_outlets = get_task_outlets(task) + for outlet in task_outlets: + datajob.outlets.append(outlet.urn) + + task.log.info(f"Emitting Datahub Datajob: {datajob}") + datajob.emit(emitter, callback=_make_emit_callback(task.log)) + + if config.capture_executions: + dpi = AirflowGenerator.run_datajob( + emitter=emitter, + cluster=config.cluster, + ti=ti, + dag=dag, + dag_run=context["dag_run"], + datajob=datajob, + start_timestamp_millis=int(ti.start_date.timestamp() * 1000), + ) + + task.log.info(f"Emitted Start Datahub Dataprocess Instance: {dpi}") + + dpi = AirflowGenerator.complete_datajob( + emitter=emitter, + cluster=config.cluster, + ti=ti, + dag_run=context["dag_run"], + result=status, + dag=dag, + datajob=datajob, + end_timestamp_millis=int(ti.end_date.timestamp() * 1000), + ) + task.log.info(f"Emitted Completed Data Process Instance: {dpi}") + + emitter.flush() + + +def datahub_pre_execution(context): + ti = context["ti"] + task: "BaseOperator" = ti.task + dag = context["dag"] + config: DatahubLineageConfig = context["_datahub_config"] + + task.log.info("Running Datahub pre_execute method") + + emitter = ( + DatahubGenericHook(config.datahub_conn_id).get_underlying_hook().make_emitter() + ) + + # This code is from the original airflow lineage code -> + # https://github.com/apache/airflow/blob/main/airflow/lineage/__init__.py + inlets = get_task_inlets_advanced(task, context) + + datajob = AirflowGenerator.generate_datajob( + cluster=config.cluster, + task=ti.task, + dag=dag, + capture_tags=config.capture_tags_info, + capture_owner=config.capture_ownership_info, + ) + + for inlet in inlets: + datajob.inlets.append(inlet.urn) + + task_outlets = get_task_outlets(task) + + for outlet in task_outlets: + datajob.outlets.append(outlet.urn) + + task.log.info(f"Emitting Datahub dataJob {datajob}") + datajob.emit(emitter, callback=_make_emit_callback(task.log)) + + if config.capture_executions: + dpi = AirflowGenerator.run_datajob( + emitter=emitter, + cluster=config.cluster, + ti=ti, + dag=dag, + dag_run=context["dag_run"], + datajob=datajob, + start_timestamp_millis=int(ti.start_date.timestamp() * 1000), + ) + + task.log.info(f"Emitting Datahub Dataprocess Instance: {dpi}") + + emitter.flush() + + +def _wrap_pre_execution(pre_execution): + def custom_pre_execution(context): + config = get_lineage_config() + if config.enabled: + context["_datahub_config"] = config + datahub_pre_execution(context) + + # Call original policy + if pre_execution: + pre_execution(context) + + return custom_pre_execution + + +def _wrap_on_failure_callback(on_failure_callback): + def custom_on_failure_callback(context): + config = get_lineage_config() + if config.enabled: + context["_datahub_config"] = config + try: + datahub_task_status_callback(context, status=InstanceRunResult.FAILURE) + except Exception as e: + if not config.graceful_exceptions: + raise e + else: + print(f"Exception: {traceback.format_exc()}") + + # Call original policy + if on_failure_callback: + on_failure_callback(context) + + return custom_on_failure_callback + + +def _wrap_on_success_callback(on_success_callback): + def custom_on_success_callback(context): + config = get_lineage_config() + if config.enabled: + context["_datahub_config"] = config + try: + datahub_task_status_callback(context, status=InstanceRunResult.SUCCESS) + except Exception as e: + if not config.graceful_exceptions: + raise e + else: + print(f"Exception: {traceback.format_exc()}") + + # Call original policy + if on_success_callback: + on_success_callback(context) + + return custom_on_success_callback + + +def task_policy(task: Union[BaseOperator, MappedOperator]) -> None: + task.log.debug(f"Setting task policy for Dag: {task.dag_id} Task: {task.task_id}") + # task.add_inlets(["auto"]) + # task.pre_execute = _wrap_pre_execution(task.pre_execute) + + # MappedOperator's callbacks don't have setters until Airflow 2.X.X + # https://github.com/apache/airflow/issues/24547 + # We can bypass this by going through partial_kwargs for now + if MappedOperator and isinstance(task, MappedOperator): # type: ignore + on_failure_callback_prop: property = getattr( + MappedOperator, TASK_ON_FAILURE_CALLBACK + ) + on_success_callback_prop: property = getattr( + MappedOperator, TASK_ON_SUCCESS_CALLBACK + ) + if not on_failure_callback_prop.fset or not on_success_callback_prop.fset: + task.log.debug( + "Using MappedOperator's partial_kwargs instead of callback properties" + ) + task.partial_kwargs[TASK_ON_FAILURE_CALLBACK] = _wrap_on_failure_callback( + task.on_failure_callback + ) + task.partial_kwargs[TASK_ON_SUCCESS_CALLBACK] = _wrap_on_success_callback( + task.on_success_callback + ) + return + + task.on_failure_callback = _wrap_on_failure_callback(task.on_failure_callback) # type: ignore + task.on_success_callback = _wrap_on_success_callback(task.on_success_callback) # type: ignore + # task.pre_execute = _wrap_pre_execution(task.pre_execute) + + +def _wrap_task_policy(policy): + if policy and hasattr(policy, "_task_policy_patched_by"): + return policy + + def custom_task_policy(task): + policy(task) + task_policy(task) + + # Add a flag to the policy to indicate that we've patched it. + custom_task_policy._task_policy_patched_by = "datahub_plugin" # type: ignore[attr-defined] + return custom_task_policy + + +def _patch_policy(settings): + if hasattr(settings, "task_policy"): + datahub_task_policy = _wrap_task_policy(settings.task_policy) + settings.task_policy = datahub_task_policy + + +def _patch_datahub_policy(): + with contextlib.suppress(ImportError): + import airflow_local_settings + + _patch_policy(airflow_local_settings) + + from airflow.models.dagbag import settings + + _patch_policy(settings) + + +_patch_datahub_policy() diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_lineage_core.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/_lineage_core.py similarity index 72% rename from metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_lineage_core.py rename to metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/_lineage_core.py index d91c039ffa718..f5f519fa23b11 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_lineage_core.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/_lineage_core.py @@ -1,11 +1,10 @@ from datetime import datetime from typing import TYPE_CHECKING, Dict, List -import datahub.emitter.mce_builder as builder from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult -from datahub.configuration.common import ConfigModel from datahub.utilities.urns.dataset_urn import DatasetUrn +from datahub_airflow_plugin._config import DatahubLineageConfig from datahub_airflow_plugin.client.airflow_generator import AirflowGenerator from datahub_airflow_plugin.entities import _Entity @@ -15,39 +14,14 @@ from airflow.models.taskinstance import TaskInstance from datahub_airflow_plugin._airflow_shims import Operator - from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook def _entities_to_urn_list(iolets: List[_Entity]) -> List[DatasetUrn]: return [DatasetUrn.create_from_string(let.urn) for let in iolets] -class DatahubBasicLineageConfig(ConfigModel): - enabled: bool = True - - # DataHub hook connection ID. - datahub_conn_id: str - - # Cluster to associate with the pipelines and tasks. Defaults to "prod". - cluster: str = builder.DEFAULT_FLOW_CLUSTER - - # If true, the owners field of the DAG will be capture as a DataHub corpuser. - capture_ownership_info: bool = True - - # If true, the tags field of the DAG will be captured as DataHub tags. - capture_tags_info: bool = True - - capture_executions: bool = False - - def make_emitter_hook(self) -> "DatahubGenericHook": - # This is necessary to avoid issues with circular imports. - from datahub_airflow_plugin.hooks.datahub import DatahubGenericHook - - return DatahubGenericHook(self.datahub_conn_id) - - def send_lineage_to_datahub( - config: DatahubBasicLineageConfig, + config: DatahubLineageConfig, operator: "Operator", inlets: List[_Entity], outlets: List[_Entity], diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/datahub.py index c41bb2b2a1e37..3ebe7831d08f9 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/lineage/datahub.py @@ -4,8 +4,8 @@ from airflow.configuration import conf from airflow.lineage.backend import LineageBackend -from datahub_airflow_plugin._lineage_core import ( - DatahubBasicLineageConfig, +from datahub_airflow_plugin.lineage._lineage_core import ( + DatahubLineageConfig, send_lineage_to_datahub, ) @@ -13,14 +13,7 @@ from airflow.models.baseoperator import BaseOperator -class DatahubLineageConfig(DatahubBasicLineageConfig): - # If set to true, most runtime errors in the lineage backend will be - # suppressed and will not cause the overall task to fail. Note that - # configuration issues will still throw exceptions. - graceful_exceptions: bool = True - - -def get_lineage_config() -> DatahubLineageConfig: +def get_lineage_backend_config() -> DatahubLineageConfig: """Load the lineage config from airflow.cfg.""" # The kwargs pattern is also used for secret backends. @@ -51,8 +44,7 @@ class DatahubLineageBackend(LineageBackend): datahub_kwargs = { "datahub_conn_id": "datahub_rest_default", "capture_ownership_info": true, - "capture_tags_info": true, - "graceful_exceptions": true } + "capture_tags_info": true } # The above indentation is important! """ @@ -61,7 +53,7 @@ def __init__(self) -> None: # By attempting to get and parse the config, we can detect configuration errors # ahead of time. The init method is only called in Airflow 2.x. - _ = get_lineage_config() + _ = get_lineage_backend_config() # With Airflow 2.0, this can be an instance method. However, with Airflow 1.10.x, this # method is used statically, even though LineageBackend declares it as an instance variable. @@ -72,7 +64,7 @@ def send_lineage( outlets: Optional[List] = None, # unused context: Optional[Dict] = None, ) -> None: - config = get_lineage_config() + config = get_lineage_backend_config() if not config.enabled: return @@ -82,10 +74,4 @@ def send_lineage( config, operator, operator.inlets, operator.outlets, context ) except Exception as e: - if config.graceful_exceptions: - operator.log.error(e) - operator.log.info( - "Suppressing error because graceful_exceptions is set" - ) - else: - raise + operator.log.error(e) diff --git a/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py b/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py index 7eb6fc8c8d1a9..3cd23f63c1ab7 100644 --- a/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py +++ b/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py @@ -8,9 +8,8 @@ AzkabanJobTypeClass, DataJobInfoClass, DataJobInputOutputClass, - DataJobSnapshotClass, + FineGrainedLineageClass, GlobalTagsClass, - MetadataChangeEventClass, OwnerClass, OwnershipClass, OwnershipSourceClass, @@ -59,6 +58,7 @@ class DataJob: group_owners: Set[str] = field(default_factory=set) inlets: List[DatasetUrn] = field(default_factory=list) outlets: List[DatasetUrn] = field(default_factory=list) + fine_grained_lineages: List[FineGrainedLineageClass] = field(default_factory=list) upstream_urns: List[DataJobUrn] = field(default_factory=list) def __post_init__(self): @@ -103,31 +103,6 @@ def generate_tags_aspect(self) -> Iterable[GlobalTagsClass]: ) return [tags] - def generate_mce(self) -> MetadataChangeEventClass: - job_mce = MetadataChangeEventClass( - proposedSnapshot=DataJobSnapshotClass( - urn=str(self.urn), - aspects=[ - DataJobInfoClass( - name=self.name if self.name is not None else self.id, - type=AzkabanJobTypeClass.COMMAND, - description=self.description, - customProperties=self.properties, - externalUrl=self.url, - ), - DataJobInputOutputClass( - inputDatasets=[str(urn) for urn in self.inlets], - outputDatasets=[str(urn) for urn in self.outlets], - inputDatajobs=[str(urn) for urn in self.upstream_urns], - ), - *self.generate_ownership_aspect(), - *self.generate_tags_aspect(), - ], - ) - ) - - return job_mce - def generate_mcp(self) -> Iterable[MetadataChangeProposalWrapper]: mcp = MetadataChangeProposalWrapper( entityUrn=str(self.urn), @@ -179,6 +154,7 @@ def generate_data_input_output_mcp(self) -> Iterable[MetadataChangeProposalWrapp inputDatasets=[str(urn) for urn in self.inlets], outputDatasets=[str(urn) for urn in self.outlets], inputDatajobs=[str(urn) for urn in self.upstream_urns], + fineGrainedLineages=self.fine_grained_lineages, ), ) yield mcp diff --git a/metadata-ingestion/src/datahub/emitter/rest_emitter.py b/metadata-ingestion/src/datahub/emitter/rest_emitter.py index 937e0902d6d8c..c25d42d10b84e 100644 --- a/metadata-ingestion/src/datahub/emitter/rest_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/rest_emitter.py @@ -4,7 +4,7 @@ import logging import os from json.decoder import JSONDecodeError -from typing import Any, Callable, Dict, List, Optional, Tuple, Union +from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional, Tuple, Union import requests from deprecated import deprecated @@ -23,6 +23,9 @@ ) from datahub.metadata.com.linkedin.pegasus2avro.usage import UsageAggregation +if TYPE_CHECKING: + from datahub.ingestion.graph.client import DataHubGraph + logger = logging.getLogger(__name__) _DEFAULT_CONNECT_TIMEOUT_SEC = 30 # 30 seconds should be plenty to connect @@ -190,6 +193,11 @@ def test_connection(self) -> dict: message += "\nPlease check your configuration and make sure you are talking to the DataHub GMS (usually :8080) or Frontend GMS API (usually :9002/api/gms)." raise ConfigurationError(message) + def to_graph(self) -> "DataHubGraph": + from datahub.ingestion.graph.client import DataHubGraph + + return DataHubGraph.from_emitter(self) + def emit( self, item: Union[ diff --git a/metadata-ingestion/src/datahub/ingestion/graph/client.py b/metadata-ingestion/src/datahub/ingestion/graph/client.py index b371ab181e133..1099b6305f1a4 100644 --- a/metadata-ingestion/src/datahub/ingestion/graph/client.py +++ b/metadata-ingestion/src/datahub/ingestion/graph/client.py @@ -150,6 +150,23 @@ def __init__(self, config: DatahubClientConfig) -> None: self.server_id = "missing" logger.debug(f"Failed to get server id due to {e}") + @classmethod + def from_emitter(cls, emitter: DatahubRestEmitter) -> "DataHubGraph": + return cls( + DatahubClientConfig( + server=emitter._gms_server, + token=emitter._token, + timeout_sec=emitter._read_timeout_sec, + retry_status_codes=emitter._retry_status_codes, + retry_max_times=emitter._retry_max_times, + # TODO: Support these headers. + # extra_headers=emitter._extra_headers, + # ca_certificate_path=emitter._ca_certificate_path, + # client_certificate_path=emitter._client_certificate_path, + # disable_ssl_verification=emitter._disable_ssl_verification, + ) + ) + def _send_restli_request(self, method: str, url: str, **kwargs: Any) -> Dict: try: response = self._session.request(method, url, **kwargs) diff --git a/metadata-ingestion/src/datahub/ingestion/source/tableau.py b/metadata-ingestion/src/datahub/ingestion/source/tableau.py index 4cc00a66116e9..6214cba342622 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/tableau.py +++ b/metadata-ingestion/src/datahub/ingestion/source/tableau.py @@ -1179,8 +1179,6 @@ def get_upstream_fields_of_field_in_datasource( def get_upstream_fields_from_custom_sql( self, datasource: dict, datasource_urn: str ) -> List[FineGrainedLineage]: - fine_grained_lineages: List[FineGrainedLineage] = [] - parsed_result = self.parse_custom_sql( datasource=datasource, datasource_urn=datasource_urn, @@ -1194,13 +1192,20 @@ def get_upstream_fields_from_custom_sql( logger.info( f"Failed to extract column level lineage from datasource {datasource_urn}" ) - return fine_grained_lineages + return [] + if parsed_result.debug_info.error: + logger.info( + f"Failed to extract column level lineage from datasource {datasource_urn}: {parsed_result.debug_info.error}" + ) + return [] cll: List[ColumnLineageInfo] = ( parsed_result.column_lineage if parsed_result.column_lineage is not None else [] ) + + fine_grained_lineages: List[FineGrainedLineage] = [] for cll_info in cll: downstream = ( [ diff --git a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py index d677b0874b985..8ec7a42f3e2e7 100644 --- a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py +++ b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py @@ -872,8 +872,7 @@ def create_lineage_sql_parsed_result( env: str, schema: Optional[str] = None, graph: Optional[DataHubGraph] = None, -) -> Optional["SqlParsingResult"]: - parsed_result: Optional["SqlParsingResult"] = None +) -> SqlParsingResult: try: schema_resolver = ( graph._make_schema_resolver( @@ -890,14 +889,18 @@ def create_lineage_sql_parsed_result( ) ) - parsed_result = sqlglot_lineage( + return sqlglot_lineage( query, schema_resolver=schema_resolver, default_db=database, default_schema=schema, ) except Exception as e: - logger.debug(f"Fail to prase query {query}", exc_info=e) - logger.warning("Fail to parse custom SQL") - - return parsed_result + return SqlParsingResult( + in_tables=[], + out_tables=[], + column_lineage=None, + debug_info=SqlParsingDebugInfo( + table_error=e, + ), + ) From 66c20c8eaca07e680f9fb66d34102eb48f91ee97 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 18 Sep 2023 17:13:57 -0700 Subject: [PATCH 03/60] start updating docs --- docs/lineage/airflow.md | 9 +++++---- .../src/datahub_airflow_plugin/_extractors.py | 2 ++ 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index 49de5352f6d58..a090aec61cc00 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -1,10 +1,11 @@ # Airflow Integration -DataHub supports integration of +The DataHub Airflow plugin supports: -- Airflow Pipeline (DAG) metadata -- DAG and Task run information as well as -- Lineage information when present +- Automatic lineage extraction from various operators e.g. `SqlOperator` (including `MySqlOperator`, `PostgresOperator`, `SnowflakeOperator`, and more), `S3FileTransformOperator`, and a few others. +- Airflow DAG metadata +- Extracting DAG and Task run information +- Manual lineage annotations using `inlets` and `outlets` on Airflow operators You can use either the DataHub Airflow lineage plugin (recommended) or the Airflow lineage backend (deprecated). diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py index 40758ad1909e1..3242e8d9657bd 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py @@ -66,6 +66,8 @@ def _patch_extractors(self): # TODO: Override the BigQuery extractor to use the DataHub SQL parser. # self.extractor_manager.add_extractor() + # TODO: Override the Athena extractor to use the DataHub SQL parser. + yield def extract_metadata( From b4545cc004883f35e9b5734a86ad64db325ff559 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 19 Sep 2023 11:58:52 -0700 Subject: [PATCH 04/60] update workflow --- .github/workflows/airflow-plugin.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/airflow-plugin.yml b/.github/workflows/airflow-plugin.yml index 92097911fb067..dbea2fa1a112f 100644 --- a/.github/workflows/airflow-plugin.yml +++ b/.github/workflows/airflow-plugin.yml @@ -33,9 +33,9 @@ jobs: matrix: include: - python-version: "3.8" - extraPythonRequirement: "apache-airflow~=2.1.0" + extraPythonRequirement: "apache-airflow~=2.1.4" - python-version: "3.8" - extraPythonRequirement: "apache-airflow~=2.2.0" + extraPythonRequirement: "apache-airflow~=2.2.4" - python-version: "3.10" extraPythonRequirement: "apache-airflow~=2.4.0" - python-version: "3.10" From f875eeb353be84f2e334898b58d66e52e2a717cb Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 14:40:58 -0700 Subject: [PATCH 05/60] update docs --- docker/airflow/local_airflow.md | 2 +- docs/how/updating-datahub.md | 1 + docs/lineage/airflow.md | 242 ++++++++++-------- .../airflow-plugin/setup.py | 21 +- 4 files changed, 154 insertions(+), 112 deletions(-) diff --git a/docker/airflow/local_airflow.md b/docker/airflow/local_airflow.md index 55a64f5c122c5..fbfc1d17327c5 100644 --- a/docker/airflow/local_airflow.md +++ b/docker/airflow/local_airflow.md @@ -1,6 +1,6 @@ :::caution -This feature is currently unmaintained. As of 0.10.0 the container described is not published alongside the DataHub CLI. If you'd like to use it, please reach out to us on the [community slack.](docs/slack.md) +This guide is currently unmaintained. As of 0.10.0 the container described is not published alongside the DataHub CLI. If you'd like to use it, please reach out to us on the [community slack.](docs/slack.md) ::: diff --git a/docs/how/updating-datahub.md b/docs/how/updating-datahub.md index 9b19291ee246a..bd9fa92a7afef 100644 --- a/docs/how/updating-datahub.md +++ b/docs/how/updating-datahub.md @@ -6,6 +6,7 @@ This file documents any backwards-incompatible changes in DataHub and assists pe ### Breaking Changes - #8810 - Removed support for SQLAlchemy 1.3.x. Only SQLAlchemy 1.4.x is supported now. +- #8853 - The Airflow plugin no longer supports Airflow 2.0.x or Python 3.7. See the docs for more details. ### Potential Downtime diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index a090aec61cc00..7927ae5b5bb0f 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -1,29 +1,80 @@ # Airflow Integration +:::note + +If you're looking to schedule DataHub ingestion using Airflow, see the guide on [scheduling ingestion with Airflow](../../metadata-ingestion/schedule_docs/airflow.md). + +::: + The DataHub Airflow plugin supports: -- Automatic lineage extraction from various operators e.g. `SqlOperator` (including `MySqlOperator`, `PostgresOperator`, `SnowflakeOperator`, and more), `S3FileTransformOperator`, and a few others. -- Airflow DAG metadata -- Extracting DAG and Task run information -- Manual lineage annotations using `inlets` and `outlets` on Airflow operators +- Automatic column-level lineage extraction from various operators e.g. `SqlOperator`s (including `MySqlOperator`, `PostgresOperator`, `SnowflakeOperator`, and more), `S3FileTransformOperator`, and a few others. +- Airflow DAG and tasks, including properties, ownership, and tags. +- Task run information, including task successes and failures. +- Manual lineage annotations using `inlets` and `outlets` on Airflow operators. -You can use either the DataHub Airflow lineage plugin (recommended) or the Airflow lineage backend (deprecated). +There's two actively supported implementations of the plugin, with different Airflow version support. -## Using Datahub's Airflow lineage plugin +| Approach | Airflow Version | Notes | +| --------- | --------------- | ------------------------------------------- | +| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | +| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction. | -:::note +If you're using Airflow older than 2.1, it's possible to use the v1 plugin with older versions of `acryl-datahub-airflow-plugin`. See the [compatibility section](#compatibility) for more details. -The Airflow lineage plugin is only supported with Airflow version >= 2.0.2 or on MWAA with an Airflow version >= 2.0.2. + + -If you're using Airflow 1.x, use the Airflow lineage plugin with acryl-datahub-airflow-plugin <= 0.9.1.0. +## DataHub Plugin v2 -::: +### Installation + +The v2 plugin requires Airflow 2.3+ and Python 3.8+. If you don't meet these requirements, use the v1 plugin instead. -This plugin registers a task success/failure callback on every task with a cluster policy and emits DataHub events from that. This allows this plugin to be able to register both task success as well as failures compared to the older Airflow Lineage Backend which could only support emitting task success. +```sh +pip install acryl-datahub-airflow-plugin[plugin-v2] +``` + +### Configuration + +Set up a DataHub connection in Airflow. + +```sh +airflow connections add --conn-type 'datahub-rest' 'datahub_rest_default' --conn-host 'http://datahub-gms:8080' --conn-password '' +``` + +No additional configuration is required to use the plugin. However, there are some optional configuration parameters that can be set in the `airflow.cfg` file. -### Setup +```ini title="airflow.cfg" +[datahub] +# Additional config here. +#enabled = True # default +``` -1. You need to install the required dependency in your airflow. +| Name | Default value | Description | +| -------------------------- | -------------------- | ---------------------------------------------------------------------------------------- | +| enabled | true | If the plugin should be enabled. | +| conn_id | datahub_rest_default | The name of the datahub rest connection. | +| cluster | prod | name of the airflow cluster | +| capture_ownership_info | true | Extract DAG ownership. | +| capture_tags_info | true | Extract DAG tags. | +| capture_executions | true | Extract task runs and success/failure statuses. This will show up in DataHub "Runs" tab. | +| enable_extractors | true | Enable automatic lineage extraction. | +| log_level | no change | [debug] Set the log level for the plugin. | +| debug_emitter | false | [debug] If true, the plugin will log the emitted events. | +| disable_openlineage_plugin | true | Disable the OpenLineage plugin to avoid duplicative processing. | + +### Automatic lineage extraction + +The v2 plugin uses Airflow's built-in [OpenLineage extractors](https://openlineage.io/docs/integrations/airflow/default-extractors) to extract lineage information. + +The SQL-related extractors have been updated to use DataHub's SQL parser, which is more robust than the built-in one and uses DataHub's metadata information to generate column-level lineage. We discussed the DataHub SQL parser, including why schema-aware parsing works better and how it performs on benchmarks, during the [June 2023 community town hall](https://youtu.be/1QVcUmRQK5E?si=U27zygR7Gi_KdkzE&t=2309). + +## DataHub Plugin v1 + +### Installation + +The v1 plugin requires Airflow 2.1+ and Python 3.8+. If you're on older versions, it's still possible to use an older version of the plugin. See the [compatibility section](#compatibility) for more details. ```shell pip install acryl-datahub-airflow-plugin @@ -35,41 +86,52 @@ The [DataHub Rest](../../metadata-ingestion/sink_docs/datahub.md#datahub-rest) e ::: -2. Disable lazy plugin loading in your airflow.cfg. - On MWAA you should add this config to your [Apache Airflow configuration options](https://docs.aws.amazon.com/mwaa/latest/userguide/configuring-env-variables.html#configuring-2.0-airflow-override). + + +### Configuration + +#### Disable lazy plugin loading ```ini title="airflow.cfg" [core] lazy_load_plugins = False ``` -3. You must configure an Airflow hook for Datahub. We support both a Datahub REST hook and a Kafka-based hook, but you only need one. +On MWAA you should add this config to your [Apache Airflow configuration options](https://docs.aws.amazon.com/mwaa/latest/userguide/configuring-env-variables.html#configuring-2.0-airflow-override). - ```shell - # For REST-based: - airflow connections add --conn-type 'datahub_rest' 'datahub_rest_default' --conn-host 'http://datahub-gms:8080' --conn-password '' - # For Kafka-based (standard Kafka sink config can be passed via extras): - airflow connections add --conn-type 'datahub_kafka' 'datahub_kafka_default' --conn-host 'broker:9092' --conn-extra '{}' - ``` +#### Setup a DataHub connection -4. Add your `datahub_conn_id` and/or `cluster` to your `airflow.cfg` file if it is not align with the default values. See configuration parameters below +You must configure an Airflow connection for Datahub. We support both a Datahub REST and a Kafka-based connections, but you only need one. - **Configuration options:** +```shell +# For REST-based: +airflow connections add --conn-type 'datahub_rest' 'datahub_rest_default' --conn-host 'http://datahub-gms:8080' --conn-password '' +# For Kafka-based (standard Kafka sink config can be passed via extras): +airflow connections add --conn-type 'datahub_kafka' 'datahub_kafka_default' --conn-host 'broker:9092' --conn-extra '{}' +``` - | Name | Default value | Description | - | ------------------------------ | -------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | - | datahub.enabled | true | If the plugin should be enabled. | - | datahub.conn_id | datahub_rest_default | The name of the datahub connection you set in step 1. | - | datahub.cluster | prod | name of the airflow cluster | - | datahub.capture_ownership_info | true | If true, the owners field of the DAG will be capture as a DataHub corpuser. | - | datahub.capture_tags_info | true | If true, the tags field of the DAG will be captured as DataHub tags. | - | datahub.capture_executions | true | If true, we'll capture task runs in DataHub in addition to DAG definitions. | - | datahub.graceful_exceptions | true | If set to true, most runtime errors in the lineage backend will be suppressed and will not cause the overall task to fail. Note that configuration issues will still throw exceptions. | +#### Configure the plugin -5. Configure `inlets` and `outlets` for your Airflow operators. For reference, look at the sample DAG in [`lineage_backend_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_demo.py), or reference [`lineage_backend_taskflow_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_taskflow_demo.py) if you're using the [TaskFlow API](https://airflow.apache.org/docs/apache-airflow/stable/concepts/taskflow.html). -6. [optional] Learn more about [Airflow lineage](https://airflow.apache.org/docs/apache-airflow/stable/lineage.html), including shorthand notation and some automation. +If your config doesn't align with the default values, you can configure the plugin in your `airflow.cfg` file. -### How to validate installation +```ini title="airflow.cfg" +[datahub] +enabled = true +conn_id = datahub_rest_default +# etc. +``` + +| Name | Default value | Description | +| ---------------------- | -------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| enabled | true | If the plugin should be enabled. | +| conn_id | datahub_rest_default | The name of the datahub connection you set in step 1. | +| cluster | prod | name of the airflow cluster | +| capture_ownership_info | true | If true, the owners field of the DAG will be capture as a DataHub corpuser. | +| capture_tags_info | true | If true, the tags field of the DAG will be captured as DataHub tags. | +| capture_executions | true | If true, we'll capture task runs in DataHub in addition to DAG definitions. | +| graceful_exceptions | true | If set to true, most runtime errors in the lineage backend will be suppressed and will not cause the overall task to fail. Note that configuration issues will still throw exceptions. | + +#### Validate that the plugin is working 1. Go and check in Airflow at Admin -> Plugins menu if you can see the DataHub plugin 2. Run an Airflow DAG. In the task logs, you should see Datahub related log messages like: @@ -78,9 +140,22 @@ lazy_load_plugins = False Emitting DataHub ... ``` -### Emitting lineage via a custom operator to the Airflow Plugin +## Manual Lineage Annotation + +### Using `inlets` and `outlets` -If you have created a custom Airflow operator [docs](https://airflow.apache.org/docs/apache-airflow/stable/howto/custom-operator.html) that inherits from the BaseOperator class, +You can manually annotate lineage by setting `inlets` and `outlets` on your Airflow operators. This is useful if you're using an operator that doesn't support automatic lineage extraction, or if you want to override the automatic lineage extraction. + +We have a few code samples that demonstrate how to use `inlets` and `outlets`: + +- [`lineage_backend_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_demo.py) +- [`lineage_backend_taskflow_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_taskflow_demo.py) - uses the [TaskFlow API](https://airflow.apache.org/docs/apache-airflow/stable/concepts/taskflow.html) + +For more information, take a look at the [Airflow lineage docs](https://airflow.apache.org/docs/apache-airflow/stable/lineage.html). + +### Custom Operators + +If you have created a [custom Airflow operator](https://airflow.apache.org/docs/apache-airflow/stable/howto/custom-operator.html) that inherits from the BaseOperator class, when overriding the `execute` function, set inlets and outlets via `context['ti'].task.inlets` and `context['ti'].task.outlets`. The DataHub Airflow plugin will then pick up those inlets and outlets after the task runs. @@ -91,7 +166,7 @@ class DbtOperator(BaseOperator): def execute(self, context): # do something inlets, outlets = self._get_lineage() - # inlets/outlets are lists of either datahub_provider.entities.Dataset or datahub_provider.entities.Urn + # inlets/outlets are lists of either datahub_airflow_plugin.entities.Dataset or datahub_airflow_plugin.entities.Urn context['ti'].task.inlets = self.inlets context['ti'].task.outlets = self.outlets @@ -101,78 +176,25 @@ class DbtOperator(BaseOperator): return inlets, outlets ``` -If you override the `pre_execute` and `post_execute` function, ensure they include the `@prepare_lineage` and `@apply_lineage` decorators respectively. [source](https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/lineage.html#lineage) - -## Using DataHub's Airflow lineage backend (deprecated) - -:::caution - -The DataHub Airflow plugin (above) is the recommended way to integrate Airflow with DataHub. For managed services like MWAA, the lineage backend is not supported and so you must use the Airflow plugin. - -If you're using Airflow 1.x, we recommend using the Airflow lineage backend with acryl-datahub <= 0.9.1.0. - -::: - -:::note - -If you are looking to run Airflow and DataHub using docker locally, follow the guide [here](../../docker/airflow/local_airflow.md). Otherwise proceed to follow the instructions below. -::: - -### Setting up Airflow to use DataHub as Lineage Backend - -1. You need to install the required dependency in your airflow. See - -```shell -pip install acryl-datahub[airflow] -# If you need the Kafka-based emitter/hook: -pip install acryl-datahub[airflow,datahub-kafka] -``` - -2. You must configure an Airflow hook for Datahub. We support both a Datahub REST hook and a Kafka-based hook, but you only need one. - - ```shell - # For REST-based: - airflow connections add --conn-type 'datahub_rest' 'datahub_rest_default' --conn-host 'http://datahub-gms:8080' --conn-password '' - # For Kafka-based (standard Kafka sink config can be passed via extras): - airflow connections add --conn-type 'datahub_kafka' 'datahub_kafka_default' --conn-host 'broker:9092' --conn-extra '{}' - ``` - -3. Add the following lines to your `airflow.cfg` file. +If you override the `pre_execute` and `post_execute` function, ensure they include the `@prepare_lineage` and `@apply_lineage` decorators respectively. Reference the [Airflow docs](https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/lineage.html#lineage) for more details. - ```ini title="airflow.cfg" - [lineage] - backend = datahub_provider.lineage.datahub.DatahubLineageBackend - datahub_kwargs = { - "enabled": true, - "datahub_conn_id": "datahub_rest_default", - "cluster": "prod", - "capture_ownership_info": true, - "capture_tags_info": true, - "graceful_exceptions": true } - # The above indentation is important! - ``` +## Emit Lineage Directly - **Configuration options:** +If you can't use the plugin or annotate inlets/outlets, you can also emit lineage using the `DatahubEmitterOperator`. - - `datahub_conn_id` (required): Usually `datahub_rest_default` or `datahub_kafka_default`, depending on what you named the connection in step 1. - - `cluster` (defaults to "prod"): The "cluster" to associate Airflow DAGs and tasks with. - - `capture_ownership_info` (defaults to true): If true, the owners field of the DAG will be capture as a DataHub corpuser. - - `capture_tags_info` (defaults to true): If true, the tags field of the DAG will be captured as DataHub tags. - - `capture_executions` (defaults to false): If true, it captures task runs as DataHub DataProcessInstances. - - `graceful_exceptions` (defaults to true): If set to true, most runtime errors in the lineage backend will be suppressed and will not cause the overall task to fail. Note that configuration issues will still throw exceptions. +Reference [`lineage_emission_dag.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py) for a full example. -4. Configure `inlets` and `outlets` for your Airflow operators. For reference, look at the sample DAG in [`lineage_backend_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_demo.py), or reference [`lineage_backend_taskflow_demo.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_backend_taskflow_demo.py) if you're using the [TaskFlow API](https://airflow.apache.org/docs/apache-airflow/stable/concepts/taskflow.html). -5. [optional] Learn more about [Airflow lineage](https://airflow.apache.org/docs/apache-airflow/stable/lineage.html), including shorthand notation and some automation. +In order to use this example, you must first configure the Datahub hook. Like in ingestion, we support a Datahub REST hook and a Kafka-based hook. See the plugin configuration for examples. -## Emitting lineage via a separate operator - -Take a look at this sample DAG: +## Debugging -- [`lineage_emission_dag.py`](../../metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py) - emits lineage using the DatahubEmitterOperator. +### Missing lineage -In order to use this example, you must first configure the Datahub hook. Like in ingestion, we support a Datahub REST hook and a Kafka-based hook. See step 1 above for details. +If you're not seeing lineage in DataHub, check the following: -## Debugging +- Validate that the plugin is loaded in Airflow. Go to Admin -> Plugins and check that the DataHub plugin is listed. +- If using the v2 plugin's automatic lineage, ensure that the `enable_extractors` config is set to true and that automatic lineage is supported for your operator. +- If using manual lineage annotation, ensure that you're using the `datahub_airflow_plugin.entities.Dataset` or `datahub_airflow_plugin.entities.Urn` classes for your inlets and outlets. ### Incorrect URLs @@ -180,9 +202,21 @@ If your URLs aren't being generated correctly (usually they'll start with `http: ```ini title="airflow.cfg" [webserver] -base_url = http://airflow.example.com +base_url = http://airflow.mycorp.example.com ``` +## Compatibility + +We no longer officially support Airflow <2.1. However, you can use older versions of `acryl-datahub-airflow-plugin` with older versions of Airflow. +Both of these options support Python 3.7+. + +- Airflow 1.10.x, use DataHub plugin v1 with acryl-datahub-airflow-plugin <= 0.9.1.0. +- Airflow 2.0.x, use DataHub plugin v1 with acryl-datahub-airflow-plugin <= 0.11.0.1. + +DataHub also previously supported an Airflow [lineage backend](https://airflow.apache.org/docs/apache-airflow/2.2.0/lineage.html#lineage-backend) implementation. While the implementation is still in our codebase, it is deprecated and will be removed in a future release. +Note that the lineage backend did not support automatic lineage extraction, did not capture task failures, and did not work in AWS MWAA. +The [documentation for the lineage backend](https://docs-website-1wmaehubl-acryldata.vercel.app/docs/lineage/airflow/#using-datahubs-airflow-lineage-backend-deprecated) has already been archived. + ## Additional references Related Datahub videos: diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 4ac18b525675b..0a4082eddd4a6 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -13,6 +13,10 @@ def get_long_description(): return pathlib.Path(os.path.join(root, "README.md")).read_text() +_version = package_metadata["__version__"] +_self_pin = f"=={_version}" if not _version.endswith("dev0") else "" + + rest_common = {"requests", "requests_file"} base_requirements = { @@ -27,8 +31,12 @@ def get_long_description(): "pydantic>=1.5.1", "apache-airflow >= 2.0.2", *rest_common, - f"acryl-datahub[sql-parser] == {package_metadata['__version__']}", - "openlineage-airflow==1.2.0", +} + +plugin_v2_requirements = { + # The v2 plugin requires Python 3.8+. + f"acryl-datahub[sql-parser]{_self_pin}", + "openlineage-airflow==1.2.0; python_version >= '3.8'", } @@ -89,7 +97,7 @@ def get_long_description(): setuptools.setup( # Package metadata. name=package_metadata["__package_name__"], - version=package_metadata["__version__"], + version=_version, url="https://datahubproject.io/", project_urls={ "Documentation": "https://datahubproject.io/docs/", @@ -133,11 +141,10 @@ def get_long_description(): install_requires=list(base_requirements), extras_require={ "dev": list(dev_requirements), - "datahub-kafka": [ - f"acryl-datahub[datahub-kafka] == {package_metadata['__version__']}" - ], + "datahub-kafka": [f"acryl-datahub[datahub-kafka]{_self_pin}"], + "plugin-v2": list(plugin_v2_requirements), "integration-tests": [ - f"acryl-datahub[datahub-kafka] == {package_metadata['__version__']}", + f"acryl-datahub[datahub-kafka]{_self_pin}", # Extra requirements for Airflow. "apache-airflow[snowflake]>=2.0.2", # snowflake is used in example dags # Because of https://github.com/snowflakedb/snowflake-sqlalchemy/issues/350 we need to restrict SQLAlchemy's max version. From 125e0d9225197d6df7c8742b84840ea2bf402af3 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 14:48:23 -0700 Subject: [PATCH 06/60] plugin v1 extra --- docs/lineage/airflow.md | 12 +++++------- metadata-ingestion-modules/airflow-plugin/setup.py | 1 + 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index 7927ae5b5bb0f..51f0c99917074 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -77,14 +77,12 @@ The SQL-related extractors have been updated to use DataHub's SQL parser, which The v1 plugin requires Airflow 2.1+ and Python 3.8+. If you're on older versions, it's still possible to use an older version of the plugin. See the [compatibility section](#compatibility) for more details. ```shell -pip install acryl-datahub-airflow-plugin -``` - -:::note +pip install acryl-datahub-airflow-plugin[plugin-v1] -The [DataHub Rest](../../metadata-ingestion/sink_docs/datahub.md#datahub-rest) emitter is included in the plugin package by default. To use [DataHub Kafka](../../metadata-ingestion/sink_docs/datahub.md#datahub-kafka) install `pip install acryl-datahub-airflow-plugin[datahub-kafka]`. - -::: +# The DataHub rest connection type is included by default. +# To use the DataHub Kafka connection type, install the plugin with the kafka extras. +pip install acryl-datahub-airflow-plugin[plugin-v1,datahub-kafka] +``` diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 0a4082eddd4a6..5608206ec7606 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -142,6 +142,7 @@ def get_long_description(): extras_require={ "dev": list(dev_requirements), "datahub-kafka": [f"acryl-datahub[datahub-kafka]{_self_pin}"], + "plugin-v1": [], "plugin-v2": list(plugin_v2_requirements), "integration-tests": [ f"acryl-datahub[datahub-kafka]{_self_pin}", From 0423ff3c778a60d3337bbb7dcc7090316f024de7 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 14:53:45 -0700 Subject: [PATCH 07/60] update ci --- .github/workflows/airflow-plugin.yml | 21 ++++++++++++------- .../airflow-plugin/build.gradle | 7 +++++-- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/.github/workflows/airflow-plugin.yml b/.github/workflows/airflow-plugin.yml index dbea2fa1a112f..5371b93f77b07 100644 --- a/.github/workflows/airflow-plugin.yml +++ b/.github/workflows/airflow-plugin.yml @@ -33,15 +33,20 @@ jobs: matrix: include: - python-version: "3.8" - extraPythonRequirement: "apache-airflow~=2.1.4" + extra_pip_requirements: "apache-airflow~=2.1.4" + extra_pip_extras: plugin-v1 - python-version: "3.8" - extraPythonRequirement: "apache-airflow~=2.2.4" + extra_pip_requirements: "apache-airflow~=2.2.4" + extra_pip_extras: plugin-v1 - python-version: "3.10" - extraPythonRequirement: "apache-airflow~=2.4.0" + extra_pip_requirements: "apache-airflow~=2.4.0" + extra_pip_extras: plugin-v2 - python-version: "3.10" - extraPythonRequirement: "apache-airflow~=2.6.0" + extra_pip_requirements: "apache-airflow~=2.6.0" + extra_pip_extras: plugin-v2 - python-version: "3.10" - extraPythonRequirement: "apache-airflow>2.6.0" + extra_pip_requirements: "apache-airflow>2.6.0" + extra_pip_extras: plugin-v2 fail-fast: false steps: - uses: actions/checkout@v3 @@ -51,13 +56,13 @@ jobs: cache: "pip" - name: Install dependencies run: ./metadata-ingestion/scripts/install_deps.sh - - name: Install airflow package and test (extras ${{ matrix.extraPythonRequirement }}) - run: ./gradlew -Pextra_pip_requirements='${{ matrix.extraPythonRequirement }}' :metadata-ingestion-modules:airflow-plugin:lint :metadata-ingestion-modules:airflow-plugin:testQuick + - name: Install airflow package and test (extras ${{ matrix.extra_pip_requirements }}) + run: ./gradlew -Pextra_pip_requirements='${{ matrix.extra_pip_requirements }}' -Pextra_pip_extras='${{ matrix.extra_pip_extras }}' :metadata-ingestion-modules:airflow-plugin:lint :metadata-ingestion-modules:airflow-plugin:testQuick - name: pip freeze show list installed if: always() run: source metadata-ingestion-modules/airflow-plugin/venv/bin/activate && pip freeze - uses: actions/upload-artifact@v3 - if: ${{ always() && matrix.python-version == '3.10' && matrix.extraPythonRequirement == 'apache-airflow>2.6.0' }} + if: ${{ always() && matrix.python-version == '3.10' && matrix.extra_pip_requirements == 'apache-airflow>2.6.0' }} with: name: Test Results (Airflow Plugin ${{ matrix.python-version}}) path: | diff --git a/metadata-ingestion-modules/airflow-plugin/build.gradle b/metadata-ingestion-modules/airflow-plugin/build.gradle index 58a2bc9e670e3..15d7f60772bfe 100644 --- a/metadata-ingestion-modules/airflow-plugin/build.gradle +++ b/metadata-ingestion-modules/airflow-plugin/build.gradle @@ -10,6 +10,9 @@ ext { if (!project.hasProperty("extra_pip_requirements")) { ext.extra_pip_requirements = "" } +if (!project.hasProperty("extra_pip_extras")) { + ext.extra_pip_extras = "" +} def pip_install_command = "${venv_name}/bin/pip install -e ../../metadata-ingestion" @@ -36,7 +39,7 @@ task installPackage(type: Exec, dependsOn: [environmentSetup, ':metadata-ingesti // and https://github.com/datahub-project/datahub/pull/8435. commandLine 'bash', '-x', '-c', "${pip_install_command} install 'Cython<3.0' 'PyYAML<6' --no-build-isolation && " + - "${pip_install_command} -e . ${extra_pip_requirements} &&" + + "${pip_install_command} -e .[${extra_pip_extras}] ${extra_pip_requirements} &&" + "touch ${sentinel_file}" } @@ -47,7 +50,7 @@ task installDev(type: Exec, dependsOn: [install]) { inputs.file file('setup.py') outputs.file("${sentinel_file}") commandLine 'bash', '-x', '-c', - "${pip_install_command} -e .[dev] ${extra_pip_requirements} && " + + "${pip_install_command} -e .[dev,${extra_pip_extras}] ${extra_pip_requirements} && " + "touch ${sentinel_file}" } From e6dab0629805433dfe6f6e17506420a54e498cac Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 20:13:34 -0700 Subject: [PATCH 08/60] update mypy + build --- docs/lineage/airflow.md | 2 ++ .../airflow-plugin/build.gradle | 13 +++++++++---- .../src/datahub_airflow_plugin/datahub_listener.py | 12 +++++++++--- .../src/datahub_airflow_plugin/datahub_plugin.py | 8 ++++++-- 4 files changed, 26 insertions(+), 9 deletions(-) diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index 51f0c99917074..a44d6b86acdfb 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -76,6 +76,8 @@ The SQL-related extractors have been updated to use DataHub's SQL parser, which The v1 plugin requires Airflow 2.1+ and Python 3.8+. If you're on older versions, it's still possible to use an older version of the plugin. See the [compatibility section](#compatibility) for more details. +If you're using Airflow 2.3+, we recommend using the v2 plugin instead. If you need to use the v1 plugin with Airflow 2.3+, you must also set the environment variable `DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN=true`. + ```shell pip install acryl-datahub-airflow-plugin[plugin-v1] diff --git a/metadata-ingestion-modules/airflow-plugin/build.gradle b/metadata-ingestion-modules/airflow-plugin/build.gradle index 15d7f60772bfe..64b86a9276621 100644 --- a/metadata-ingestion-modules/airflow-plugin/build.gradle +++ b/metadata-ingestion-modules/airflow-plugin/build.gradle @@ -11,7 +11,11 @@ if (!project.hasProperty("extra_pip_requirements")) { ext.extra_pip_requirements = "" } if (!project.hasProperty("extra_pip_extras")) { - ext.extra_pip_extras = "" + ext.extra_pip_extras = "plugin-v2" +} +// If extra_pip_extras is non-empty, we need to add a comma to the beginning of the string. +if (extra_pip_extras != "") { + ext.extra_pip_extras = "," + extra_pip_extras } def pip_install_command = "${venv_name}/bin/pip install -e ../../metadata-ingestion" @@ -39,7 +43,7 @@ task installPackage(type: Exec, dependsOn: [environmentSetup, ':metadata-ingesti // and https://github.com/datahub-project/datahub/pull/8435. commandLine 'bash', '-x', '-c', "${pip_install_command} install 'Cython<3.0' 'PyYAML<6' --no-build-isolation && " + - "${pip_install_command} -e .[${extra_pip_extras}] ${extra_pip_requirements} &&" + + "${pip_install_command} -e .[ignore${extra_pip_extras}] ${extra_pip_requirements} &&" + "touch ${sentinel_file}" } @@ -50,7 +54,7 @@ task installDev(type: Exec, dependsOn: [install]) { inputs.file file('setup.py') outputs.file("${sentinel_file}") commandLine 'bash', '-x', '-c', - "${pip_install_command} -e .[dev,${extra_pip_extras}] ${extra_pip_requirements} && " + + "${pip_install_command} -e .[dev${extra_pip_extras}] ${extra_pip_requirements} && " + "touch ${sentinel_file}" } @@ -82,7 +86,8 @@ task installDevTest(type: Exec, dependsOn: [installDev]) { outputs.dir("${venv_name}") outputs.file("${sentinel_file}") commandLine 'bash', '-x', '-c', - "${pip_install_command} -e .[dev,integration-tests] && touch ${sentinel_file}" + "${pip_install_command} -e .[dev,integration-tests${extra_pip_extras}] ${extra_pip_requirements} && " + + "touch ${sentinel_file}" } def testFile = hasProperty('testFile') ? testFile : 'unknown' diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 55ade19a63371..7268241f3bb0b 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -36,12 +36,21 @@ from airflow.models import DAG, DagRun, TaskInstance from sqlalchemy.orm import Session + logger = logging.getLogger(__name__) +_F = TypeVar("_F", bound=Callable[..., None]) _airflow_listener_initialized = False _airflow_listener: Optional["DataHubListener"] = None _RUN_IN_THREAD = True +if TYPE_CHECKING: + # On Airflow versions that don't have the listener API, we placate mypy + # by making hookimpl an identity function. + + def hookimpl(f: _F) -> _F: # type: ignore[misc] # noqa: F811 + return f + def get_airflow_plugin_listener() -> Optional["DataHubListener"]: # Using globals instead of functools.lru_cache to make testing easier. @@ -65,9 +74,6 @@ def get_airflow_plugin_listener() -> Optional["DataHubListener"]: return _airflow_listener -_F = TypeVar("_F", bound=Callable[..., None]) - - def run_in_thread(f: _F) -> _F: @functools.wraps(f) def wrapper(*args, **kwargs): diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index c3db85c3a127d..a1e3fd3941d41 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -1,5 +1,7 @@ import logging import os +from types import ModuleType +from typing import List from airflow.plugins_manager import AirflowPlugin @@ -11,7 +13,7 @@ _USE_AIRFLOW_LISTENER_INTERFACE = IS_AIRFLOW_V23_PLUS and not os.getenv( - "DATAHUB_AIRFLOW_PLUGIN_USE_LEGACY_PLUGIN_POLICY", "false" + "DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN", "false" ).lower() in ("true", "1") @@ -23,7 +25,9 @@ class DatahubPlugin(AirflowPlugin): get_airflow_plugin_listener, ) - listeners = list(filter(None, [get_airflow_plugin_listener()])) + listeners: List[ModuleType] = list( + filter(None, [get_airflow_plugin_listener()]) + ) if not _USE_AIRFLOW_LISTENER_INTERFACE: From aa84b234fc54dc1489ab4be7c5175b912d0b0e84 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 20:18:14 -0700 Subject: [PATCH 09/60] fix(ingest): bump typing-extensions --- metadata-ingestion/setup.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index a0d16aa92ad9b..3367e2ada5bc1 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -18,9 +18,7 @@ def get_long_description(): base_requirements = { - # Typing extension should be >=3.10.0.2 ideally but we can't restrict due to Airflow 2.0.2 dependency conflict - "typing_extensions>=3.7.4.3 ; python_version < '3.8'", - "typing_extensions>=3.10.0.2,<4.6.0 ; python_version >= '3.8'", + "typing_extensions>=3.10.0.2", "mypy_extensions>=0.4.3", # Actual dependencies. "typing-inspect", From 2ac887b195e7540a0540d5e8d233aaef2bb4c32a Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 21:37:34 -0700 Subject: [PATCH 10/60] unify on emit method --- .../datahub_airflow_plugin/hooks/datahub.py | 75 ++++++++++++------- .../operators/datahub.py | 4 +- .../airflow-plugin/tests/unit/test_airflow.py | 8 +- 3 files changed, 52 insertions(+), 35 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index 8fb7363f8cad1..277d44697a108 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -2,6 +2,7 @@ from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook +from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.metadata.com.linkedin.pegasus2avro.mxe import ( MetadataChangeEvent, MetadataChangeProposal, @@ -80,17 +81,24 @@ def make_emitter(self) -> "DatahubRestEmitter": return datahub.emitter.rest_emitter.DatahubRestEmitter(*self._get_config()) - def emit_mces(self, mces: List[MetadataChangeEvent]) -> None: + def emit( + self, + items: List[ + Union[ + MetadataChangeEvent, + MetadataChangeProposal, + MetadataChangeProposalWrapper, + ] + ], + ) -> None: emitter = self.make_emitter() - for mce in mces: - emitter.emit_mce(mce) + for item in items: + emitter.emit(item) - def emit_mcps(self, mcps: List[MetadataChangeProposal]) -> None: - emitter = self.make_emitter() - - for mce in mcps: - emitter.emit_mcp(mce) + # Retained for backwards compatibility. + emit_mces = emit + emit_mcps = emit class DatahubKafkaHook(BaseHook): @@ -152,7 +160,16 @@ def make_emitter(self) -> "DatahubKafkaEmitter": sink_config = self._get_config() return datahub.emitter.kafka_emitter.DatahubKafkaEmitter(sink_config) - def emit_mces(self, mces: List[MetadataChangeEvent]) -> None: + def emit( + self, + items: List[ + Union[ + MetadataChangeEvent, + MetadataChangeProposal, + MetadataChangeProposalWrapper, + ] + ], + ) -> None: emitter = self.make_emitter() errors = [] @@ -160,29 +177,17 @@ def callback(exc, msg): if exc: errors.append(exc) - for mce in mces: - emitter.emit_mce_async(mce, callback) + for mce in items: + emitter.emit(mce, callback) emitter.flush() if errors: - raise AirflowException(f"failed to push some MCEs: {errors}") + raise AirflowException(f"failed to push some metadata: {errors}") - def emit_mcps(self, mcps: List[MetadataChangeProposal]) -> None: - emitter = self.make_emitter() - errors = [] - - def callback(exc, msg): - if exc: - errors.append(exc) - - for mcp in mcps: - emitter.emit_mcp_async(mcp, callback) - - emitter.flush() - - if errors: - raise AirflowException(f"failed to push some MCPs: {errors}") + # Retained for backwards compatibility. + emit_mces = emit + emit_mcps = emit class DatahubGenericHook(BaseHook): @@ -225,5 +230,17 @@ def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: def make_emitter(self) -> Union["DatahubRestEmitter", "DatahubKafkaEmitter"]: return self.get_underlying_hook().make_emitter() - def emit_mces(self, mces: List[MetadataChangeEvent]) -> None: - return self.get_underlying_hook().emit_mces(mces) + def emit( + self, + items: List[ + Union[ + MetadataChangeEvent, + MetadataChangeProposal, + MetadataChangeProposalWrapper, + ] + ], + ) -> None: + return self.get_underlying_hook().emit(items) + + # Retained for backwards compatibility. + emit_mces = emit diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/operators/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/operators/datahub.py index 109e7ddfe4dfa..15b50c51a561d 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/operators/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/operators/datahub.py @@ -57,7 +57,7 @@ def __init__( # type: ignore[no-untyped-def] datahub_conn_id=datahub_conn_id, **kwargs, ) - self.mces = mces + self.metadata = mces def execute(self, context): - self.generic_hook.get_underlying_hook().emit_mces(self.mces) + self.generic_hook.get_underlying_hook().emit(self.metadata) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py index 9aa901171cfa6..9fe1ec443cf38 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py @@ -105,7 +105,7 @@ def test_datahub_rest_hook(mock_emitter): mock_emitter.assert_called_once_with(config.host, None, None) instance = mock_emitter.return_value - instance.emit_mce.assert_called_with(lineage_mce) + instance.emit.assert_called_with(lineage_mce) @mock.patch("datahub.emitter.rest_emitter.DatahubRestEmitter", autospec=True) @@ -119,7 +119,7 @@ def test_datahub_rest_hook_with_timeout(mock_emitter): mock_emitter.assert_called_once_with(config.host, None, 5) instance = mock_emitter.return_value - instance.emit_mce.assert_called_with(lineage_mce) + instance.emit.assert_called_with(lineage_mce) @mock.patch("datahub.emitter.kafka_emitter.DatahubKafkaEmitter", autospec=True) @@ -131,11 +131,11 @@ def test_datahub_kafka_hook(mock_emitter): mock_emitter.assert_called_once() instance = mock_emitter.return_value - instance.emit_mce_async.assert_called() + instance.emit.assert_called() instance.flush.assert_called_once() -@mock.patch("datahub_provider.hooks.datahub.DatahubRestHook.emit_mces") +@mock.patch("datahub_provider.hooks.datahub.DatahubRestHook.emit") def test_datahub_lineage_operator(mock_emit): with patch_airflow_connection(datahub_rest_connection_config) as config: assert config.conn_id From c711a7e1e83dd34323c2ed5e17d6d217efe18658 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Mon, 25 Sep 2023 21:44:24 -0700 Subject: [PATCH 11/60] add simple unit test --- .../airflow-plugin/tests/unit/test_dummy.py | 2 -- .../airflow-plugin/tests/unit/test_packaging.py | 8 ++++++++ metadata-ingestion/build.gradle | 2 +- 3 files changed, 9 insertions(+), 3 deletions(-) delete mode 100644 metadata-ingestion-modules/airflow-plugin/tests/unit/test_dummy.py create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/unit/test_packaging.py diff --git a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_dummy.py b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_dummy.py deleted file mode 100644 index 10cf3ad0a608a..0000000000000 --- a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_dummy.py +++ /dev/null @@ -1,2 +0,0 @@ -def test_dummy(): - pass diff --git a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_packaging.py b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_packaging.py new file mode 100644 index 0000000000000..1d0ce5835f958 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_packaging.py @@ -0,0 +1,8 @@ +import setuptools + + +def test_package_list_match_inits(): + where = "./src" + package_list = set(setuptools.find_packages(where)) + namespace_packages = set(setuptools.find_namespace_packages(where)) + assert package_list == namespace_packages, "are you missing a package init file?" diff --git a/metadata-ingestion/build.gradle b/metadata-ingestion/build.gradle index ea7990ab9c660..b0ef13a99fe89 100644 --- a/metadata-ingestion/build.gradle +++ b/metadata-ingestion/build.gradle @@ -12,7 +12,7 @@ if (!project.hasProperty("extra_pip_requirements")) { } def get_coverage_arg(test_name) { - return "--cov-report term --cov-report xml:coverage_${test_name}.xml " + return "--cov-report xml:coverage_${test_name}.xml " } task checkPythonVersion(type: Exec) { From f73a380406ba862b5ccff8d4c52c1d45f96d4cc6 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 11:20:21 -0700 Subject: [PATCH 12/60] start tests --- .../src/datahub_airflow_plugin/hooks/datahub.py | 8 ++++---- .../tests/integration/integration_test_dummy.py | 2 -- 2 files changed, 4 insertions(+), 6 deletions(-) delete mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/integration_test_dummy.py diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index 277d44697a108..6871ed68ecda8 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -1,4 +1,4 @@ -from typing import TYPE_CHECKING, Any, Dict, List, Optional, Tuple, Union +from typing import TYPE_CHECKING, Any, Dict, List, Optional, Sequence, Tuple, Union from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook @@ -83,7 +83,7 @@ def make_emitter(self) -> "DatahubRestEmitter": def emit( self, - items: List[ + items: Sequence[ Union[ MetadataChangeEvent, MetadataChangeProposal, @@ -162,7 +162,7 @@ def make_emitter(self) -> "DatahubKafkaEmitter": def emit( self, - items: List[ + items: Sequence[ Union[ MetadataChangeEvent, MetadataChangeProposal, @@ -232,7 +232,7 @@ def make_emitter(self) -> Union["DatahubRestEmitter", "DatahubKafkaEmitter"]: def emit( self, - items: List[ + items: Sequence[ Union[ MetadataChangeEvent, MetadataChangeProposal, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/integration_test_dummy.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/integration_test_dummy.py deleted file mode 100644 index 10cf3ad0a608a..0000000000000 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/integration_test_dummy.py +++ /dev/null @@ -1,2 +0,0 @@ -def test_dummy(): - pass From 048b6e9d76a70a9448b7aacd370922374950d946 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 12:34:14 -0700 Subject: [PATCH 13/60] loosen airflow req --- metadata-ingestion-modules/airflow-plugin/setup.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 5608206ec7606..d406b958e99a4 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -22,12 +22,8 @@ def get_long_description(): base_requirements = { # Compatibility. "dataclasses>=0.6; python_version < '3.7'", - # Typing extension should be >=3.10.0.2 ideally but we can't restrict due to Airflow 2.0.2 dependency conflict - "typing_extensions>=3.7.4.3 ; python_version < '3.8'", - "typing_extensions>=3.10.0.2,<4.6.0 ; python_version >= '3.8'", "mypy_extensions>=0.4.3", # Actual dependencies. - "typing-inspect", "pydantic>=1.5.1", "apache-airflow >= 2.0.2", *rest_common, @@ -54,8 +50,6 @@ def get_long_description(): # versions 0.1.13 and 0.1.14 seem to have issues "types-click==0.1.12", "types-tabulate", - # avrogen package requires this - "types-pytz", } base_dev_requirements = { From 9c8538be3e4a2ce98899f0b8c2fe7260de76802f Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 15:34:06 -0700 Subject: [PATCH 14/60] start testing setup --- .../airflow-plugin/setup.py | 1 + .../tests/integration/test_plugin.py | 92 +++++++++++++++++++ 2 files changed, 93 insertions(+) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index d406b958e99a4..83dd13f6d1ec5 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -69,6 +69,7 @@ def get_long_description(): "pytest-cov>=2.8.1", "tox", "deepdiff", + "tenacity", "requests-mock", "freezegun", "jsonpickle", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py new file mode 100644 index 0000000000000..3dea9208ce0c6 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -0,0 +1,92 @@ +import contextlib +import logging +import os +import pathlib +import random +import signal +import subprocess +import time +from typing import Iterator + +import requests +import tenacity + +logger = logging.getLogger(__name__) + + +@tenacity.retry( + reraise=True, + wait=tenacity.wait_fixed(1), + stop=tenacity.stop_after_delay(60), + retry=tenacity.retry_if_exception_type( + AssertionError, requests.exceptions.RequestException + ), +) +def _wait_for_airflow_healthy(airflow_port: int) -> None: + print("Checking if Airflow is ready...") + res = requests.get(f"http://localhost:{airflow_port}/health", timeout=5) + res.raise_for_status() + + airflow_health = res.json() + assert airflow_health["metadatabase"]["status"] == "healthy" + assert airflow_health["scheduler"]["status"] == "healthy" + + +@contextlib.contextmanager +def _run_airflow(tmp_path: pathlib.Path, dags_folder: pathlib.Path) -> Iterator[None]: + airflow_home = tmp_path / "airflow_home" + print(f"Using airflow home: {airflow_home}") + + airflow_port = random.randint(10000, 12000) + print(f"Using airflow port: {airflow_port}") + + environment = { + **os.environ, + "AIRFLOW_HOME": str(airflow_home), + "AIRFLOW__WEBSERVER__WEB_SERVER_PORT": str(airflow_port), + "AIRFLOW__CORE__LOAD_EXAMPLES": "False", + "AIRFLOW__CORE__DAGS_FOLDER": str(dags_folder), + # TODO change the datahub plugin connection id + # TODO set up a datahub file hook + } + + # Start airflow in a background subprocess. + airflow_process = subprocess.Popen( + ["airflow", "standalone"], + env=environment, + ) + + try: + _wait_for_airflow_healthy(airflow_port) + print("Airflow is ready!") + + airflow_username = "admin" + airflow_password = (airflow_home / "standalone_admin_password.txt").read_text() + + yield + finally: + # Attempt a graceful shutdown. + print("Shutting down airflow...") + airflow_process.send_signal(signal.SIGINT) + airflow_process.wait(timeout=30) + + # If the graceful shutdown failed, kill the process. + airflow_process.kill() + airflow_process.wait(timeout=3) + + +def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: + # This test: + # - Configures the plugin. + # - Starts a local airflow instance in a subprocess. + # - Runs a DAG that uses an operator supported by the extractor. + # - Waits for the DAG to complete. + # - Checks that the metadata was emitted to DataHub. + + # TODO cleanup old running airflow instances + + dags_folder = pathlib.Path(__file__).parent / "dags" + + with _run_airflow(tmp_path, dags_folder=dags_folder): + breakpoint() + print("done") From 1f8f7a7923b298cfd177b56c5b5e4864db3a3315 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 15:49:12 -0700 Subject: [PATCH 15/60] add an emitter interface --- .../client/airflow_generator.py | 17 +++++------ .../datahub_airflow_plugin/hooks/datahub.py | 3 +- .../datahub/api/entities/datajob/dataflow.py | 19 ++---------- .../datahub/api/entities/datajob/datajob.py | 9 ++---- .../src/datahub/emitter/generic_emitter.py | 29 +++++++++++++++++++ .../src/datahub/emitter/kafka_emitter.py | 3 +- .../src/datahub/emitter/rest_emitter.py | 6 ++-- 7 files changed, 49 insertions(+), 37 deletions(-) create mode 100644 metadata-ingestion/src/datahub/emitter/generic_emitter.py diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index 8dc84f9d4e46f..6041a8c1c0a9f 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -1,5 +1,5 @@ from datetime import datetime -from typing import TYPE_CHECKING, Dict, List, Optional, Set, Union, cast +from typing import TYPE_CHECKING, Dict, List, Optional, Set, cast from airflow.configuration import conf from datahub.api.entities.datajob import DataFlow, DataJob @@ -7,6 +7,7 @@ DataProcessInstance, InstanceRunResult, ) +from datahub.emitter.generic_emitter import Emitter from datahub.metadata.schema_classes import DataProcessTypeClass from datahub.utilities.urns.data_flow_urn import DataFlowUrn from datahub.utilities.urns.data_job_urn import DataJobUrn @@ -18,8 +19,6 @@ if TYPE_CHECKING: from airflow import DAG from airflow.models import DagRun, TaskInstance - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - from datahub.emitter.rest_emitter import DatahubRestEmitter from datahub_airflow_plugin._airflow_shims import Operator @@ -289,7 +288,7 @@ def create_datajob_instance( @staticmethod def run_dataflow( - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, cluster: str, dag_run: "DagRun", start_timestamp_millis: Optional[int] = None, @@ -341,7 +340,7 @@ def run_dataflow( @staticmethod def complete_dataflow( - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, cluster: str, dag_run: "DagRun", end_timestamp_millis: Optional[int] = None, @@ -349,7 +348,7 @@ def complete_dataflow( ) -> None: """ - :param emitter: DatahubRestEmitter - the datahub rest emitter to emit the generated mcps + :param emitter: Emitter - the datahub emitter to emit the generated mcps :param cluster: str - name of the cluster :param dag_run: DagRun :param end_timestamp_millis: Optional[int] - the completion time in milliseconds if not set the current time will be used. @@ -387,7 +386,7 @@ def complete_dataflow( @staticmethod def run_datajob( - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, cluster: str, ti: "TaskInstance", dag: "DAG", @@ -461,7 +460,7 @@ def run_datajob( @staticmethod def complete_datajob( - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, cluster: str, ti: "TaskInstance", dag: "DAG", @@ -472,7 +471,7 @@ def complete_datajob( ) -> DataProcessInstance: """ - :param emitter: DatahubRestEmitter + :param emitter: Emitter - the datahub emitter to emit the generated mcps :param cluster: str :param ti: TaskInstance :param dag: DAG diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index 6871ed68ecda8..1851cad5b8956 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -2,6 +2,7 @@ from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.metadata.com.linkedin.pegasus2avro.mxe import ( MetadataChangeEvent, @@ -227,7 +228,7 @@ def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: f"DataHub cannot handle conn_type {conn.conn_type} in {conn}" ) - def make_emitter(self) -> Union["DatahubRestEmitter", "DatahubKafkaEmitter"]: + def make_emitter(self) -> Emitter: return self.get_underlying_hook().make_emitter() def emit( diff --git a/metadata-ingestion/src/datahub/api/entities/datajob/dataflow.py b/metadata-ingestion/src/datahub/api/entities/datajob/dataflow.py index 8a04768bc0a72..acd708ee81a5c 100644 --- a/metadata-ingestion/src/datahub/api/entities/datajob/dataflow.py +++ b/metadata-ingestion/src/datahub/api/entities/datajob/dataflow.py @@ -1,18 +1,9 @@ import logging from dataclasses import dataclass, field -from typing import ( - TYPE_CHECKING, - Callable, - Dict, - Iterable, - List, - Optional, - Set, - Union, - cast, -) +from typing import Callable, Dict, Iterable, List, Optional, Set, cast import datahub.emitter.mce_builder as builder +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.metadata.schema_classes import ( AuditStampClass, @@ -29,10 +20,6 @@ ) from datahub.utilities.urns.data_flow_urn import DataFlowUrn -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - from datahub.emitter.rest_emitter import DatahubRestEmitter - logger = logging.getLogger(__name__) @@ -170,7 +157,7 @@ def generate_mcp(self) -> Iterable[MetadataChangeProposalWrapper]: def emit( self, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ diff --git a/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py b/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py index 3cd23f63c1ab7..0face6415bacc 100644 --- a/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py +++ b/metadata-ingestion/src/datahub/api/entities/datajob/datajob.py @@ -1,7 +1,8 @@ from dataclasses import dataclass, field -from typing import TYPE_CHECKING, Callable, Dict, Iterable, List, Optional, Set, Union +from typing import Callable, Dict, Iterable, List, Optional, Set import datahub.emitter.mce_builder as builder +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.metadata.schema_classes import ( AuditStampClass, @@ -22,10 +23,6 @@ from datahub.utilities.urns.data_job_urn import DataJobUrn from datahub.utilities.urns.dataset_urn import DatasetUrn -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - from datahub.emitter.rest_emitter import DatahubRestEmitter - @dataclass class DataJob: @@ -134,7 +131,7 @@ def generate_mcp(self) -> Iterable[MetadataChangeProposalWrapper]: def emit( self, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ diff --git a/metadata-ingestion/src/datahub/emitter/generic_emitter.py b/metadata-ingestion/src/datahub/emitter/generic_emitter.py new file mode 100644 index 0000000000000..2447ce051c56b --- /dev/null +++ b/metadata-ingestion/src/datahub/emitter/generic_emitter.py @@ -0,0 +1,29 @@ +from typing import Any, Callable, Optional, Protocol, Union + +from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.metadata.com.linkedin.pegasus2avro.mxe import ( + MetadataChangeEvent, + MetadataChangeProposal, +) + + +class Emitter(Protocol): + def emit( + self, + item: Union[ + MetadataChangeEvent, + MetadataChangeProposal, + MetadataChangeProposalWrapper, + ], + # NOTE: This signature should have the exception be optional rather than + # required. However, this would be a breaking change that may need + # more careful consideration. + callback: Optional[Callable[[Exception, str], None]] = None, + # TODO: The rest emitter returns timestamps as the return type. For now + # we smooth over that detail using Any, but eventually we should + # standardize on a return type. + ) -> Any: + raise NotImplementedError + + def flush(self) -> None: + pass diff --git a/metadata-ingestion/src/datahub/emitter/kafka_emitter.py b/metadata-ingestion/src/datahub/emitter/kafka_emitter.py index ec0c8f3418a4a..781930011b78f 100644 --- a/metadata-ingestion/src/datahub/emitter/kafka_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/kafka_emitter.py @@ -10,6 +10,7 @@ from datahub.configuration.common import ConfigModel from datahub.configuration.kafka import KafkaProducerConnectionConfig from datahub.configuration.validate_field_rename import pydantic_renamed_field +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.ingestion.api.closeable import Closeable from datahub.metadata.schema_classes import ( @@ -55,7 +56,7 @@ def validate_topic_routes(cls, v: Dict[str, str]) -> Dict[str, str]: return v -class DatahubKafkaEmitter(Closeable): +class DatahubKafkaEmitter(Closeable, Emitter): def __init__(self, config: KafkaEmitterConfig): self.config = config schema_registry_conf = { diff --git a/metadata-ingestion/src/datahub/emitter/rest_emitter.py b/metadata-ingestion/src/datahub/emitter/rest_emitter.py index c25d42d10b84e..afb19df9791af 100644 --- a/metadata-ingestion/src/datahub/emitter/rest_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/rest_emitter.py @@ -13,6 +13,7 @@ from datahub.cli.cli_utils import get_system_auth from datahub.configuration.common import ConfigurationError, OperationalError +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.request_helper import make_curl_command from datahub.emitter.serialization_helper import pre_json_transform @@ -45,7 +46,7 @@ ) -class DataHubRestEmitter(Closeable): +class DataHubRestEmitter(Closeable, Emitter): _gms_server: str _token: Optional[str] _session: requests.Session @@ -206,9 +207,6 @@ def emit( MetadataChangeProposalWrapper, UsageAggregation, ], - # NOTE: This signature should have the exception be optional rather than - # required. However, this would be a breaking change that may need - # more careful consideration. callback: Optional[Callable[[Exception, str], None]] = None, ) -> Tuple[datetime.datetime, datetime.datetime]: start_time = datetime.datetime.now() From 8b0ef4c1d850dda736dc27b89ffbf541ddafed47 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 19:46:25 -0700 Subject: [PATCH 16/60] sync file emitter --- .../airflow-plugin/setup.py | 8 ++-- .../datahub_airflow_plugin/hooks/datahub.py | 20 ++++++++- metadata-ingestion/setup.py | 1 + .../emitter/synchronized_file_emitter.py | 45 +++++++++++++++++++ 4 files changed, 68 insertions(+), 6 deletions(-) create mode 100644 metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 83dd13f6d1ec5..e56b14ee38127 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -52,7 +52,7 @@ def get_long_description(): "types-tabulate", } -base_dev_requirements = { +dev_requirements = { *base_requirements, *mypy_stubs, "black==22.12.0", @@ -76,10 +76,7 @@ def get_long_description(): "build", "twine", "packaging", -} - -dev_requirements = { - *base_dev_requirements, + f"acryl-datahub[sync-file-emitter,testing-utils]{_self_pin}", } @@ -137,6 +134,7 @@ def get_long_description(): extras_require={ "dev": list(dev_requirements), "datahub-kafka": [f"acryl-datahub[datahub-kafka]{_self_pin}"], + "datahub-file": [f"acryl-datahub[sync-file-emitter]{_self_pin}"], "plugin-v1": [], "plugin-v2": list(plugin_v2_requirements), "integration-tests": [ diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index 1851cad5b8956..b7f56c18d00d1 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -1,4 +1,4 @@ -from typing import TYPE_CHECKING, Any, Dict, List, Optional, Sequence, Tuple, Union +from typing import TYPE_CHECKING, Any, Dict, Optional, Sequence, Tuple, Union from airflow.exceptions import AirflowException from airflow.hooks.base import BaseHook @@ -13,6 +13,7 @@ from airflow.models.connection import Connection from datahub.emitter.kafka_emitter import DatahubKafkaEmitter from datahub.emitter.rest_emitter import DatahubRestEmitter + from datahub.emitter.synchronized_file_emitter import SynchronizedFileEmitter from datahub.ingestion.sink.datahub_kafka import KafkaSinkConfig @@ -219,6 +220,8 @@ def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: or conn.conn_type == DatahubKafkaHook.conn_type.replace("-", "_") ): return DatahubKafkaHook(self.datahub_conn_id) + elif conn.conn_type == SynchronizedFileEmitter.conn_type: + return SynchronizedFileEmitter(self.datahub_conn_id) elif "rest" in self.datahub_conn_id: return DatahubRestHook(self.datahub_conn_id) elif "kafka" in self.datahub_conn_id: @@ -245,3 +248,18 @@ def emit( # Retained for backwards compatibility. emit_mces = emit + + +class SynchronizedFileHook(BaseHook): + conn_type = "datahub-file" + + def __init__(self, datahub_conn_id: str) -> None: + super().__init__() + self.datahub_conn_id = datahub_conn_id + + def make_emitter(self) -> "SynchronizedFileEmitter": + from datahub.emitter.synchronized_file_emitter import SynchronizedFileEmitter + + conn = self.get_connection(self.datahub_conn_id) + + return SynchronizedFileEmitter(filename=conn.host) diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 3367e2ada5bc1..37ec7fbe7e0e3 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -270,6 +270,7 @@ def get_long_description(): # Sink plugins. "datahub-kafka": kafka_common, "datahub-rest": rest_common, + "sync-file-emitter": {"filelock"}, "datahub-lite": { "duckdb", "fastapi", diff --git a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py new file mode 100644 index 0000000000000..70e5d1b364955 --- /dev/null +++ b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py @@ -0,0 +1,45 @@ +import pathlib +from typing import Callable, Union + +import filelock + +from datahub.emitter.generic_emitter import Emitter +from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.ingestion.api.closeable import Closeable +from datahub.ingestion.sink.file import write_metadata_file +from datahub.ingestion.source.file import read_metadata_file +from datahub.metadata.com.linkedin.pegasus2avro.mxe import ( + MetadataChangeEvent, + MetadataChangeProposal, +) + + +class SynchronizedFileEmitter(Closeable, Emitter): + """ + A multiprocessing-safe emitter that writes to a file. + + This emitter is intended for testing purposes only. It is not performant + because it reads and writes the full file on every emit call to ensure + that the file is always valid JSON. + """ + + def __init__(self, filename: str) -> None: + self._filename = pathlib.Path(filename) + self._lock = filelock.FileLock(self._filename.with_suffix(".lock")) + + def emit( + self, + item: Union[ + MetadataChangeEvent, MetadataChangeProposal, MetadataChangeProposalWrapper + ], + callback: Callable[[Exception, str], None] | None = None, + ) -> None: + with self._lock: + if self._filename.exists(): + metadata = list(read_metadata_file(self._filename)) + else: + metadata = [] + + metadata.append(item) + + write_metadata_file(self._filename, metadata) From 7119008bcd1e1e1b01e267de411181ad723a69a4 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 19:46:34 -0700 Subject: [PATCH 17/60] docs tweaks --- docs/lineage/airflow.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index a44d6b86acdfb..3d506ca80aa4e 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -32,7 +32,7 @@ If you're using Airflow older than 2.1, it's possible to use the v1 plugin with The v2 plugin requires Airflow 2.3+ and Python 3.8+. If you don't meet these requirements, use the v1 plugin instead. ```sh -pip install acryl-datahub-airflow-plugin[plugin-v2] +pip install 'acryl-datahub-airflow-plugin[plugin-v2]' ``` ### Configuration @@ -47,8 +47,8 @@ No additional configuration is required to use the plugin. However, there are so ```ini title="airflow.cfg" [datahub] -# Additional config here. -#enabled = True # default +# Optional - additional config here. +enabled = True # default ``` | Name | Default value | Description | @@ -60,13 +60,13 @@ No additional configuration is required to use the plugin. However, there are so | capture_tags_info | true | Extract DAG tags. | | capture_executions | true | Extract task runs and success/failure statuses. This will show up in DataHub "Runs" tab. | | enable_extractors | true | Enable automatic lineage extraction. | +| disable_openlineage_plugin | true | Disable the OpenLineage plugin to avoid duplicative processing. | | log_level | no change | [debug] Set the log level for the plugin. | | debug_emitter | false | [debug] If true, the plugin will log the emitted events. | -| disable_openlineage_plugin | true | Disable the OpenLineage plugin to avoid duplicative processing. | ### Automatic lineage extraction -The v2 plugin uses Airflow's built-in [OpenLineage extractors](https://openlineage.io/docs/integrations/airflow/default-extractors) to extract lineage information. +To automatically extract lineage information, the v2 plugin builds on top of Airflow's built-in [OpenLineage extractors](https://openlineage.io/docs/integrations/airflow/default-extractors). The SQL-related extractors have been updated to use DataHub's SQL parser, which is more robust than the built-in one and uses DataHub's metadata information to generate column-level lineage. We discussed the DataHub SQL parser, including why schema-aware parsing works better and how it performs on benchmarks, during the [June 2023 community town hall](https://youtu.be/1QVcUmRQK5E?si=U27zygR7Gi_KdkzE&t=2309). @@ -79,11 +79,11 @@ The v1 plugin requires Airflow 2.1+ and Python 3.8+. If you're on older versions If you're using Airflow 2.3+, we recommend using the v2 plugin instead. If you need to use the v1 plugin with Airflow 2.3+, you must also set the environment variable `DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN=true`. ```shell -pip install acryl-datahub-airflow-plugin[plugin-v1] +pip install 'acryl-datahub-airflow-plugin[plugin-v1]' # The DataHub rest connection type is included by default. # To use the DataHub Kafka connection type, install the plugin with the kafka extras. -pip install acryl-datahub-airflow-plugin[plugin-v1,datahub-kafka] +pip install 'acryl-datahub-airflow-plugin[plugin-v1,datahub-kafka]' ``` @@ -117,7 +117,7 @@ If your config doesn't align with the default values, you can configure the plug ```ini title="airflow.cfg" [datahub] enabled = true -conn_id = datahub_rest_default +conn_id = datahub_rest_default # or datahub_kafka_default # etc. ``` From 3436095c29d6d127aaf1c872770e20b2ce09574c Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 26 Sep 2023 22:49:13 -0700 Subject: [PATCH 18/60] most of test scaffolding --- .../datahub_airflow_plugin/hooks/datahub.py | 7 +- .../tests/integration/test_plugin.py | 111 ++++++++++++++++-- .../emitter/synchronized_file_emitter.py | 4 + 3 files changed, 111 insertions(+), 11 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index b7f56c18d00d1..6c40ca0f2e4f7 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -220,8 +220,11 @@ def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: or conn.conn_type == DatahubKafkaHook.conn_type.replace("-", "_") ): return DatahubKafkaHook(self.datahub_conn_id) - elif conn.conn_type == SynchronizedFileEmitter.conn_type: - return SynchronizedFileEmitter(self.datahub_conn_id) + elif ( + conn.conn_type == SynchronizedFileHook.conn_type + or conn.conn_type == SynchronizedFileHook.conn_type.replace("-", "_") + ): + return SynchronizedFileHook(self.datahub_conn_id) elif "rest" in self.datahub_conn_id: return DatahubRestHook(self.datahub_conn_id) elif "kafka" in self.datahub_conn_id: diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 3dea9208ce0c6..b76f2403fe4b9 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -1,25 +1,47 @@ import contextlib +import dataclasses +import functools import logging import os import pathlib -import random import signal import subprocess -import time from typing import Iterator import requests import tenacity +from airflow.models.connection import Connection logger = logging.getLogger(__name__) +@dataclasses.dataclass +class AirflowInstance: + airflow_home: pathlib.Path + airflow_port: int + pid: int + env_vars: dict + + username: str + password: str + + @property + def airflow_url(self) -> str: + return f"http://localhost:{self.airflow_port}" + + @functools.cached_property + def session(self) -> requests.Session: + session = requests.Session() + session.auth = (self.username, self.password) + return session + + @tenacity.retry( reraise=True, wait=tenacity.wait_fixed(1), stop=tenacity.stop_after_delay(60), retry=tenacity.retry_if_exception_type( - AssertionError, requests.exceptions.RequestException + (AssertionError, requests.exceptions.RequestException) ), ) def _wait_for_airflow_healthy(airflow_port: int) -> None: @@ -32,22 +54,63 @@ def _wait_for_airflow_healthy(airflow_port: int) -> None: assert airflow_health["scheduler"]["status"] == "healthy" +class NotReadyError(Exception): + pass + + +@tenacity.retry( + reraise=True, + wait=tenacity.wait_fixed(1), + stop=tenacity.stop_after_delay(60), + retry=tenacity.retry_if_exception_type(NotReadyError), +) +def _wait_for_dag_finish(airflow_instance: AirflowInstance, dag_id: str) -> None: + print("Checking if DAG is finished") + res = airflow_instance.session.get( + f"{airflow_instance.airflow_url}/api/v1/dags/{dag_id}/dagRuns", timeout=5 + ) + res.raise_for_status() + + dag_runs = res.json()["dag_runs"] + if not dag_runs: + raise NotReadyError("No DAG runs found") + + dag_run = dag_runs[0] + if dag_run["state"] == "failed": + raise ValueError("DAG failed") + + if dag_run["state"] != "success": + raise NotReadyError(f"DAG has not finished yet: {dag_run['state']}") + + @contextlib.contextmanager -def _run_airflow(tmp_path: pathlib.Path, dags_folder: pathlib.Path) -> Iterator[None]: +def _run_airflow( + tmp_path: pathlib.Path, dags_folder: pathlib.Path +) -> Iterator[AirflowInstance]: airflow_home = tmp_path / "airflow_home" print(f"Using airflow home: {airflow_home}") - airflow_port = random.randint(10000, 12000) + # airflow_port = random.randint(10000, 12000) + airflow_port = 11792 print(f"Using airflow port: {airflow_port}") + datahub_connection_name = "datahub_file_default" + meta_file = tmp_path / "datahub_metadata.json" + environment = { **os.environ, "AIRFLOW_HOME": str(airflow_home), "AIRFLOW__WEBSERVER__WEB_SERVER_PORT": str(airflow_port), "AIRFLOW__CORE__LOAD_EXAMPLES": "False", "AIRFLOW__CORE__DAGS_FOLDER": str(dags_folder), - # TODO change the datahub plugin connection id - # TODO set up a datahub file hook + "AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION": "False", + # Have the datahub plugin write metadata to a file. + "AIRFLOW__DATAHUB__CONN_ID": datahub_connection_name, + f"AIRFLOW_CONN_{datahub_connection_name.upper()}": Connection( + conn_id="datahub_file_default", + conn_type="datahub-file", + host=str(meta_file), + ).get_uri(), } # Start airflow in a background subprocess. @@ -63,7 +126,14 @@ def _run_airflow(tmp_path: pathlib.Path, dags_folder: pathlib.Path) -> Iterator[ airflow_username = "admin" airflow_password = (airflow_home / "standalone_admin_password.txt").read_text() - yield + yield AirflowInstance( + airflow_home=airflow_home, + airflow_port=airflow_port, + pid=airflow_process.pid, + env_vars=environment, + username=airflow_username, + password=airflow_password, + ) finally: # Attempt a graceful shutdown. print("Shutting down airflow...") @@ -87,6 +157,29 @@ def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: dags_folder = pathlib.Path(__file__).parent / "dags" - with _run_airflow(tmp_path, dags_folder=dags_folder): + with _run_airflow(tmp_path, dags_folder=dags_folder) as airflow_instance: + dag_id = "simple_dag" + breakpoint() + + print(f"Running DAG {dag_id}...") + subprocess.check_call( + [ + "airflow", + "dags", + "trigger", + "-r", + "manual_run_test", + dag_id, + ], + env=airflow_instance.env_vars, + ) + + breakpoint() + + print("Waiting for DAG to finish...") + _wait_for_dag_finish(airflow_instance, dag_id) + + breakpoint() + print("done") diff --git a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py index 70e5d1b364955..67469e728f6ef 100644 --- a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py @@ -43,3 +43,7 @@ def emit( metadata.append(item) write_metadata_file(self._filename, metadata) + + def close(self) -> None: + # No-op. + pass From 1b4f4313807547712ca9b9eed176e4e4450eebce Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 12:05:24 -0700 Subject: [PATCH 19/60] stuff --- .../tests/integration/test_plugin.py | 40 +++++++++++++++---- metadata-ingestion/setup.py | 3 +- 2 files changed, 34 insertions(+), 9 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index b76f2403fe4b9..596c4c8e94b62 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -14,6 +14,8 @@ logger = logging.getLogger(__name__) +IS_LOCAL = os.environ.get("CI", "false") == "false" + @dataclasses.dataclass class AirflowInstance: @@ -101,10 +103,14 @@ def _run_airflow( **os.environ, "AIRFLOW_HOME": str(airflow_home), "AIRFLOW__WEBSERVER__WEB_SERVER_PORT": str(airflow_port), + # Point airflow to the DAGs folder. "AIRFLOW__CORE__LOAD_EXAMPLES": "False", "AIRFLOW__CORE__DAGS_FOLDER": str(dags_folder), "AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION": "False", - # Have the datahub plugin write metadata to a file. + # Have the Airflow API use username/password authentication. + "AIRFLOW__API__AUTH_BACKEND": "airflow.api.auth.backend.basic_auth", + # Configure the datahub plugin and have it write the MCPs to a file. + "AIRFLOW__CORE__LAZY_LOAD_PLUGINS": "False", # TODO: only do this for plugin v1 "AIRFLOW__DATAHUB__CONN_ID": datahub_connection_name, f"AIRFLOW_CONN_{datahub_connection_name.upper()}": Connection( conn_id="datahub_file_default", @@ -123,6 +129,30 @@ def _run_airflow( _wait_for_airflow_healthy(airflow_port) print("Airflow is ready!") + # Create an extra "airflow" user for easy testing. + if IS_LOCAL: + print("Creating an extra test user...") + subprocess.check_call( + [ + "airflow", + "users", + "create", + "--username", + "airflow", + "--password", + "airflow", + "--firstname", + "admin", + "--lastname", + "admin", + "--role", + "Admin", + "--email", + "airflow@example.com", + ], + env=environment, + ) + airflow_username = "admin" airflow_password = (airflow_home / "standalone_admin_password.txt").read_text() @@ -153,15 +183,11 @@ def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: # - Waits for the DAG to complete. # - Checks that the metadata was emitted to DataHub. - # TODO cleanup old running airflow instances - dags_folder = pathlib.Path(__file__).parent / "dags" with _run_airflow(tmp_path, dags_folder=dags_folder) as airflow_instance: dag_id = "simple_dag" - breakpoint() - print(f"Running DAG {dag_id}...") subprocess.check_call( [ @@ -175,11 +201,9 @@ def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: env=airflow_instance.env_vars, ) - breakpoint() - print("Waiting for DAG to finish...") _wait_for_dag_finish(airflow_instance, dag_id) breakpoint() - print("done") + print("test finished and all good") diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 37ec7fbe7e0e3..2a5cc0cc9b2a7 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -18,7 +18,8 @@ def get_long_description(): base_requirements = { - "typing_extensions>=3.10.0.2", + # Typing extension should be >=3.10.0.2 ideally but we can't restrict due to a Airflow 2.1 dependency conflict. + "typing_extensions>=3.7.4.3", "mypy_extensions>=0.4.3", # Actual dependencies. "typing-inspect", From 1ffa6235732a677b6e23033757a4e31d8e1d2924 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 14:53:02 -0700 Subject: [PATCH 20/60] get basic plugin v1 test working --- docs/lineage/airflow.md | 10 +- .../airflow-plugin/pyproject.toml | 1 + .../airflow-plugin/setup.py | 48 +- .../datahub_airflow_plugin/_airflow_shims.py | 9 +- .../client/airflow_generator.py | 14 +- .../datahub_airflow_plugin/datahub_plugin.py | 4 +- .../airflow-plugin/tests/conftest.py | 6 + .../tests/integration/dags/simple_dag.py | 33 + .../integration/goldens/v1_simple_dag.json | 726 ++++++++++++++++++ .../tests/integration/test_plugin.py | 68 +- 10 files changed, 875 insertions(+), 44 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/conftest.py create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index 3d506ca80aa4e..e42bdefc49345 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -15,10 +15,10 @@ The DataHub Airflow plugin supports: There's two actively supported implementations of the plugin, with different Airflow version support. -| Approach | Airflow Version | Notes | -| --------- | --------------- | ------------------------------------------- | -| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | -| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction. | +| Approach | Airflow Version | Notes | +| --------- | --------------- | --------------------------------------------------------------------------------------- | +| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | +| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction; will not extract lineage if the task fails. | If you're using Airflow older than 2.1, it's possible to use the v1 plugin with older versions of `acryl-datahub-airflow-plugin`. See the [compatibility section](#compatibility) for more details. @@ -61,7 +61,7 @@ enabled = True # default | capture_executions | true | Extract task runs and success/failure statuses. This will show up in DataHub "Runs" tab. | | enable_extractors | true | Enable automatic lineage extraction. | | disable_openlineage_plugin | true | Disable the OpenLineage plugin to avoid duplicative processing. | -| log_level | no change | [debug] Set the log level for the plugin. | +| log_level | _no change_ | [debug] Set the log level for the plugin. | | debug_emitter | false | [debug] If true, the plugin will log the emitted events. | ### Automatic lineage extraction diff --git a/metadata-ingestion-modules/airflow-plugin/pyproject.toml b/metadata-ingestion-modules/airflow-plugin/pyproject.toml index fba81486b9f67..648040c1951db 100644 --- a/metadata-ingestion-modules/airflow-plugin/pyproject.toml +++ b/metadata-ingestion-modules/airflow-plugin/pyproject.toml @@ -12,6 +12,7 @@ include = '\.pyi?$' [tool.isort] indent = ' ' +known_future_library = ['__future__', 'datahub.utilities._markupsafe_compat', 'datahub_provider._airflow_compat'] profile = 'black' sections = 'FUTURE,STDLIB,THIRDPARTY,FIRSTPARTY,LOCALFOLDER' diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index e56b14ee38127..80f10d2d36691 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -29,12 +29,27 @@ def get_long_description(): *rest_common, } -plugin_v2_requirements = { - # The v2 plugin requires Python 3.8+. - f"acryl-datahub[sql-parser]{_self_pin}", - "openlineage-airflow==1.2.0; python_version >= '3.8'", +plugins = { + "datahub-rest": { + f"acryl-datahub[datahub-rest]{_self_pin}", + }, + "datahub-kafka": { + f"acryl-datahub[datahub-kafka]{_self_pin}", + }, + "datahub-file": { + f"acryl-datahub[sync-file-emitter]{_self_pin}", + }, + "plugin-v1": set(), + "plugin-v2": { + # The v2 plugin requires Python 3.8+. + f"acryl-datahub[sql-parser]{_self_pin}", + "openlineage-airflow==1.2.0; python_version >= '3.8'", + }, } +# Include datahub-rest in the base requirements. +base_requirements |= plugins["datahub-rest"] + mypy_stubs = { "types-dataclasses", @@ -76,7 +91,16 @@ def get_long_description(): "build", "twine", "packaging", - f"acryl-datahub[sync-file-emitter,testing-utils]{_self_pin}", +} + +integration_test_requirements = { + *dev_requirements, + *plugins["datahub-file"], + *plugins["datahub-kafka"], + f"acryl-datahub[testing-utils]{_self_pin}", + # Extra requirements for loading our test dags. + "apache-airflow[snowflake]>=2.0.2", + "virtualenv", # needed by PythonVirtualenvOperator } @@ -132,18 +156,8 @@ def get_long_description(): # Dependencies. install_requires=list(base_requirements), extras_require={ + **{plugin: list(dependencies) for (plugin, dependencies) in plugins.items()}, "dev": list(dev_requirements), - "datahub-kafka": [f"acryl-datahub[datahub-kafka]{_self_pin}"], - "datahub-file": [f"acryl-datahub[sync-file-emitter]{_self_pin}"], - "plugin-v1": [], - "plugin-v2": list(plugin_v2_requirements), - "integration-tests": [ - f"acryl-datahub[datahub-kafka]{_self_pin}", - # Extra requirements for Airflow. - "apache-airflow[snowflake]>=2.0.2", # snowflake is used in example dags - # Because of https://github.com/snowflakedb/snowflake-sqlalchemy/issues/350 we need to restrict SQLAlchemy's max version. - "SQLAlchemy<1.4.42", - "virtualenv", # needed by PythonVirtualenvOperator - ], + "integration-tests": list(integration_test_requirements), }, ) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py index e2e85d6f7eb6e..5e95ba592144b 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py @@ -6,10 +6,6 @@ from datahub_airflow_plugin._airflow_compat import AIRFLOW_PATCHED -AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) - -IS_AIRFLOW_V23_PLUS = AIRFLOW_VERSION >= packaging.version.parse("2.3.0.dev0") - try: from airflow.models.mappedoperator import MappedOperator from airflow.models.operator import Operator @@ -29,6 +25,9 @@ assert AIRFLOW_PATCHED +AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) +HAS_AIRFLOW_LISTENER_API = AIRFLOW_VERSION >= packaging.version.parse("2.3.0.dev0") + def get_task_inlets(operator: "Operator") -> List: # From Airflow 2.4 _inlets is dropped and inlets used consistently. Earlier it was not the case, so we have to stick there to _inlets @@ -51,7 +50,7 @@ def get_task_outlets(operator: "Operator") -> List: __all__ = [ "AIRFLOW_VERSION", - "IS_AIRFLOW_V23_PLUS", + "HAS_AIRFLOW_LISTENER_API", "Operator", "MappedOperator", "EmptyOperator", diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index 6041a8c1c0a9f..6fcdc74d4e776 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -228,9 +228,6 @@ def generate_datajob( job_property_bag: Dict[str, str] = {} allowed_task_keys = [ - "_downstream_task_ids", - "_inlets", - "_outlets", "_task_type", "_task_module", "depends_on_past", @@ -243,15 +240,20 @@ def generate_datajob( "trigger_rule", "wait_for_downstream", # In Airflow 2.3, _downstream_task_ids was renamed to downstream_task_ids - "downstream_task_ids", + ("downstream_task_ids", "_downstream_task_ids"), # In Airflow 2.4, _inlets and _outlets were removed in favor of non-private versions. - "inlets", - "outlets", + ("inlets", "_inlets"), + ("outlets", "_outlets"), ] for key in allowed_task_keys: + old_key = None + if isinstance(key, tuple): + key, old_key = key if hasattr(task, key): job_property_bag[key] = repr(getattr(task, key)) + elif old_key is not None and hasattr(task, old_key): + job_property_bag[key] = repr(getattr(task, old_key)) datajob.properties = job_property_bag base_url = conf.get("webserver", "base_url") diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index a1e3fd3941d41..eca8d9444f7b8 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -6,13 +6,13 @@ from airflow.plugins_manager import AirflowPlugin from datahub_airflow_plugin._airflow_compat import AIRFLOW_PATCHED -from datahub_airflow_plugin._airflow_shims import IS_AIRFLOW_V23_PLUS +from datahub_airflow_plugin._airflow_shims import HAS_AIRFLOW_LISTENER_API assert AIRFLOW_PATCHED logger = logging.getLogger(__name__) -_USE_AIRFLOW_LISTENER_INTERFACE = IS_AIRFLOW_V23_PLUS and not os.getenv( +_USE_AIRFLOW_LISTENER_INTERFACE = HAS_AIRFLOW_LISTENER_API and not os.getenv( "DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN", "false" ).lower() in ("true", "1") diff --git a/metadata-ingestion-modules/airflow-plugin/tests/conftest.py b/metadata-ingestion-modules/airflow-plugin/tests/conftest.py new file mode 100644 index 0000000000000..d2c45e723f1b0 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/conftest.py @@ -0,0 +1,6 @@ +def pytest_addoption(parser): + parser.addoption( + "--update-golden-files", + action="store_true", + default=False, + ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py new file mode 100644 index 0000000000000..1f4042d971423 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py @@ -0,0 +1,33 @@ +from datetime import datetime + +from airflow import DAG +from airflow.operators.bash import BashOperator + +from datahub_airflow_plugin.entities import Dataset, Urn + +with DAG( + "simple_dag", + start_date=datetime(2023, 1, 1), + schedule_interval=None, + catchup=False, +) as dag: + task1 = BashOperator( + task_id="task_1", + dag=dag, + bash_command="echo 'task 1'", + inlets=[ + Dataset(platform="snowflake", name="mydb.schema.tableA"), + Urn( + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ), + ], + outlets=[Dataset("snowflake", "mydb.schema.tableD")], + ) + + task2 = BashOperator( + task_id="run_another_data_task", + dag=dag, + bash_command="echo 'task 2'", + ) + + task1 >> task2 diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json new file mode 100644 index 0000000000000..f079b014cfe1f --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -0,0 +1,726 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "_default_view": "'tree'", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "None", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "0.166158", + "start_date": "2023-09-27 21:47:53.144328+00:00", + "end_date": "2023-09-27 21:47:53.310486+00:00", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "1", + "hostname": "aluminum-b.local", + "max_tries": "0", + "external_executor_id": "None", + "pid": "55422", + "state": "success", + "operator": "BashOperator", + "priority_weight": "2", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag", + "name": "simple_dag_task_1_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695851273144, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695851273144, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 2 + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695851273310, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "_default_view": "'tree'", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "None", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "0.125855", + "start_date": "2023-09-27 21:47:56.482097+00:00", + "end_date": "2023-09-27 21:47:56.607952+00:00", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "1", + "hostname": "aluminum-b.local", + "max_tries": "0", + "external_executor_id": "None", + "pid": "55429", + "state": "success", + "operator": "BashOperator", + "priority_weight": "1", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag", + "name": "simple_dag_run_another_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695851276482, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695851276482, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 2 + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695851276607, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 596c4c8e94b62..f85056183685c 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -6,11 +6,15 @@ import pathlib import signal import subprocess -from typing import Iterator +from typing import Iterator, Sequence +import pytest import requests import tenacity from airflow.models.connection import Connection +from datahub.testing.compare_metadata_json import assert_metadata_files_equal + +from datahub_airflow_plugin._airflow_shims import HAS_AIRFLOW_LISTENER_API logger = logging.getLogger(__name__) @@ -27,6 +31,8 @@ class AirflowInstance: username: str password: str + metadata_file: pathlib.Path + @property def airflow_url(self) -> str: return f"http://localhost:{self.airflow_port}" @@ -87,7 +93,7 @@ def _wait_for_dag_finish(airflow_instance: AirflowInstance, dag_id: str) -> None @contextlib.contextmanager def _run_airflow( - tmp_path: pathlib.Path, dags_folder: pathlib.Path + tmp_path: pathlib.Path, dags_folder: pathlib.Path, is_v1: bool ) -> Iterator[AirflowInstance]: airflow_home = tmp_path / "airflow_home" print(f"Using airflow home: {airflow_home}") @@ -103,6 +109,7 @@ def _run_airflow( **os.environ, "AIRFLOW_HOME": str(airflow_home), "AIRFLOW__WEBSERVER__WEB_SERVER_PORT": str(airflow_port), + "AIRFLOW__WEBSERVER__BASE_URL": "http://airflow.example.com", # Point airflow to the DAGs folder. "AIRFLOW__CORE__LOAD_EXAMPLES": "False", "AIRFLOW__CORE__DAGS_FOLDER": str(dags_folder), @@ -110,7 +117,7 @@ def _run_airflow( # Have the Airflow API use username/password authentication. "AIRFLOW__API__AUTH_BACKEND": "airflow.api.auth.backend.basic_auth", # Configure the datahub plugin and have it write the MCPs to a file. - "AIRFLOW__CORE__LAZY_LOAD_PLUGINS": "False", # TODO: only do this for plugin v1 + "AIRFLOW__CORE__LAZY_LOAD_PLUGINS": "False" if is_v1 else "True", "AIRFLOW__DATAHUB__CONN_ID": datahub_connection_name, f"AIRFLOW_CONN_{datahub_connection_name.upper()}": Connection( conn_id="datahub_file_default", @@ -163,6 +170,7 @@ def _run_airflow( env_vars=environment, username=airflow_username, password=airflow_password, + metadata_file=meta_file, ) finally: # Attempt a graceful shutdown. @@ -175,7 +183,24 @@ def _run_airflow( airflow_process.wait(timeout=3) -def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: +def check_golden_file( + pytestconfig: pytest.Config, + output_path: pathlib.Path, + golden_path: pathlib.Path, + ignore_paths: Sequence[str] = (), +) -> None: + update_golden = pytestconfig.getoption("--update-golden-files") + + assert_metadata_files_equal( + output_path=output_path, + golden_path=golden_path, + update_golden=update_golden, + copy_output=False, + ignore_paths=ignore_paths, + ) + + +def test_airflow_plugin(pytestconfig: pytest.Config, tmp_path: pathlib.Path) -> None: # This test: # - Configures the plugin. # - Starts a local airflow instance in a subprocess. @@ -184,16 +209,22 @@ def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: # - Checks that the metadata was emitted to DataHub. dags_folder = pathlib.Path(__file__).parent / "dags" + goldens_folder = pathlib.Path(__file__).parent / "goldens" - with _run_airflow(tmp_path, dags_folder=dags_folder) as airflow_instance: - dag_id = "simple_dag" + dag_id = "simple_dag" + is_v1 = not HAS_AIRFLOW_LISTENER_API + with _run_airflow( + tmp_path, dags_folder=dags_folder, is_v1=is_v1 + ) as airflow_instance: print(f"Running DAG {dag_id}...") subprocess.check_call( [ "airflow", "dags", "trigger", + "--exec-date", + "2023-09-27T21:34:38+00:00", "-r", "manual_run_test", dag_id, @@ -204,6 +235,25 @@ def test_airflow_plugin_v2(tmp_path: pathlib.Path) -> None: print("Waiting for DAG to finish...") _wait_for_dag_finish(airflow_instance, dag_id) - breakpoint() - - print("test finished and all good") + if is_v1: + golden_path = goldens_folder / f"v1_{dag_id}.json" + else: + golden_path = goldens_folder / f"v2_{dag_id}.json" + + check_golden_file( + pytestconfig=pytestconfig, + output_path=airflow_instance.metadata_file, + golden_path=golden_path, + ignore_paths=[ + # Timing-related items. + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['start_date'\]", + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['end_date'\]", + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['duration'\]", + # Host-specific items. + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['pid'\]", + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['hostname'\]", + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['unixname'\]", + # TODO: If we switched to Git urls, maybe we could get this to work consistently. + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['fileloc'\]", + ], + ) From 73e3acc47bf16676fde051e310ba10b0f24164e1 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 15:11:04 -0700 Subject: [PATCH 21/60] add another simple test --- .../tests/integration/dags/basic_iolets.py | 40 ++ .../integration/goldens/v1_basic_iolets.json | 538 ++++++++++++++++++ .../tests/integration/test_plugin.py | 43 +- 3 files changed, 602 insertions(+), 19 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py new file mode 100644 index 0000000000000..cd7e0cdcb3219 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py @@ -0,0 +1,40 @@ +"""Lineage Backend + +An example DAG demonstrating the usage of DataHub's Airflow lineage backend. +""" + +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.operators.bash import BashOperator +from airflow.utils.dates import days_ago + +from datahub_airflow_plugin.entities import Dataset, Urn + +with DAG( + "basic_iolets", + start_date=datetime(2023, 1, 1), + schedule_interval=None, + catchup=False, +) as dag: + task = BashOperator( + task_id="run_data_task", + dag=dag, + bash_command="echo 'This is where you might run your data tooling.'", + inlets=[ + Dataset(platform="snowflake", name="mydb.schema.tableA"), + Dataset(platform="snowflake", name="mydb.schema.tableB", env="DEV"), + Dataset( + platform="snowflake", + name="mydb.schema.tableC", + platform_instance="cloud", + ), + Urn( + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ), + ], + outlets=[ + Dataset("snowflake", "mydb.schema.tableD"), + Dataset("snowflake", "mydb.schema.tableE"), + ], + ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json new file mode 100644 index 0000000000000..c594c28a11d28 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json @@ -0,0 +1,538 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "_default_view": "'tree'", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "None", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", + "name": "basic_iolets", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "'all_success'", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "0.16349", + "start_date": "2023-09-27 22:07:50.874653+00:00", + "end_date": "2023-09-27 22:07:51.038143+00:00", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "1", + "hostname": "1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa", + "max_tries": "0", + "external_executor_id": "None", + "pid": "60941", + "state": "success", + "operator": "BashOperator", + "priority_weight": "1", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets", + "name": "basic_iolets_run_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695852470874, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695852470874, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 2 + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695852471038, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index f85056183685c..a50c351c81399 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -141,21 +141,15 @@ def _run_airflow( print("Creating an extra test user...") subprocess.check_call( [ - "airflow", - "users", - "create", - "--username", - "airflow", - "--password", - "airflow", - "--firstname", - "admin", - "--lastname", - "admin", - "--role", - "Admin", - "--email", - "airflow@example.com", + # fmt: off + "airflow", "users", "create", + "--username", "airflow", + "--password", "airflow", + "--firstname", "admin", + "--lastname", "admin", + "--role", "Admin", + "--email", "airflow@example.com", + # fmt: on ], env=environment, ) @@ -200,7 +194,16 @@ def check_golden_file( ) -def test_airflow_plugin(pytestconfig: pytest.Config, tmp_path: pathlib.Path) -> None: +@pytest.mark.parametrize( + ["dag_id", "is_v1"], + [ + pytest.param("simple_dag", True), + pytest.param("basic_iolets", True), + ], +) +def test_airflow_plugin( + pytestconfig: pytest.Config, tmp_path: pathlib.Path, dag_id: str, is_v1: bool +) -> None: # This test: # - Configures the plugin. # - Starts a local airflow instance in a subprocess. @@ -208,12 +211,14 @@ def test_airflow_plugin(pytestconfig: pytest.Config, tmp_path: pathlib.Path) -> # - Waits for the DAG to complete. # - Checks that the metadata was emitted to DataHub. + if not HAS_AIRFLOW_LISTENER_API and not is_v1: + pytest.skip("Cannot test plugin v2 without the Airflow plugin listener API") + if HAS_AIRFLOW_LISTENER_API and is_v1: + pytest.skip("Not testing plugin v1 on older Airflow versions") + dags_folder = pathlib.Path(__file__).parent / "dags" goldens_folder = pathlib.Path(__file__).parent / "goldens" - dag_id = "simple_dag" - is_v1 = not HAS_AIRFLOW_LISTENER_API - with _run_airflow( tmp_path, dags_folder=dags_folder, is_v1=is_v1 ) as airflow_instance: From 5c2ad609ac197fedaf371365421041888fd28909 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 15:15:39 -0700 Subject: [PATCH 22/60] set test ids --- .../tests/integration/test_plugin.py | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index a50c351c81399..b724db5d033d2 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -195,14 +195,20 @@ def check_golden_file( @pytest.mark.parametrize( - ["dag_id", "is_v1"], + ["golden_filename", "dag_id", "is_v1"], [ - pytest.param("simple_dag", True), - pytest.param("basic_iolets", True), + pytest.param("v1_simple_dag.json", "simple_dag", True, id="v1_simple_dag"), + pytest.param( + "v1_basic_iolets.json", "basic_iolets", True, id="v1_basic_iolets" + ), ], ) def test_airflow_plugin( - pytestconfig: pytest.Config, tmp_path: pathlib.Path, dag_id: str, is_v1: bool + pytestconfig: pytest.Config, + tmp_path: pathlib.Path, + golden_filename: str, + dag_id: str, + is_v1: bool, ) -> None: # This test: # - Configures the plugin. @@ -219,6 +225,8 @@ def test_airflow_plugin( dags_folder = pathlib.Path(__file__).parent / "dags" goldens_folder = pathlib.Path(__file__).parent / "goldens" + golden_path = goldens_folder / f"{golden_filename}.json" + with _run_airflow( tmp_path, dags_folder=dags_folder, is_v1=is_v1 ) as airflow_instance: @@ -240,11 +248,6 @@ def test_airflow_plugin( print("Waiting for DAG to finish...") _wait_for_dag_finish(airflow_instance, dag_id) - if is_v1: - golden_path = goldens_folder / f"v1_{dag_id}.json" - else: - golden_path = goldens_folder / f"v2_{dag_id}.json" - check_golden_file( pytestconfig=pytestconfig, output_path=airflow_instance.metadata_file, From 9fddd68bac88351948e87982c10b10b9ee3240bd Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 16:02:21 -0700 Subject: [PATCH 23/60] update tests more --- .../airflow-plugin/setup.py | 2 + .../client/airflow_generator.py | 5 +- .../tests/integration/dags/basic_iolets.py | 8 +-- .../integration/dags/snowflake_operator.py | 32 ++++++++++ .../tests/integration/test_plugin.py | 62 ++++++++++++++++--- 5 files changed, 91 insertions(+), 18 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 80f10d2d36691..221b956ac6be3 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -100,6 +100,8 @@ def get_long_description(): f"acryl-datahub[testing-utils]{_self_pin}", # Extra requirements for loading our test dags. "apache-airflow[snowflake]>=2.0.2", + # https://github.com/snowflakedb/snowflake-sqlalchemy/issues/350 + "snowflake-sqlalchemy>=1.4.3", "virtualenv", # needed by PythonVirtualenvOperator } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index 6fcdc74d4e776..0e12a980d1d73 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -153,7 +153,7 @@ def generate_dataflow( allowed_flow_keys = [ "_access_control", "_concurrency", - "_default_view", + # "_default_view", "catchup", "fileloc", "is_paused_upon_creation", @@ -415,16 +415,13 @@ def run_datajob( job_property_bag["end_date"] = str(ti.end_date) job_property_bag["execution_date"] = str(ti.execution_date) job_property_bag["try_number"] = str(ti.try_number - 1) - job_property_bag["hostname"] = str(ti.hostname) job_property_bag["max_tries"] = str(ti.max_tries) # Not compatible with Airflow 1 if hasattr(ti, "external_executor_id"): job_property_bag["external_executor_id"] = str(ti.external_executor_id) - job_property_bag["pid"] = str(ti.pid) job_property_bag["state"] = str(ti.state) job_property_bag["operator"] = str(ti.operator) job_property_bag["priority_weight"] = str(ti.priority_weight) - job_property_bag["unixname"] = str(ti.unixname) job_property_bag["log_url"] = ti.log_url dpi.properties.update(job_property_bag) dpi.url = ti.log_url diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py index cd7e0cdcb3219..8b0803ab98422 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py @@ -1,13 +1,7 @@ -"""Lineage Backend - -An example DAG demonstrating the usage of DataHub's Airflow lineage backend. -""" - -from datetime import datetime, timedelta +from datetime import datetime from airflow import DAG from airflow.operators.bash import BashOperator -from airflow.utils.dates import days_ago from datahub_airflow_plugin.entities import Dataset, Urn diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py new file mode 100644 index 0000000000000..347d0f88b0cd0 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py @@ -0,0 +1,32 @@ +from datetime import datetime + +from airflow import DAG +from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator + +SNOWFLAKE_COST_TABLE = "costs" +SNOWFLAKE_PROCESSED_TABLE = "processed_costs" + +with DAG( + "snowflake_operator", + start_date=datetime(2023, 1, 1), + schedule_interval=None, + catchup=False, +) as dag: + transform_cost_table = SnowflakeOperator( + snowflake_conn_id="my_snowflake", + task_id="transform_cost_table", + sql=""" + CREATE OR REPLACE TABLE {{ params.out_table_name }} AS + SELECT + id, + month, + total_cost, + area, + total_cost / area as cost_per_area + FROM {{ params.in_table_name }} + """, + params={ + "in_table_name": SNOWFLAKE_COST_TABLE, + "out_table_name": SNOWFLAKE_PROCESSED_TABLE, + }, + ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index b724db5d033d2..6bb3474ff62bd 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -124,6 +124,23 @@ def _run_airflow( conn_type="datahub-file", host=str(meta_file), ).get_uri(), + # Configure fake credentials for the Snowflake connection. + "AIRFLOW_CONN_MY_SNOWFLAKE": Connection( + conn_id="my_snowflake", + conn_type="snowflake", + login="fake_username", + password="fake_password", + schema="DATAHUB_TEST_SCHEMA", + extra={ + "account": "fake_account", + "database": "DATAHUB_TEST_DATABASE", + "warehouse": "fake_warehouse", + "role": "fake_role", + "insecure_mode": "true", + }, + ).get_uri(), + # Convenience settings. + "SQLALCHEMY_SILENCE_UBER_WARNING": "1", } # Start airflow in a background subprocess. @@ -194,20 +211,50 @@ def check_golden_file( ) +@dataclasses.dataclass +class DagTestCase: + dag_id: str + success: bool = False + + v2_only: bool = False + + +test_cases = [ + DagTestCase("simple_dag"), + DagTestCase("basic_iolets"), + DagTestCase("snowflake_operator", success=False, v2_only=True), +] + + @pytest.mark.parametrize( - ["golden_filename", "dag_id", "is_v1"], + ["golden_filename", "test_case", "is_v1"], [ - pytest.param("v1_simple_dag.json", "simple_dag", True, id="v1_simple_dag"), - pytest.param( - "v1_basic_iolets.json", "basic_iolets", True, id="v1_basic_iolets" - ), + *[ + pytest.param( + f"v1_{test_case.dag_id}", + test_case, + True, + id=f"v1_{test_case.dag_id}", + ) + for test_case in test_cases + if not test_case.v2_only + ], + *[ + pytest.param( + f"v2_{test_case.dag_id}", + test_case, + False, + id=f"v2_{test_case.dag_id}", + ) + for test_case in test_cases + ], ], ) def test_airflow_plugin( pytestconfig: pytest.Config, tmp_path: pathlib.Path, golden_filename: str, - dag_id: str, + test_case: DagTestCase, is_v1: bool, ) -> None: # This test: @@ -215,7 +262,7 @@ def test_airflow_plugin( # - Starts a local airflow instance in a subprocess. # - Runs a DAG that uses an operator supported by the extractor. # - Waits for the DAG to complete. - # - Checks that the metadata was emitted to DataHub. + # - Validates the metadata generated against a golden file. if not HAS_AIRFLOW_LISTENER_API and not is_v1: pytest.skip("Cannot test plugin v2 without the Airflow plugin listener API") @@ -226,6 +273,7 @@ def test_airflow_plugin( goldens_folder = pathlib.Path(__file__).parent / "goldens" golden_path = goldens_folder / f"{golden_filename}.json" + dag_id = test_case.dag_id with _run_airflow( tmp_path, dags_folder=dags_folder, is_v1=is_v1 From 09a0d17f08c5c696c25b2f9c581d02eb690c1e5e Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 16:18:44 -0700 Subject: [PATCH 24/60] snowflake test --- .../datahub_airflow_plugin/datahub_plugin.py | 19 + .../integration/goldens/v2_basic_iolets.json | 612 +++++++++++++++ .../integration/goldens/v2_simple_dag.json | 709 ++++++++++++++++++ .../goldens/v2_snowflake_operator.json | 508 +++++++++++++ .../tests/integration/test_plugin.py | 14 +- 5 files changed, 1858 insertions(+), 4 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index eca8d9444f7b8..1404aede19bcb 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -1,3 +1,4 @@ +import contextlib import logging import os from types import ModuleType @@ -16,6 +17,24 @@ "DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN", "false" ).lower() in ("true", "1") +with contextlib.suppress(Exception): + if not os.getenv("DATAHUB_AIRFLOW_PLUGIN_SKIP_FORK_PATCH", "false").lower() in ( + "true", + "1", + ): + # From https://github.com/apache/airflow/discussions/24463#discussioncomment-4404542 + # I'm not exactly sure why this fixes it, but I suspect it's that this + # forces the proxy settings to get cached before the fork happens. + # + # For more details, see https://github.com/python/cpython/issues/58037 + # and https://wefearchange.org/2018/11/forkmacos.rst.html + # and https://bugs.python.org/issue30385#msg293958 + # An alternative fix is to set NO_PROXY='*' + + from _scproxy import _get_proxy_settings + + _get_proxy_settings() + class DatahubPlugin(AirflowPlugin): name = "datahub_plugin" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json new file mode 100644 index 0000000000000..27067772cba41 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -0,0 +1,612 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "_default_view": "'grid'", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", + "name": "basic_iolets", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-27 22:41:25.543449+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "hostname": "aluminum-b.local", + "max_tries": "0", + "external_executor_id": "None", + "pid": "74129", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1", + "name": "basic_iolets_run_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695854485543, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854485543, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854486150, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json new file mode 100644 index 0000000000000..1d5181a6b74e6 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -0,0 +1,709 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "_default_view": "'grid'", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-27 22:40:45.392053+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "hostname": "aluminum-b.local", + "max_tries": "0", + "external_executor_id": "None", + "pid": "73891", + "state": "running", + "operator": "BashOperator", + "priority_weight": "2", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_task_1_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695854445392, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854445392, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854445796, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-27 22:40:50.033987+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "hostname": "aluminum-b.local", + "max_tries": "0", + "external_executor_id": "None", + "pid": "73894", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "unixname": "hsheth", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_run_another_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695854450033, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854450033, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695854450466, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json new file mode 100644 index 0000000000000..d3196e615656f --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json @@ -0,0 +1,508 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=snowflake_operator", + "name": "snowflake_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-27 23:13:23.272262+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SnowflakeOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1", + "name": "snowflake_operator_transform_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695856403272, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695856403272, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695856404172, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "FAILURE", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 6bb3474ff62bd..d7f4a3c19d954 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -72,7 +72,9 @@ class NotReadyError(Exception): stop=tenacity.stop_after_delay(60), retry=tenacity.retry_if_exception_type(NotReadyError), ) -def _wait_for_dag_finish(airflow_instance: AirflowInstance, dag_id: str) -> None: +def _wait_for_dag_finish( + airflow_instance: AirflowInstance, dag_id: str, require_success: bool +) -> None: print("Checking if DAG is finished") res = airflow_instance.session.get( f"{airflow_instance.airflow_url}/api/v1/dags/{dag_id}/dagRuns", timeout=5 @@ -85,9 +87,11 @@ def _wait_for_dag_finish(airflow_instance: AirflowInstance, dag_id: str) -> None dag_run = dag_runs[0] if dag_run["state"] == "failed": - raise ValueError("DAG failed") + if require_success: + raise ValueError("DAG failed") + # else - success is not required, so we're done. - if dag_run["state"] != "success": + elif dag_run["state"] != "success": raise NotReadyError(f"DAG has not finished yet: {dag_run['state']}") @@ -294,7 +298,9 @@ def test_airflow_plugin( ) print("Waiting for DAG to finish...") - _wait_for_dag_finish(airflow_instance, dag_id) + _wait_for_dag_finish( + airflow_instance, dag_id, require_success=test_case.success + ) check_golden_file( pytestconfig=pytestconfig, From ed55d67333431c860b7e8e73be840fe55d4fca25 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 16:21:34 -0700 Subject: [PATCH 25/60] update v2 goldens --- .../tests/integration/goldens/v2_basic_iolets.json | 1 - .../airflow-plugin/tests/integration/goldens/v2_simple_dag.json | 1 - 2 files changed, 2 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json index 27067772cba41..9d671f2d7908c 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -8,7 +8,6 @@ "json": { "customProperties": { "_access_control": "None", - "_default_view": "'grid'", "catchup": "False", "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py'", "is_paused_upon_creation": "None", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index 1d5181a6b74e6..26ad51a251b13 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -8,7 +8,6 @@ "json": { "customProperties": { "_access_control": "None", - "_default_view": "'grid'", "catchup": "False", "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", "is_paused_upon_creation": "None", From 64635003132bec18955df68e3b29e92bbd8b3df7 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 16:57:33 -0700 Subject: [PATCH 26/60] fix mypy issues --- .../airflow-plugin/setup.py | 7 +- .../client/airflow_generator.py | 20 +- .../datahub_airflow_plugin/hooks/datahub.py | 52 ++- .../integration/goldens/v2_basic_iolets.json | 2 +- .../integration/goldens/v2_simple_dag.json | 387 ------------------ .../api/entities/corpgroup/corpgroup.py | 33 +- .../datahub/api/entities/corpuser/corpuser.py | 9 +- .../dataprocess/dataprocess_instance.py | 21 +- .../api/entities/dataproduct/dataproduct.py | 10 +- .../emitter/synchronized_file_emitter.py | 8 +- .../src/datahub/ingestion/graph/client.py | 4 +- 11 files changed, 80 insertions(+), 473 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 221b956ac6be3..4db70f1efb6f8 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -1,5 +1,6 @@ import os import pathlib +from typing import Dict, Set import setuptools @@ -29,7 +30,7 @@ def get_long_description(): *rest_common, } -plugins = { +plugins: Dict[str, Set[str]] = { "datahub-rest": { f"acryl-datahub[datahub-rest]{_self_pin}", }, @@ -48,7 +49,7 @@ def get_long_description(): } # Include datahub-rest in the base requirements. -base_requirements |= plugins["datahub-rest"] +base_requirements.update(plugins["datahub-rest"]) mypy_stubs = { @@ -158,7 +159,7 @@ def get_long_description(): # Dependencies. install_requires=list(base_requirements), extras_require={ - **{plugin: list(dependencies) for (plugin, dependencies) in plugins.items()}, + **{plugin: list(dependencies) for plugin, dependencies in plugins.items()}, "dev": list(dev_requirements), "integration-tests": list(integration_test_requirements), }, diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index 0e12a980d1d73..c32ed4a8dc3f5 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -1,5 +1,5 @@ from datetime import datetime -from typing import TYPE_CHECKING, Dict, List, Optional, Set, cast +from typing import TYPE_CHECKING, Dict, List, Optional, Set, Tuple, Union, cast from airflow.configuration import conf from datahub.api.entities.datajob import DataFlow, DataJob @@ -227,7 +227,7 @@ def generate_datajob( job_property_bag: Dict[str, str] = {} - allowed_task_keys = [ + allowed_task_keys: List[Union[str, Tuple[str, ...]]] = [ "_task_type", "_task_module", "depends_on_past", @@ -247,13 +247,17 @@ def generate_datajob( ] for key in allowed_task_keys: - old_key = None if isinstance(key, tuple): - key, old_key = key - if hasattr(task, key): - job_property_bag[key] = repr(getattr(task, key)) - elif old_key is not None and hasattr(task, old_key): - job_property_bag[key] = repr(getattr(task, old_key)) + out_key: str = key[0] + try_keys = key + else: + out_key = key + try_keys = (key,) + + for k in try_keys: + if hasattr(task, k): + job_property_bag[out_key] = repr(getattr(task, k)) + break datajob.properties = job_property_bag base_url = conf.get("webserver", "base_url") diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py index 6c40ca0f2e4f7..9604931795ccb 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/hooks/datahub.py @@ -192,6 +192,39 @@ def callback(exc, msg): emit_mcps = emit +class SynchronizedFileHook(BaseHook): + conn_type = "datahub-file" + + def __init__(self, datahub_conn_id: str) -> None: + super().__init__() + self.datahub_conn_id = datahub_conn_id + + def make_emitter(self) -> "SynchronizedFileEmitter": + from datahub.emitter.synchronized_file_emitter import SynchronizedFileEmitter + + conn = self.get_connection(self.datahub_conn_id) + filename = conn.host + if not filename: + raise AirflowException("filename parameter is required") + + return SynchronizedFileEmitter(filename=filename) + + def emit( + self, + items: Sequence[ + Union[ + MetadataChangeEvent, + MetadataChangeProposal, + MetadataChangeProposalWrapper, + ] + ], + ) -> None: + emitter = self.make_emitter() + + for item in items: + emitter.emit(item) + + class DatahubGenericHook(BaseHook): """ Emits Metadata Change Events using either the DatahubRestHook or the @@ -205,7 +238,9 @@ def __init__(self, datahub_conn_id: str) -> None: super().__init__() self.datahub_conn_id = datahub_conn_id - def get_underlying_hook(self) -> Union[DatahubRestHook, DatahubKafkaHook]: + def get_underlying_hook( + self, + ) -> Union[DatahubRestHook, DatahubKafkaHook, SynchronizedFileHook]: conn = self.get_connection(self.datahub_conn_id) # We need to figure out the underlying hook type. First check the @@ -251,18 +286,3 @@ def emit( # Retained for backwards compatibility. emit_mces = emit - - -class SynchronizedFileHook(BaseHook): - conn_type = "datahub-file" - - def __init__(self, datahub_conn_id: str) -> None: - super().__init__() - self.datahub_conn_id = datahub_conn_id - - def make_emitter(self) -> "SynchronizedFileEmitter": - from datahub.emitter.synchronized_file_emitter import SynchronizedFileEmitter - - conn = self.get_connection(self.datahub_conn_id) - - return SynchronizedFileEmitter(filename=conn.host) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json index 9d671f2d7908c..90155969c9241 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -74,7 +74,7 @@ "downstream_task_ids": "set()", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", - "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" }, "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", "name": "run_data_task", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index 26ad51a251b13..e3b49688ac033 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -55,54 +55,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'task_1'", - "execution_timeout": "None", - "sla": "None", - "task_id": "'task_1'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", - "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", - "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", - "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", - "name": "task_1", - "type": { - "string": "COMMAND" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" - ], - "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" - ], - "inputDatajobs": [], - "fineGrainedLineages": [] - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", @@ -136,163 +88,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", - "changeType": "UPSERT", - "aspectName": "ownership", - "aspect": { - "json": { - "owners": [ - { - "owner": "urn:li:corpuser:airflow", - "type": "DEVELOPER", - "source": { - "type": "SERVICE" - } - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:airflow" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", - "changeType": "UPSERT", - "aspectName": "globalTags", - "aspect": { - "json": { - "tags": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceProperties", - "aspect": { - "json": { - "customProperties": { - "run_id": "manual_run_test", - "duration": "None", - "start_date": "2023-09-27 22:40:45.392053+00:00", - "end_date": "None", - "execution_date": "2023-09-27 21:34:38+00:00", - "try_number": "0", - "hostname": "aluminum-b.local", - "max_tries": "0", - "external_executor_id": "None", - "pid": "73891", - "state": "running", - "operator": "BashOperator", - "priority_weight": "2", - "unixname": "hsheth", - "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1" - }, - "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1", - "name": "simple_dag_task_1_manual_run_test", - "type": "BATCH_AD_HOC", - "created": { - "time": 1695854445392, - "actor": "urn:li:corpuser:datahub" - } - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRelationships", - "aspect": { - "json": { - "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", - "upstreamInstances": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", - "aspect": { - "json": { - "inputs": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" - ] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceOutput", - "aspect": { - "json": { - "outputs": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" - ] - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695854445392, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", @@ -444,26 +239,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695854445796, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" - } - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", @@ -542,167 +317,5 @@ "tags": [] } } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceProperties", - "aspect": { - "json": { - "customProperties": { - "run_id": "manual_run_test", - "duration": "None", - "start_date": "2023-09-27 22:40:50.033987+00:00", - "end_date": "None", - "execution_date": "2023-09-27 21:34:38+00:00", - "try_number": "0", - "hostname": "aluminum-b.local", - "max_tries": "0", - "external_executor_id": "None", - "pid": "73894", - "state": "running", - "operator": "BashOperator", - "priority_weight": "1", - "unixname": "hsheth", - "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" - }, - "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", - "name": "simple_dag_run_another_data_task_manual_run_test", - "type": "BATCH_AD_HOC", - "created": { - "time": 1695854450033, - "actor": "urn:li:corpuser:datahub" - } - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRelationships", - "aspect": { - "json": { - "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", - "upstreamInstances": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695854450033, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'run_another_data_task'", - "execution_timeout": "None", - "sla": "None", - "task_id": "'run_another_data_task'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "set()", - "inlets": "[]", - "outlets": "[]", - "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", - "name": "run_another_data_task", - "type": { - "string": "COMMAND" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [], - "outputDatasets": [], - "inputDatajobs": [ - "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" - ], - "fineGrainedLineages": [] - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", - "changeType": "UPSERT", - "aspectName": "ownership", - "aspect": { - "json": { - "owners": [ - { - "owner": "urn:li:corpuser:airflow", - "type": "DEVELOPER", - "source": { - "type": "SERVICE" - } - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:airflow" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", - "changeType": "UPSERT", - "aspectName": "globalTags", - "aspect": { - "json": { - "tags": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695854450466, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" - } - } - } } ] \ No newline at end of file diff --git a/metadata-ingestion/src/datahub/api/entities/corpgroup/corpgroup.py b/metadata-ingestion/src/datahub/api/entities/corpgroup/corpgroup.py index 796786beba21b..a898e35bb810e 100644 --- a/metadata-ingestion/src/datahub/api/entities/corpgroup/corpgroup.py +++ b/metadata-ingestion/src/datahub/api/entities/corpgroup/corpgroup.py @@ -2,7 +2,7 @@ import logging from dataclasses import dataclass -from typing import TYPE_CHECKING, Callable, Iterable, List, Optional, Union +from typing import Callable, Iterable, List, Optional, Union import pydantic from pydantic import BaseModel @@ -11,9 +11,10 @@ from datahub.api.entities.corpuser.corpuser import CorpUser, CorpUserGenerationConfig from datahub.configuration.common import ConfigurationError from datahub.configuration.validate_field_rename import pydantic_renamed_field +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.rest_emitter import DatahubRestEmitter -from datahub.ingestion.graph.client import DatahubClientConfig, DataHubGraph +from datahub.ingestion.graph.client import DataHubGraph from datahub.metadata.schema_classes import ( CorpGroupEditableInfoClass, CorpGroupInfoClass, @@ -25,9 +26,6 @@ _Aspect, ) -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - logger = logging.getLogger(__name__) @@ -194,30 +192,9 @@ def generate_mcp( entityUrn=urn, aspect=StatusClass(removed=False) ) - @staticmethod - def _datahub_graph_from_datahub_rest_emitter( - rest_emitter: DatahubRestEmitter, - ) -> DataHubGraph: - """ - Create a datahub graph instance from a REST Emitter. - A stop-gap implementation which is expected to be removed after PATCH support is implemented - for membership updates for users <-> groups - """ - graph = DataHubGraph( - config=DatahubClientConfig( - server=rest_emitter._gms_server, - token=rest_emitter._token, - timeout_sec=rest_emitter._connect_timeout_sec, - retry_status_codes=rest_emitter._retry_status_codes, - extra_headers=rest_emitter._session.headers, - disable_ssl_verification=rest_emitter._session.verify is False, - ) - ) - return graph - def emit( self, - emitter: Union[DatahubRestEmitter, "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ @@ -235,7 +212,7 @@ def emit( # who are passing in a DataHubRestEmitter today # we won't need this in the future once PATCH support is implemented as all emitters # will work - datahub_graph = self._datahub_graph_from_datahub_rest_emitter(emitter) + datahub_graph = emitter.to_graph() for mcp in self.generate_mcp( generation_config=CorpGroupGenerationConfig( override_editable=self.overrideEditable, datahub_graph=datahub_graph diff --git a/metadata-ingestion/src/datahub/api/entities/corpuser/corpuser.py b/metadata-ingestion/src/datahub/api/entities/corpuser/corpuser.py index c67eb02a870a5..9fe1ebedafca7 100644 --- a/metadata-ingestion/src/datahub/api/entities/corpuser/corpuser.py +++ b/metadata-ingestion/src/datahub/api/entities/corpuser/corpuser.py @@ -1,14 +1,14 @@ from __future__ import annotations from dataclasses import dataclass -from typing import TYPE_CHECKING, Callable, Iterable, List, Optional, Union +from typing import Callable, Iterable, List, Optional import pydantic import datahub.emitter.mce_builder as builder from datahub.configuration.common import ConfigModel +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper -from datahub.emitter.rest_emitter import DatahubRestEmitter from datahub.metadata.schema_classes import ( CorpUserEditableInfoClass, CorpUserInfoClass, @@ -16,9 +16,6 @@ StatusClass, ) -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - @dataclass class CorpUserGenerationConfig: @@ -144,7 +141,7 @@ def generate_mcp( def emit( self, - emitter: Union[DatahubRestEmitter, "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ diff --git a/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py b/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py index 9ec389c3a0989..cf6080c7072e6 100644 --- a/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py +++ b/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py @@ -1,9 +1,10 @@ import time from dataclasses import dataclass, field from enum import Enum -from typing import TYPE_CHECKING, Callable, Dict, Iterable, List, Optional, Union, cast +from typing import Callable, Dict, Iterable, List, Optional, Union, cast from datahub.api.entities.datajob import DataFlow, DataJob +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.mcp_builder import DatahubKey from datahub.metadata.com.linkedin.pegasus2avro.dataprocess import ( @@ -26,10 +27,6 @@ from datahub.utilities.urns.data_process_instance_urn import DataProcessInstanceUrn from datahub.utilities.urns.dataset_urn import DatasetUrn -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - from datahub.emitter.rest_emitter import DatahubRestEmitter - class DataProcessInstanceKey(DatahubKey): cluster: str @@ -106,7 +103,7 @@ def start_event_mcp( def emit_process_start( self, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, start_timestamp_millis: int, attempt: Optional[int] = None, emit_template: bool = True, @@ -197,7 +194,7 @@ def end_event_mcp( def emit_process_end( self, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, end_timestamp_millis: int, result: InstanceRunResult, result_type: Optional[str] = None, @@ -207,7 +204,7 @@ def emit_process_end( """ Generate an DataProcessInstance finish event and emits is - :param emitter: (Union[DatahubRestEmitter, DatahubKafkaEmitter]) the datahub emitter to emit generated mcps + :param emitter: (Emitter) the datahub emitter to emit generated mcps :param end_timestamp_millis: (int) the end time of the execution in milliseconds :param result: (InstanceRunResult) The result of the run :param result_type: (string) It identifies the system where the native result comes from like Airflow, Azkaban @@ -261,24 +258,24 @@ def generate_mcp( @staticmethod def _emit_mcp( mcp: MetadataChangeProposalWrapper, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ - :param emitter: (Union[DatahubRestEmitter, DatahubKafkaEmitter]) the datahub emitter to emit generated mcps + :param emitter: (Emitter) the datahub emitter to emit generated mcps :param callback: (Optional[Callable[[Exception, str], None]]) the callback method for KafkaEmitter if it is used """ emitter.emit(mcp, callback) def emit( self, - emitter: Union["DatahubRestEmitter", "DatahubKafkaEmitter"], + emitter: Emitter, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: """ - :param emitter: (Union[DatahubRestEmitter, DatahubKafkaEmitter]) the datahub emitter to emit generated mcps + :param emitter: (Emitter) the datahub emitter to emit generated mcps :param callback: (Optional[Callable[[Exception, str], None]]) the callback method for KafkaEmitter if it is used """ for mcp in self.generate_mcp(): diff --git a/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py b/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py index 04f12b4f61d1e..e1a1555016dba 100644 --- a/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py +++ b/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py @@ -19,8 +19,8 @@ import datahub.emitter.mce_builder as builder from datahub.configuration.common import ConfigModel +from datahub.emitter.generic_emitter import Emitter from datahub.emitter.mcp import MetadataChangeProposalWrapper -from datahub.emitter.rest_emitter import DatahubRestEmitter from datahub.ingestion.graph.client import DataHubGraph from datahub.metadata.schema_classes import ( AuditStampClass, @@ -43,9 +43,6 @@ from datahub.utilities.registries.domain_registry import DomainRegistry from datahub.utilities.urns.urn import Urn -if TYPE_CHECKING: - from datahub.emitter.kafka_emitter import DatahubKafkaEmitter - def patch_list( orig_list: Optional[list], @@ -225,7 +222,6 @@ def _generate_properties_mcp( def generate_mcp( self, upsert: bool ) -> Iterable[Union[MetadataChangeProposalWrapper, MetadataChangeProposalClass]]: - if self._resolved_domain_urn is None: raise Exception( f"Unable to generate MCP-s because we were unable to resolve the domain {self.domain} to an urn." @@ -282,7 +278,7 @@ def generate_mcp( def emit( self, - emitter: Union[DatahubRestEmitter, "DatahubKafkaEmitter"], + emitter: Emitter, upsert: bool, callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: @@ -440,7 +436,6 @@ def patch_yaml( original_dataproduct: DataProduct, output_file: Path, ) -> bool: - update_needed = False if not original_dataproduct._original_yaml_dict: raise Exception("Original Data Product was not loaded from yaml") @@ -523,7 +518,6 @@ def to_yaml( self, file: Path, ) -> None: - with open(file, "w") as fp: yaml = YAML(typ="rt") # default, if not specfied, is 'rt' (round-trip) yaml.indent(mapping=2, sequence=4, offset=2) diff --git a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py index 67469e728f6ef..a5c581d244877 100644 --- a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py @@ -1,5 +1,5 @@ import pathlib -from typing import Callable, Union +from typing import Callable, Optional, Union import filelock @@ -32,7 +32,7 @@ def emit( item: Union[ MetadataChangeEvent, MetadataChangeProposal, MetadataChangeProposalWrapper ], - callback: Callable[[Exception, str], None] | None = None, + callback: Optional[Callable[[Exception, str], None]] = None, ) -> None: with self._lock: if self._filename.exists(): @@ -44,6 +44,10 @@ def emit( write_metadata_file(self._filename, metadata) + def flush(self) -> None: + # No-op. + pass + def close(self) -> None: # No-op. pass diff --git a/metadata-ingestion/src/datahub/ingestion/graph/client.py b/metadata-ingestion/src/datahub/ingestion/graph/client.py index ce5c2ca2a878b..cd112a158de08 100644 --- a/metadata-ingestion/src/datahub/ingestion/graph/client.py +++ b/metadata-ingestion/src/datahub/ingestion/graph/client.py @@ -162,11 +162,11 @@ def from_emitter(cls, emitter: DatahubRestEmitter) -> "DataHubGraph": timeout_sec=emitter._read_timeout_sec, retry_status_codes=emitter._retry_status_codes, retry_max_times=emitter._retry_max_times, + extra_headers=emitter._session.headers, + disable_ssl_verification=emitter._session.verify is False, # TODO: Support these headers. - # extra_headers=emitter._extra_headers, # ca_certificate_path=emitter._ca_certificate_path, # client_certificate_path=emitter._client_certificate_path, - # disable_ssl_verification=emitter._disable_ssl_verification, ) ) From c537166a3318a4333e079b6d1e6926cdb0c5b82c Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 17:18:58 -0700 Subject: [PATCH 27/60] update v1 tests --- docs/lineage/airflow.md | 12 +- .../airflow-plugin/setup.py | 6 +- .../datahub_listener.py | 18 +- .../datahub_airflow_plugin/datahub_plugin.py | 6 +- .../example_dags/astro_snowflake_dag.py | 332 ++++++++++++++++++ .../integration/goldens/v1_basic_iolets.json | 1 - .../integration/goldens/v1_simple_dag.json | 1 - 7 files changed, 353 insertions(+), 23 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index e42bdefc49345..f1d3763e11323 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -15,10 +15,10 @@ The DataHub Airflow plugin supports: There's two actively supported implementations of the plugin, with different Airflow version support. -| Approach | Airflow Version | Notes | -| --------- | --------------- | --------------------------------------------------------------------------------------- | -| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | -| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction; will not extract lineage if the task fails. | +| Approach | Airflow Version | Notes | +| --------- | --------------- | -------------------------------------------------------------------------------------- | +| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | +| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction; may not extract lineage if the task fails. | If you're using Airflow older than 2.1, it's possible to use the v1 plugin with older versions of `acryl-datahub-airflow-plugin`. See the [compatibility section](#compatibility) for more details. @@ -31,7 +31,7 @@ If you're using Airflow older than 2.1, it's possible to use the v1 plugin with The v2 plugin requires Airflow 2.3+ and Python 3.8+. If you don't meet these requirements, use the v1 plugin instead. -```sh +```shell pip install 'acryl-datahub-airflow-plugin[plugin-v2]' ``` @@ -39,7 +39,7 @@ pip install 'acryl-datahub-airflow-plugin[plugin-v2]' Set up a DataHub connection in Airflow. -```sh +```shell airflow connections add --conn-type 'datahub-rest' 'datahub_rest_default' --conn-host 'http://datahub-gms:8080' --conn-password '' ``` diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 4db70f1efb6f8..fa5700fc8648c 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -102,7 +102,11 @@ def get_long_description(): # Extra requirements for loading our test dags. "apache-airflow[snowflake]>=2.0.2", # https://github.com/snowflakedb/snowflake-sqlalchemy/issues/350 - "snowflake-sqlalchemy>=1.4.3", + # Eventually we want to set this to "snowflake-sqlalchemy>=1.4.3". + # However, that doesn't work with older versions of Airflow. Instead + # of splitting this into integration-test-old and integration-test-new, + # adding a bound to SQLAlchemy was the simplest solution. + "sqlalchemy<1.4.42", "virtualenv", # needed by PythonVirtualenvOperator } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 7268241f3bb0b..1e3a262737b01 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -5,7 +5,6 @@ from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast import datahub.emitter.mce_builder as builder -from airflow.listeners import hookimpl from datahub.api.entities.datajob import DataJob from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult from datahub.emitter.rest_emitter import DatahubRestEmitter @@ -36,21 +35,22 @@ from airflow.models import DAG, DagRun, TaskInstance from sqlalchemy.orm import Session + # To placate mypy on Airflow versions that don't have the listener API, + # we define a dummy hookimpl that's an identity function. + _F = TypeVar("_F", bound=Callable[..., None]) + + def hookimpl(f: _F) -> _F: # type: ignore[misc] # noqa: F811 + return f + +else: + from airflow.listeners import hookimpl logger = logging.getLogger(__name__) -_F = TypeVar("_F", bound=Callable[..., None]) _airflow_listener_initialized = False _airflow_listener: Optional["DataHubListener"] = None _RUN_IN_THREAD = True -if TYPE_CHECKING: - # On Airflow versions that don't have the listener API, we placate mypy - # by making hookimpl an identity function. - - def hookimpl(f: _F) -> _F: # type: ignore[misc] # noqa: F811 - return f - def get_airflow_plugin_listener() -> Optional["DataHubListener"]: # Using globals instead of functools.lru_cache to make testing easier. diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index 1404aede19bcb..b074ca96b7695 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -1,8 +1,6 @@ import contextlib import logging import os -from types import ModuleType -from typing import List from airflow.plugins_manager import AirflowPlugin @@ -44,9 +42,7 @@ class DatahubPlugin(AirflowPlugin): get_airflow_plugin_listener, ) - listeners: List[ModuleType] = list( - filter(None, [get_airflow_plugin_listener()]) - ) + listeners: list = list(filter(None, [get_airflow_plugin_listener()])) if not _USE_AIRFLOW_LISTENER_INTERFACE: diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py new file mode 100644 index 0000000000000..9602f6c2eaa75 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py @@ -0,0 +1,332 @@ +# This file is copied from https://docs.astronomer.io/learn/airflow-snowflake. + +from airflow import DAG +from airflow.models.baseoperator import chain +from airflow.operators.empty import EmptyOperator +from airflow.providers.common.sql.operators.sql import ( + SQLColumnCheckOperator, + SQLTableCheckOperator, +) +from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator +from airflow.utils.task_group import TaskGroup +from pendulum import datetime + +SNOWFLAKE_FORESTFIRE_TABLE = "forestfires" +SNOWFLAKE_COST_TABLE = "costs" +SNOWFLAKE_FORESTFIRE_COST_TABLE = "forestfire_costs" + +SNOWFLAKE_CONN_ID = "snowflake_default" + +ROW_COUNT_CHECK = "COUNT(*) = 9" + + +class sql_stmts: + create_forestfire_table = """ + CREATE OR REPLACE TABLE {{ params.table_name }} + ( + id INT, + y INT, + month VARCHAR(25), + day VARCHAR(25), + ffmc FLOAT, + dmc FLOAT, + dc FLOAT, + isi FLOAT, + temp FLOAT, + rh FLOAT, + wind FLOAT, + rain FLOAT, + area FLOAT + ); + """ + + create_cost_table = """ + CREATE OR REPLACE TABLE {{ params.table_name }} + ( + id INT, + land_damage_cost INT, + property_damage_cost INT, + lost_profits_cost INT + ); + """ + + create_forestfire_cost_table = """ + CREATE OR REPLACE TABLE {{ params.table_name }} + ( + id INT, + land_damage_cost INT, + property_damage_cost INT, + lost_profits_cost INT, + total_cost INT, + y INT, + month VARCHAR(25), + day VARCHAR(25), + area FLOAT + ); + """ + + load_forestfire_data = """ + INSERT INTO {{ params.table_name }} VALUES + (1,2,'aug','fri',91,166.9,752.6,7.1,25.9,41,3.6,0,100), + (2,2,'feb','mon',84,9.3,34,2.1,13.9,40,5.4,0,57.8), + (3,4,'mar','sat',69,2.4,15.5,0.7,17.4,24,5.4,0,92.9), + (4,4,'mar','mon',87.2,23.9,64.7,4.1,11.8,35,1.8,0,1300), + (5,5,'mar','sat',91.7,35.8,80.8,7.8,15.1,27,5.4,0,4857), + (6,5,'sep','wed',92.9,133.3,699.6,9.2,26.4,21,4.5,0,9800), + (7,5,'mar','fri',86.2,26.2,94.3,5.1,8.2,51,6.7,0,14), + (8,6,'mar','fri',91.7,33.3,77.5,9,8.3,97,4,0.2,74.5), + (9,9,'feb','thu',84.2,6.8,26.6,7.7,6.7,79,3.1,0,8880.7); + """ + + load_cost_data = """ + INSERT INTO {{ params.table_name }} VALUES + (1,150000,32000,10000), + (2,200000,50000,50000), + (3,90000,120000,300000), + (4,230000,14000,7000), + (5,98000,27000,48000), + (6,72000,800000,0), + (7,50000,2500000,0), + (8,8000000,33000000,0), + (9,6325000,450000,76000); + """ + + load_forestfire_cost_data = """ + INSERT INTO forestfire_costs ( + id, land_damage_cost, property_damage_cost, lost_profits_cost, + total_cost, y, month, day, area + ) + SELECT + c.id, + c.land_damage_cost, + c.property_damage_cost, + c.lost_profits_cost, + c.land_damage_cost + c.property_damage_cost + c.lost_profits_cost, + ff.y, + ff.month, + ff.day, + ff.area + FROM costs c + LEFT JOIN forestfires ff + ON c.id = ff.id + """ + + transform_forestfire_cost_table = """ + SELECT + id, + month, + day, + total_cost, + area, + total_cost / area as cost_per_area + FROM {{ params.table_name }} + """ + + delete_table = """ + DROP TABLE {{ params.table_name }} ; + """ + + +with DAG( + "complex_snowflake_example", + description=""" + Example DAG showcasing loading, transforming, + and data quality checking with multiple datasets in Snowflake. + """, + doc_md=__doc__, + start_date=datetime(2022, 12, 1), + schedule_interval=None, + catchup=False, +) as dag: + """ + #### Snowflake table creation + Create the tables to store sample data. + """ + create_forestfire_table = SnowflakeOperator( + task_id="create_forestfire_table", + sql=sql_stmts.create_forestfire_table, + params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, + ) + + create_cost_table = SnowflakeOperator( + task_id="create_cost_table", + sql=sql_stmts.create_cost_table, + params={"table_name": SNOWFLAKE_COST_TABLE}, + ) + + create_forestfire_cost_table = SnowflakeOperator( + task_id="create_forestfire_cost_table", + sql=sql_stmts.create_forestfire_cost_table, + params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, + ) + + """ + #### Insert data + Insert data into the Snowflake tables using existing SQL queries + stored in the include/sql/snowflake_examples/ directory. + """ + load_forestfire_data = SnowflakeOperator( + task_id="load_forestfire_data", + sql=sql_stmts.load_forestfire_data, + params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, + ) + + load_cost_data = SnowflakeOperator( + task_id="load_cost_data", + sql=sql_stmts.load_cost_data, + params={"table_name": SNOWFLAKE_COST_TABLE}, + ) + + load_forestfire_cost_data = SnowflakeOperator( + task_id="load_forestfire_cost_data", + sql=sql_stmts.load_forestfire_cost_data, + params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, + ) + + """ + #### Transform + Transform the forestfire_costs table to perform + sample logic. + """ + transform_forestfire_cost_table = SnowflakeOperator( + task_id="transform_forestfire_cost_table", + sql=sql_stmts.transform_forestfire_cost_table, + params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, + ) + + """ + #### Quality checks + Perform data quality checks on the various tables. + """ + with TaskGroup( + group_id="quality_check_group_forestfire", + default_args={ + "conn_id": SNOWFLAKE_CONN_ID, + }, + ) as quality_check_group_forestfire: + """ + #### Column-level data quality check + Run data quality checks on columns of the forestfire table + """ + forestfire_column_checks = SQLColumnCheckOperator( + task_id="forestfire_column_checks", + table=SNOWFLAKE_FORESTFIRE_TABLE, + column_mapping={ + "ID": {"null_check": {"equal_to": 0}}, + "RH": {"max": {"leq_to": 100}}, + }, + ) + + """ + #### Table-level data quality check + Run data quality checks on the forestfire table + """ + forestfire_table_checks = SQLTableCheckOperator( + task_id="forestfire_table_checks", + table=SNOWFLAKE_FORESTFIRE_TABLE, + checks={"row_count_check": {"check_statement": ROW_COUNT_CHECK}}, + ) + + with TaskGroup( + group_id="quality_check_group_cost", + default_args={ + "conn_id": SNOWFLAKE_CONN_ID, + }, + ) as quality_check_group_cost: + """ + #### Column-level data quality check + Run data quality checks on columns of the forestfire table + """ + cost_column_checks = SQLColumnCheckOperator( + task_id="cost_column_checks", + table=SNOWFLAKE_COST_TABLE, + column_mapping={ + "ID": {"null_check": {"equal_to": 0}}, + "LAND_DAMAGE_COST": {"min": {"geq_to": 0}}, + "PROPERTY_DAMAGE_COST": {"min": {"geq_to": 0}}, + "LOST_PROFITS_COST": {"min": {"geq_to": 0}}, + }, + ) + + """ + #### Table-level data quality check + Run data quality checks on the forestfire table + """ + cost_table_checks = SQLTableCheckOperator( + task_id="cost_table_checks", + table=SNOWFLAKE_COST_TABLE, + checks={"row_count_check": {"check_statement": ROW_COUNT_CHECK}}, + ) + + with TaskGroup( + group_id="quality_check_group_forestfire_costs", + default_args={ + "conn_id": SNOWFLAKE_CONN_ID, + }, + ) as quality_check_group_forestfire_costs: + """ + #### Column-level data quality check + Run data quality checks on columns of the forestfire table + """ + forestfire_costs_column_checks = SQLColumnCheckOperator( + task_id="forestfire_costs_column_checks", + table=SNOWFLAKE_FORESTFIRE_COST_TABLE, + column_mapping={"AREA": {"min": {"geq_to": 0}}}, + ) + + """ + #### Table-level data quality check + Run data quality checks on the forestfire table + """ + forestfire_costs_table_checks = SQLTableCheckOperator( + task_id="forestfire_costs_table_checks", + table=SNOWFLAKE_FORESTFIRE_COST_TABLE, + checks={ + "row_count_check": {"check_statement": ROW_COUNT_CHECK}, + "total_cost_check": { + "check_statement": "land_damage_cost + \ + property_damage_cost + lost_profits_cost = total_cost" + }, + }, + ) + + """ + #### Delete tables + Clean up the tables created for the example. + """ + delete_forestfire_table = SnowflakeOperator( + task_id="delete_forestfire_table", + sql=sql_stmts.delete_table, + params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, + ) + + delete_cost_table = SnowflakeOperator( + task_id="delete_costs_table", + sql=sql_stmts.delete_table, + params={"table_name": SNOWFLAKE_COST_TABLE}, + ) + + delete_forestfire_cost_table = SnowflakeOperator( + task_id="delete_forestfire_cost_table", + sql=sql_stmts.delete_table, + params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, + ) + + begin = EmptyOperator(task_id="begin") + create_done = EmptyOperator(task_id="create_done") + load_done = EmptyOperator(task_id="load_done") + end = EmptyOperator(task_id="end") + + chain( + begin, + [create_forestfire_table, create_cost_table, create_forestfire_cost_table], + create_done, + [load_forestfire_data, load_cost_data], + load_done, + [quality_check_group_forestfire, quality_check_group_cost], + load_forestfire_cost_data, + quality_check_group_forestfire_costs, + transform_forestfire_cost_table, + [delete_forestfire_table, delete_cost_table, delete_forestfire_cost_table], + end, + ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json index c594c28a11d28..e5082afd704b5 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json @@ -8,7 +8,6 @@ "json": { "customProperties": { "_access_control": "None", - "_default_view": "'tree'", "catchup": "False", "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py'", "is_paused_upon_creation": "None", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json index f079b014cfe1f..d43cdf84df7a7 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -8,7 +8,6 @@ "json": { "customProperties": { "_access_control": "None", - "_default_view": "'tree'", "catchup": "False", "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", "is_paused_upon_creation": "None", From f4639cce2cfbcdcf1eaf9e910991f8c505cf82cf Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 17:45:31 -0700 Subject: [PATCH 28/60] tweak tests --- .../airflow-plugin/build.gradle | 13 +++-------- .../airflow-plugin/setup.cfg | 4 +++- .../integration/goldens/v1_simple_dag.json | 23 ------------------- .../tests/integration/test_plugin.py | 12 ++++++++-- .../airflow-plugin/tox.ini | 2 +- .../api/entities/dataproduct/dataproduct.py | 12 +--------- 6 files changed, 18 insertions(+), 48 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/build.gradle b/metadata-ingestion-modules/airflow-plugin/build.gradle index 64b86a9276621..8f599b2ae0a65 100644 --- a/metadata-ingestion-modules/airflow-plugin/build.gradle +++ b/metadata-ingestion-modules/airflow-plugin/build.gradle @@ -104,21 +104,14 @@ task testSingle(dependsOn: [installDevTest]) { } } -task testQuick(type: Exec, dependsOn: installDevTest) { - // We can't enforce the coverage requirements if we run a subset of the tests. +task test(type: Exec, dependsOn: installDevTest) { inputs.files(project.fileTree(dir: "src/", include: "**/*.py")) inputs.files(project.fileTree(dir: "tests/")) - outputs.dir("${venv_name}") commandLine 'bash', '-x', '-c', - "source ${venv_name}/bin/activate && pytest -vv --continue-on-collection-errors --junit-xml=junit.quick.xml" + "source ${venv_name}/bin/activate && pytest -vv --continue-on-collection-errors --junit-xml=junit.xml" } -task testFull(type: Exec, dependsOn: [testQuick, installDevTest]) { - commandLine 'bash', '-x', '-c', - "source ${venv_name}/bin/activate && pytest -m 'not slow_integration' -vv --continue-on-collection-errors --junit-xml=junit.full.xml" -} - task cleanPythonCache(type: Exec) { commandLine 'bash', '-c', "find src -type f -name '*.py[co]' -delete -o -type d -name __pycache__ -delete -o -type d -empty -delete" @@ -129,7 +122,7 @@ task buildWheel(type: Exec, dependsOn: [install, cleanPythonCache]) { build.dependsOn install check.dependsOn lint -check.dependsOn testQuick +check.dependsOn test clean { delete venv_name diff --git a/metadata-ingestion-modules/airflow-plugin/setup.cfg b/metadata-ingestion-modules/airflow-plugin/setup.cfg index 157bcce1c298d..3e22ddc217908 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.cfg +++ b/metadata-ingestion-modules/airflow-plugin/setup.cfg @@ -41,7 +41,9 @@ ignore_missing_imports = no [tool:pytest] asyncio_mode = auto -addopts = --cov=src --cov-report term-missing --cov-config setup.cfg --strict-markers +addopts = --cov=src --cov-report term-missing --cov-config setup.cfg --strict-markers -s -v +markers = + integration: marks tests to only run in integration (deselect with '-m "not integration"') testpaths = tests/unit diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json index d43cdf84df7a7..c31be6f4eb5f6 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -426,29 +426,6 @@ } } }, -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", - "changeType": "UPSERT", - "aspectName": "dataFlowInfo", - "aspect": { - "json": { - "customProperties": { - "_access_control": "None", - "_default_view": "'tree'", - "catchup": "False", - "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", - "is_paused_upon_creation": "None", - "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", - "tags": "None", - "timezone": "Timezone('UTC')" - }, - "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", - "name": "simple_dag", - "description": "None\n\n" - } - } -}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index d7f4a3c19d954..5c357361c75c3 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -4,8 +4,10 @@ import logging import os import pathlib +import random import signal import subprocess +import time from typing import Iterator, Sequence import pytest @@ -18,6 +20,7 @@ logger = logging.getLogger(__name__) +pytestmark = pytest.mark.integration IS_LOCAL = os.environ.get("CI", "false") == "false" @@ -102,8 +105,10 @@ def _run_airflow( airflow_home = tmp_path / "airflow_home" print(f"Using airflow home: {airflow_home}") - # airflow_port = random.randint(10000, 12000) - airflow_port = 11792 + if IS_LOCAL: + airflow_port = 11792 + else: + airflow_port = random.randint(10000, 12000) print(f"Using airflow port: {airflow_port}") datahub_connection_name = "datahub_file_default" @@ -302,6 +307,9 @@ def test_airflow_plugin( airflow_instance, dag_id, require_success=test_case.success ) + print("Sleeping for a few seconds to let the plugin finish...") + time.sleep(5) + check_golden_file( pytestconfig=pytestconfig, output_path=airflow_instance.metadata_file, diff --git a/metadata-ingestion-modules/airflow-plugin/tox.ini b/metadata-ingestion-modules/airflow-plugin/tox.ini index 6a1c06aed8cdd..7e29ca336b47a 100644 --- a/metadata-ingestion-modules/airflow-plugin/tox.ini +++ b/metadata-ingestion-modules/airflow-plugin/tox.ini @@ -22,7 +22,7 @@ deps = -e ../../metadata-ingestion/[.dev] commands = pytest --cov={envsitepackagesdir}/datahub --cov={envsitepackagesdir}/datahub_provider \ - py3-quick: -m 'not integration and not slow_integration' --junit-xml=junit.quick.xml \ + py3-quick: -m 'not integration' --junit-xml=junit.quick.xml \ py3-full: --cov-fail-under 65 --junit-xml=junit.full.xml \ --continue-on-collection-errors \ -vv diff --git a/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py b/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py index e1a1555016dba..2d9b14ceb2d06 100644 --- a/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py +++ b/metadata-ingestion/src/datahub/api/entities/dataproduct/dataproduct.py @@ -2,17 +2,7 @@ import time from pathlib import Path -from typing import ( - TYPE_CHECKING, - Any, - Callable, - Dict, - Iterable, - List, - Optional, - Tuple, - Union, -) +from typing import Any, Callable, Dict, Iterable, List, Optional, Tuple, Union import pydantic from ruamel.yaml import YAML From a36a036792ec4e7a27ced6ec3506982675c9e666 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 17:55:05 -0700 Subject: [PATCH 29/60] fix _F --- .../src/datahub_airflow_plugin/datahub_listener.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 1e3a262737b01..38090a1c08725 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -31,13 +31,13 @@ from datahub_airflow_plugin.client.airflow_generator import AirflowGenerator from datahub_airflow_plugin.entities import _Entity +_F = TypeVar("_F", bound=Callable[..., None]) if TYPE_CHECKING: from airflow.models import DAG, DagRun, TaskInstance from sqlalchemy.orm import Session # To placate mypy on Airflow versions that don't have the listener API, # we define a dummy hookimpl that's an identity function. - _F = TypeVar("_F", bound=Callable[..., None]) def hookimpl(f: _F) -> _F: # type: ignore[misc] # noqa: F811 return f From 095290533c5fd4d9ec3001b1609f993ad21b5f7e Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 17:58:24 -0700 Subject: [PATCH 30/60] hack to inherit from module type --- .../src/datahub_airflow_plugin/datahub_listener.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 38090a1c08725..850a4a0b4eb9c 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -2,6 +2,7 @@ import functools import logging import threading +import types from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast import datahub.emitter.mce_builder as builder @@ -97,7 +98,10 @@ def wrapper(*args, **kwargs): return cast(_F, wrapper) -class DataHubListener: +# This inherits from types.ModuleType to avoid issues with Airflow's listener plugin loader. +# It previously (v2.4.x and likely other versions too) would throw errors if it was not a module. +# https://github.com/apache/airflow/blob/e99a518970b2d349a75b1647f6b738c8510fa40e/airflow/listeners/listener.py#L56 +class DataHubListener(types.ModuleType): __name__ = "DataHubListener" def __init__(self, config: DatahubLineageConfig): From 7878337ab349b44cd1cdaefc951354e97c41310d Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 18:02:07 -0700 Subject: [PATCH 31/60] fix gradle --- metadata-ingestion-modules/airflow-plugin/build.gradle | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/build.gradle b/metadata-ingestion-modules/airflow-plugin/build.gradle index 8f599b2ae0a65..dacf12dc020df 100644 --- a/metadata-ingestion-modules/airflow-plugin/build.gradle +++ b/metadata-ingestion-modules/airflow-plugin/build.gradle @@ -104,11 +104,11 @@ task testSingle(dependsOn: [installDevTest]) { } } -task test(type: Exec, dependsOn: installDevTest) { +task testQuick(type: Exec, dependsOn: installDevTest) { inputs.files(project.fileTree(dir: "src/", include: "**/*.py")) inputs.files(project.fileTree(dir: "tests/")) commandLine 'bash', '-x', '-c', - "source ${venv_name}/bin/activate && pytest -vv --continue-on-collection-errors --junit-xml=junit.xml" + "source ${venv_name}/bin/activate && pytest -vv --continue-on-collection-errors --junit-xml=junit.quick.xml" } @@ -122,7 +122,7 @@ task buildWheel(type: Exec, dependsOn: [install, cleanPythonCache]) { build.dependsOn install check.dependsOn lint -check.dependsOn test +check.dependsOn testQuick clean { delete venv_name From 505f4efa8d526bb30bd8f78d76f5bb814cc76abe Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 18:22:38 -0700 Subject: [PATCH 32/60] enable debug log in test --- .../datahub_listener.py | 10 +- .../example_dags/astro_snowflake_dag.py | 332 ------------------ .../tests/integration/test_plugin.py | 1 + 3 files changed, 7 insertions(+), 336 deletions(-) delete mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 850a4a0b4eb9c..3c1c38fcdebaa 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -296,9 +296,10 @@ def on_task_instance_running( # This if statement mirrors the logic in https://github.com/OpenLineage/OpenLineage/pull/508. if not hasattr(task_instance, "task"): + # The type ignore is to placate mypy on Airflow 2.1.x. logger.warning( f"No task set for task_id: {task_instance.task_id} - " - f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}" + f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}" # type: ignore[attr-defined] ) return @@ -311,7 +312,8 @@ def on_task_instance_running( task_instance = copy.deepcopy(task_instance) task_instance.render_templates() - dagrun: "DagRun" = task_instance.dag_run + # The type ignore is to placate mypy on Airflow 2.1.x. + dagrun: "DagRun" = task_instance.dag_run # type: ignore[attr-defined] task = task_instance.task dag: "DAG" = task.dag # type: ignore[assignment] @@ -319,7 +321,7 @@ def on_task_instance_running( # Handle async operators in Airflow 2.3 by skipping deferred state. # Inspired by https://github.com/OpenLineage/OpenLineage/pull/1601 - if task_instance.next_method is not None: + if task_instance.next_method is not None: # type: ignore[attr-defined] return datajob = AirflowGenerator.generate_datajob( @@ -357,7 +359,7 @@ def on_task_instance_running( def on_task_instance_finish( self, task_instance: "TaskInstance", status: InstanceRunResult ) -> None: - dagrun: "DagRun" = task_instance.dag_run + dagrun: "DagRun" = task_instance.dag_run # type: ignore[attr-defined] task = self._task_holder.get_task(task_instance) or task_instance.task dag: "DAG" = task.dag # type: ignore[assignment] diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py deleted file mode 100644 index 9602f6c2eaa75..0000000000000 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/astro_snowflake_dag.py +++ /dev/null @@ -1,332 +0,0 @@ -# This file is copied from https://docs.astronomer.io/learn/airflow-snowflake. - -from airflow import DAG -from airflow.models.baseoperator import chain -from airflow.operators.empty import EmptyOperator -from airflow.providers.common.sql.operators.sql import ( - SQLColumnCheckOperator, - SQLTableCheckOperator, -) -from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator -from airflow.utils.task_group import TaskGroup -from pendulum import datetime - -SNOWFLAKE_FORESTFIRE_TABLE = "forestfires" -SNOWFLAKE_COST_TABLE = "costs" -SNOWFLAKE_FORESTFIRE_COST_TABLE = "forestfire_costs" - -SNOWFLAKE_CONN_ID = "snowflake_default" - -ROW_COUNT_CHECK = "COUNT(*) = 9" - - -class sql_stmts: - create_forestfire_table = """ - CREATE OR REPLACE TABLE {{ params.table_name }} - ( - id INT, - y INT, - month VARCHAR(25), - day VARCHAR(25), - ffmc FLOAT, - dmc FLOAT, - dc FLOAT, - isi FLOAT, - temp FLOAT, - rh FLOAT, - wind FLOAT, - rain FLOAT, - area FLOAT - ); - """ - - create_cost_table = """ - CREATE OR REPLACE TABLE {{ params.table_name }} - ( - id INT, - land_damage_cost INT, - property_damage_cost INT, - lost_profits_cost INT - ); - """ - - create_forestfire_cost_table = """ - CREATE OR REPLACE TABLE {{ params.table_name }} - ( - id INT, - land_damage_cost INT, - property_damage_cost INT, - lost_profits_cost INT, - total_cost INT, - y INT, - month VARCHAR(25), - day VARCHAR(25), - area FLOAT - ); - """ - - load_forestfire_data = """ - INSERT INTO {{ params.table_name }} VALUES - (1,2,'aug','fri',91,166.9,752.6,7.1,25.9,41,3.6,0,100), - (2,2,'feb','mon',84,9.3,34,2.1,13.9,40,5.4,0,57.8), - (3,4,'mar','sat',69,2.4,15.5,0.7,17.4,24,5.4,0,92.9), - (4,4,'mar','mon',87.2,23.9,64.7,4.1,11.8,35,1.8,0,1300), - (5,5,'mar','sat',91.7,35.8,80.8,7.8,15.1,27,5.4,0,4857), - (6,5,'sep','wed',92.9,133.3,699.6,9.2,26.4,21,4.5,0,9800), - (7,5,'mar','fri',86.2,26.2,94.3,5.1,8.2,51,6.7,0,14), - (8,6,'mar','fri',91.7,33.3,77.5,9,8.3,97,4,0.2,74.5), - (9,9,'feb','thu',84.2,6.8,26.6,7.7,6.7,79,3.1,0,8880.7); - """ - - load_cost_data = """ - INSERT INTO {{ params.table_name }} VALUES - (1,150000,32000,10000), - (2,200000,50000,50000), - (3,90000,120000,300000), - (4,230000,14000,7000), - (5,98000,27000,48000), - (6,72000,800000,0), - (7,50000,2500000,0), - (8,8000000,33000000,0), - (9,6325000,450000,76000); - """ - - load_forestfire_cost_data = """ - INSERT INTO forestfire_costs ( - id, land_damage_cost, property_damage_cost, lost_profits_cost, - total_cost, y, month, day, area - ) - SELECT - c.id, - c.land_damage_cost, - c.property_damage_cost, - c.lost_profits_cost, - c.land_damage_cost + c.property_damage_cost + c.lost_profits_cost, - ff.y, - ff.month, - ff.day, - ff.area - FROM costs c - LEFT JOIN forestfires ff - ON c.id = ff.id - """ - - transform_forestfire_cost_table = """ - SELECT - id, - month, - day, - total_cost, - area, - total_cost / area as cost_per_area - FROM {{ params.table_name }} - """ - - delete_table = """ - DROP TABLE {{ params.table_name }} ; - """ - - -with DAG( - "complex_snowflake_example", - description=""" - Example DAG showcasing loading, transforming, - and data quality checking with multiple datasets in Snowflake. - """, - doc_md=__doc__, - start_date=datetime(2022, 12, 1), - schedule_interval=None, - catchup=False, -) as dag: - """ - #### Snowflake table creation - Create the tables to store sample data. - """ - create_forestfire_table = SnowflakeOperator( - task_id="create_forestfire_table", - sql=sql_stmts.create_forestfire_table, - params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, - ) - - create_cost_table = SnowflakeOperator( - task_id="create_cost_table", - sql=sql_stmts.create_cost_table, - params={"table_name": SNOWFLAKE_COST_TABLE}, - ) - - create_forestfire_cost_table = SnowflakeOperator( - task_id="create_forestfire_cost_table", - sql=sql_stmts.create_forestfire_cost_table, - params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, - ) - - """ - #### Insert data - Insert data into the Snowflake tables using existing SQL queries - stored in the include/sql/snowflake_examples/ directory. - """ - load_forestfire_data = SnowflakeOperator( - task_id="load_forestfire_data", - sql=sql_stmts.load_forestfire_data, - params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, - ) - - load_cost_data = SnowflakeOperator( - task_id="load_cost_data", - sql=sql_stmts.load_cost_data, - params={"table_name": SNOWFLAKE_COST_TABLE}, - ) - - load_forestfire_cost_data = SnowflakeOperator( - task_id="load_forestfire_cost_data", - sql=sql_stmts.load_forestfire_cost_data, - params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, - ) - - """ - #### Transform - Transform the forestfire_costs table to perform - sample logic. - """ - transform_forestfire_cost_table = SnowflakeOperator( - task_id="transform_forestfire_cost_table", - sql=sql_stmts.transform_forestfire_cost_table, - params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, - ) - - """ - #### Quality checks - Perform data quality checks on the various tables. - """ - with TaskGroup( - group_id="quality_check_group_forestfire", - default_args={ - "conn_id": SNOWFLAKE_CONN_ID, - }, - ) as quality_check_group_forestfire: - """ - #### Column-level data quality check - Run data quality checks on columns of the forestfire table - """ - forestfire_column_checks = SQLColumnCheckOperator( - task_id="forestfire_column_checks", - table=SNOWFLAKE_FORESTFIRE_TABLE, - column_mapping={ - "ID": {"null_check": {"equal_to": 0}}, - "RH": {"max": {"leq_to": 100}}, - }, - ) - - """ - #### Table-level data quality check - Run data quality checks on the forestfire table - """ - forestfire_table_checks = SQLTableCheckOperator( - task_id="forestfire_table_checks", - table=SNOWFLAKE_FORESTFIRE_TABLE, - checks={"row_count_check": {"check_statement": ROW_COUNT_CHECK}}, - ) - - with TaskGroup( - group_id="quality_check_group_cost", - default_args={ - "conn_id": SNOWFLAKE_CONN_ID, - }, - ) as quality_check_group_cost: - """ - #### Column-level data quality check - Run data quality checks on columns of the forestfire table - """ - cost_column_checks = SQLColumnCheckOperator( - task_id="cost_column_checks", - table=SNOWFLAKE_COST_TABLE, - column_mapping={ - "ID": {"null_check": {"equal_to": 0}}, - "LAND_DAMAGE_COST": {"min": {"geq_to": 0}}, - "PROPERTY_DAMAGE_COST": {"min": {"geq_to": 0}}, - "LOST_PROFITS_COST": {"min": {"geq_to": 0}}, - }, - ) - - """ - #### Table-level data quality check - Run data quality checks on the forestfire table - """ - cost_table_checks = SQLTableCheckOperator( - task_id="cost_table_checks", - table=SNOWFLAKE_COST_TABLE, - checks={"row_count_check": {"check_statement": ROW_COUNT_CHECK}}, - ) - - with TaskGroup( - group_id="quality_check_group_forestfire_costs", - default_args={ - "conn_id": SNOWFLAKE_CONN_ID, - }, - ) as quality_check_group_forestfire_costs: - """ - #### Column-level data quality check - Run data quality checks on columns of the forestfire table - """ - forestfire_costs_column_checks = SQLColumnCheckOperator( - task_id="forestfire_costs_column_checks", - table=SNOWFLAKE_FORESTFIRE_COST_TABLE, - column_mapping={"AREA": {"min": {"geq_to": 0}}}, - ) - - """ - #### Table-level data quality check - Run data quality checks on the forestfire table - """ - forestfire_costs_table_checks = SQLTableCheckOperator( - task_id="forestfire_costs_table_checks", - table=SNOWFLAKE_FORESTFIRE_COST_TABLE, - checks={ - "row_count_check": {"check_statement": ROW_COUNT_CHECK}, - "total_cost_check": { - "check_statement": "land_damage_cost + \ - property_damage_cost + lost_profits_cost = total_cost" - }, - }, - ) - - """ - #### Delete tables - Clean up the tables created for the example. - """ - delete_forestfire_table = SnowflakeOperator( - task_id="delete_forestfire_table", - sql=sql_stmts.delete_table, - params={"table_name": SNOWFLAKE_FORESTFIRE_TABLE}, - ) - - delete_cost_table = SnowflakeOperator( - task_id="delete_costs_table", - sql=sql_stmts.delete_table, - params={"table_name": SNOWFLAKE_COST_TABLE}, - ) - - delete_forestfire_cost_table = SnowflakeOperator( - task_id="delete_forestfire_cost_table", - sql=sql_stmts.delete_table, - params={"table_name": SNOWFLAKE_FORESTFIRE_COST_TABLE}, - ) - - begin = EmptyOperator(task_id="begin") - create_done = EmptyOperator(task_id="create_done") - load_done = EmptyOperator(task_id="load_done") - end = EmptyOperator(task_id="end") - - chain( - begin, - [create_forestfire_table, create_cost_table, create_forestfire_cost_table], - create_done, - [load_forestfire_data, load_cost_data], - load_done, - [quality_check_group_forestfire, quality_check_group_cost], - load_forestfire_cost_data, - quality_check_group_forestfire_costs, - transform_forestfire_cost_table, - [delete_forestfire_table, delete_cost_table, delete_forestfire_cost_table], - end, - ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 5c357361c75c3..7631815cbf9a1 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -149,6 +149,7 @@ def _run_airflow( }, ).get_uri(), # Convenience settings. + "AIRFLOW__DATAHUB__LOG_LEVEL": "DEBUG", "SQLALCHEMY_SILENCE_UBER_WARNING": "1", } From 13a5e9d926ba641453636f91bcfc12ffef6032e8 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 18:54:48 -0700 Subject: [PATCH 33/60] list loaded plugins --- .github/workflows/airflow-plugin.yml | 4 ++-- .../airflow-plugin/tests/integration/test_plugin.py | 12 +++++++++++- .../src/datahub/emitter/generic_emitter.py | 4 +++- .../src/datahub/emitter/synchronized_file_emitter.py | 3 +++ 4 files changed, 19 insertions(+), 4 deletions(-) diff --git a/.github/workflows/airflow-plugin.yml b/.github/workflows/airflow-plugin.yml index 5371b93f77b07..a250bddcc16d1 100644 --- a/.github/workflows/airflow-plugin.yml +++ b/.github/workflows/airflow-plugin.yml @@ -45,7 +45,7 @@ jobs: extra_pip_requirements: "apache-airflow~=2.6.0" extra_pip_extras: plugin-v2 - python-version: "3.10" - extra_pip_requirements: "apache-airflow>2.6.0" + extra_pip_requirements: "apache-airflow>=2.7.0" extra_pip_extras: plugin-v2 fail-fast: false steps: @@ -62,7 +62,7 @@ jobs: if: always() run: source metadata-ingestion-modules/airflow-plugin/venv/bin/activate && pip freeze - uses: actions/upload-artifact@v3 - if: ${{ always() && matrix.python-version == '3.10' && matrix.extra_pip_requirements == 'apache-airflow>2.6.0' }} + if: ${{ always() && matrix.python-version == '3.10' && matrix.extra_pip_requirements == 'apache-airflow>=2.7.0' }} with: name: Test Results (Airflow Plugin ${{ matrix.python-version}}) path: | diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 7631815cbf9a1..a0c9d24a6782e 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -181,10 +181,18 @@ def _run_airflow( env=environment, ) + # Sanity check that the plugin got loaded. + if not is_v1: + print("[debug] Listing loaded plugins") + subprocess.check_call( + ["airflow", "plugins", "-v"], + env=environment, + ) + airflow_username = "admin" airflow_password = (airflow_home / "standalone_admin_password.txt").read_text() - yield AirflowInstance( + airflow_instance = AirflowInstance( airflow_home=airflow_home, airflow_port=airflow_port, pid=airflow_process.pid, @@ -193,6 +201,8 @@ def _run_airflow( password=airflow_password, metadata_file=meta_file, ) + + yield airflow_instance finally: # Attempt a graceful shutdown. print("Shutting down airflow...") diff --git a/metadata-ingestion/src/datahub/emitter/generic_emitter.py b/metadata-ingestion/src/datahub/emitter/generic_emitter.py index 2447ce051c56b..28138c6182758 100644 --- a/metadata-ingestion/src/datahub/emitter/generic_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/generic_emitter.py @@ -1,4 +1,6 @@ -from typing import Any, Callable, Optional, Protocol, Union +from typing import Any, Callable, Optional, Union + +from typing_extensions import Protocol from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.metadata.com.linkedin.pegasus2avro.mxe import ( diff --git a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py index a5c581d244877..0a1855e553f35 100644 --- a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py @@ -44,6 +44,9 @@ def emit( write_metadata_file(self._filename, metadata) + def __repr__(self) -> str: + return f"SynchronizedFileEmitter('{self._filename}')" + def flush(self) -> None: # No-op. pass From d4b4161b5cc9fdc1633a92710c672994c259bdc9 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 23:12:31 -0700 Subject: [PATCH 34/60] attempt 1 at refactoring for airflow 2.3-2.4 --- .../datahub_airflow_plugin/_airflow_shims.py | 1 + .../datahub_listener.py | 73 +++++++++++++++---- .../tests/integration/test_plugin.py | 29 ++++++-- 3 files changed, 79 insertions(+), 24 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py index 5e95ba592144b..4f0b7edc56690 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py @@ -27,6 +27,7 @@ AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) HAS_AIRFLOW_LISTENER_API = AIRFLOW_VERSION >= packaging.version.parse("2.3.0.dev0") +HAS_AIRFLOW_DAG_LISTENER_API = AIRFLOW_VERSION >= packaging.version.parse("2.5.0.dev0") def get_task_inlets(operator: "Operator") -> List: diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 3c1c38fcdebaa..271067335edef 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -2,8 +2,8 @@ import functools import logging import threading -import types -from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast +import unittest.mock +from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional, TypeVar, cast import datahub.emitter.mce_builder as builder from datahub.api.entities.datajob import DataJob @@ -22,6 +22,7 @@ from openlineage.client.serde import Serde from datahub_airflow_plugin._airflow_shims import ( + HAS_AIRFLOW_DAG_LISTENER_API, Operator, get_task_inlets, get_task_outlets, @@ -66,6 +67,32 @@ def get_airflow_plugin_listener() -> Optional["DataHubListener"]: if plugin_config.enabled: _airflow_listener = DataHubListener(config=plugin_config) + # On Airflow < 2.5, we monkeypatch the listener manager's + # add_listener method to skip the isinstance check. + # The DAG listener API was added at the same time as this method + # was fixed, so we're reusing the same check variable. + # + # Related Airflow change: https://github.com/apache/airflow/pull/27113. + if not HAS_AIRFLOW_DAG_LISTENER_API: + from airflow.listeners.listener import ( + ListenerManager, + _listener_manager, + ) + + def add_listener(self: "ListenerManager", listener: Any) -> None: + if self.pm.is_registered(listener): + return + self.pm.register(listener) + + if _listener_manager: + unittest.mock.patch.object( + _listener_manager, "add_listener", add_listener + ) + else: + unittest.mock.patch.object( + ListenerManager, "add_listener", add_listener + ) + if plugin_config.disable_openlineage_plugin: # Deactivate the OpenLineagePlugin listener to avoid conflicts. from openlineage.airflow.plugin import OpenLineagePlugin @@ -98,10 +125,7 @@ def wrapper(*args, **kwargs): return cast(_F, wrapper) -# This inherits from types.ModuleType to avoid issues with Airflow's listener plugin loader. -# It previously (v2.4.x and likely other versions too) would throw errors if it was not a module. -# https://github.com/apache/airflow/blob/e99a518970b2d349a75b1647f6b738c8510fa40e/airflow/listeners/listener.py#L56 -class DataHubListener(types.ModuleType): +class DataHubListener: __name__ = "DataHubListener" def __init__(self, config: DatahubLineageConfig): @@ -122,6 +146,11 @@ def __init__(self, config: DatahubLineageConfig): self.extractor_manager = ExtractorManager() + # This "inherits" from types.ModuleType to avoid issues with Airflow's listener plugin loader. + # It previously (v2.4.x and likely other versions too) would throw errors if it was not a module. + # https://github.com/apache/airflow/blob/e99a518970b2d349a75b1647f6b738c8510fa40e/airflow/listeners/listener.py#L56 + # self.__class__ = types.ModuleType + @property def emitter(self): return self._emitter @@ -324,6 +353,13 @@ def on_task_instance_running( if task_instance.next_method is not None: # type: ignore[attr-defined] return + # If we don't have the DAG listener API, we just pretend that + # the start of the task is the start of the DAG. + # This generates duplicate events, but it's better than not + # generating anything. + if not HAS_AIRFLOW_DAG_LISTENER_API: + self.on_dag_start(dagrun) + datajob = AirflowGenerator.generate_datajob( cluster=self.config.cluster, task=task, @@ -420,15 +456,7 @@ def on_task_instance_failed( # TODO: Handle UP_FOR_RETRY state. self.on_task_instance_finish(task_instance, status=InstanceRunResult.FAILURE) - @hookimpl - @run_in_thread - def on_dag_run_running(self, dag_run: "DagRun", msg: str) -> None: - self._set_log_level() - - logger.debug( - f"DataHub listener got notification about dag run start for {dag_run.dag_id}" - ) - + def on_dag_start(self, dag_run: "DagRun") -> None: dag = dag_run.dag if not dag: return @@ -441,6 +469,19 @@ def on_dag_run_running(self, dag_run: "DagRun", msg: str) -> None: ) dataflow.emit(self.emitter, callback=self._make_emit_callback()) - self.emitter.flush() + if HAS_AIRFLOW_DAG_LISTENER_API: + + @hookimpl + @run_in_thread + def on_dag_run_running(self, dag_run: "DagRun", msg: str) -> None: + self._set_log_level() + + logger.debug( + f"DataHub listener got notification about dag run start for {dag_run.dag_id}" + ) + + self.on_dag_start(dag_run) + + self.emitter.flush() # TODO: Add hooks for on_dag_run_success, on_dag_run_failed -> call AirflowGenerator.complete_dataflow diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index a0c9d24a6782e..a36bac05e9a89 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -16,7 +16,10 @@ from airflow.models.connection import Connection from datahub.testing.compare_metadata_json import assert_metadata_files_equal -from datahub_airflow_plugin._airflow_shims import HAS_AIRFLOW_LISTENER_API +from datahub_airflow_plugin._airflow_shims import ( + HAS_AIRFLOW_DAG_LISTENER_API, + HAS_AIRFLOW_LISTENER_API, +) logger = logging.getLogger(__name__) @@ -249,22 +252,36 @@ class DagTestCase: @pytest.mark.parametrize( ["golden_filename", "test_case", "is_v1"], [ + # On Airflow <= 2.2, test plugin v1. *[ pytest.param( f"v1_{test_case.dag_id}", test_case, True, id=f"v1_{test_case.dag_id}", + marks=pytest.mark.skipif( + HAS_AIRFLOW_LISTENER_API, + reason="Not testing plugin v1 on newer Airflow versions", + ), ) for test_case in test_cases if not test_case.v2_only ], *[ pytest.param( - f"v2_{test_case.dag_id}", + # On Airflow 2.3-2.4, test plugin v2 without dataFlows. + f"v2_{test_case.dag_id}" + if HAS_AIRFLOW_DAG_LISTENER_API + else f"v2_{test_case.dag_id}_no_dag_listener", test_case, False, - id=f"v2_{test_case.dag_id}", + id=f"v2_{test_case.dag_id}" + if HAS_AIRFLOW_DAG_LISTENER_API + else f"v2_{test_case.dag_id}_no_dag_listener", + marks=pytest.mark.skipif( + not HAS_AIRFLOW_LISTENER_API, + reason="Cannot test plugin v2 without the Airflow plugin listener API", + ), ) for test_case in test_cases ], @@ -284,11 +301,6 @@ def test_airflow_plugin( # - Waits for the DAG to complete. # - Validates the metadata generated against a golden file. - if not HAS_AIRFLOW_LISTENER_API and not is_v1: - pytest.skip("Cannot test plugin v2 without the Airflow plugin listener API") - if HAS_AIRFLOW_LISTENER_API and is_v1: - pytest.skip("Not testing plugin v1 on older Airflow versions") - dags_folder = pathlib.Path(__file__).parent / "dags" goldens_folder = pathlib.Path(__file__).parent / "goldens" @@ -336,5 +348,6 @@ def test_airflow_plugin( r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['unixname'\]", # TODO: If we switched to Git urls, maybe we could get this to work consistently. r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['fileloc'\]", + r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['openlineage_.*'\]", ], ) From 28c99d6db3b1563a6799686e9fcfcdc7041e7e3c Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 23:15:38 -0700 Subject: [PATCH 35/60] make it a module for compat with <2.5 --- .../_datahub_listener_module.py | 7 +++++ .../datahub_listener.py | 29 +------------------ .../datahub_airflow_plugin/datahub_plugin.py | 25 ++++++++++++---- 3 files changed, 28 insertions(+), 33 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_listener_module.py diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_listener_module.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_listener_module.py new file mode 100644 index 0000000000000..f39d37b122228 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_datahub_listener_module.py @@ -0,0 +1,7 @@ +from datahub_airflow_plugin.datahub_listener import get_airflow_plugin_listener + +_listener = get_airflow_plugin_listener() +if _listener: + on_task_instance_running = _listener.on_task_instance_running + on_task_instance_success = _listener.on_task_instance_success + on_task_instance_failed = _listener.on_task_instance_failed diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 271067335edef..2788f3bce325e 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -2,8 +2,7 @@ import functools import logging import threading -import unittest.mock -from typing import TYPE_CHECKING, Any, Callable, Dict, List, Optional, TypeVar, cast +from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast import datahub.emitter.mce_builder as builder from datahub.api.entities.datajob import DataJob @@ -67,32 +66,6 @@ def get_airflow_plugin_listener() -> Optional["DataHubListener"]: if plugin_config.enabled: _airflow_listener = DataHubListener(config=plugin_config) - # On Airflow < 2.5, we monkeypatch the listener manager's - # add_listener method to skip the isinstance check. - # The DAG listener API was added at the same time as this method - # was fixed, so we're reusing the same check variable. - # - # Related Airflow change: https://github.com/apache/airflow/pull/27113. - if not HAS_AIRFLOW_DAG_LISTENER_API: - from airflow.listeners.listener import ( - ListenerManager, - _listener_manager, - ) - - def add_listener(self: "ListenerManager", listener: Any) -> None: - if self.pm.is_registered(listener): - return - self.pm.register(listener) - - if _listener_manager: - unittest.mock.patch.object( - _listener_manager, "add_listener", add_listener - ) - else: - unittest.mock.patch.object( - ListenerManager, "add_listener", add_listener - ) - if plugin_config.disable_openlineage_plugin: # Deactivate the OpenLineagePlugin listener to avoid conflicts. from openlineage.airflow.plugin import OpenLineagePlugin diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index b074ca96b7695..a9d319edf41d1 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -5,7 +5,10 @@ from airflow.plugins_manager import AirflowPlugin from datahub_airflow_plugin._airflow_compat import AIRFLOW_PATCHED -from datahub_airflow_plugin._airflow_shims import HAS_AIRFLOW_LISTENER_API +from datahub_airflow_plugin._airflow_shims import ( + HAS_AIRFLOW_DAG_LISTENER_API, + HAS_AIRFLOW_LISTENER_API, +) assert AIRFLOW_PATCHED logger = logging.getLogger(__name__) @@ -38,11 +41,23 @@ class DatahubPlugin(AirflowPlugin): name = "datahub_plugin" if _USE_AIRFLOW_LISTENER_INTERFACE: - from datahub_airflow_plugin.datahub_listener import ( # type: ignore[misc] - get_airflow_plugin_listener, - ) + if HAS_AIRFLOW_DAG_LISTENER_API: + from datahub_airflow_plugin.datahub_listener import ( # type: ignore[misc] + get_airflow_plugin_listener, + ) - listeners: list = list(filter(None, [get_airflow_plugin_listener()])) + listeners: list = list(filter(None, [get_airflow_plugin_listener()])) + + else: + # On Airflow < 2.5, we need the listener to be a module. + # This is just a quick shim layer to make that work. + # The DAG listener API was added at the same time as this method + # was fixed, so we're reusing the same check variable. + # + # Related Airflow change: https://github.com/apache/airflow/pull/27113. + import datahub_airflow_plugin._datahub_listener_module as _listener_module # type: ignore[misc] + + listeners = [_listener_module] if not _USE_AIRFLOW_LISTENER_INTERFACE: From 9b0e86626fbc68f3069be9990ee156b61394bade Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 23:18:54 -0700 Subject: [PATCH 36/60] add tests for old plugin v2 --- .../v2_basic_iolets_no_dag_listener.json | 608 ++++++++++++++ .../v2_simple_dag_no_dag_listener.json | 758 ++++++++++++++++++ ...v2_snowflake_operator_no_dag_listener.json | 508 ++++++++++++ 3 files changed, 1874 insertions(+) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json new file mode 100644 index 0000000000000..0ae3536ecfed4 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json @@ -0,0 +1,608 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/basic_iolets.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", + "name": "basic_iolets", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,basic_iolets,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 06:15:41.234925+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1", + "name": "basic_iolets_run_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695881741234, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881741234, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", + "name": "run_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:5d666eaf9015a31b3e305e8bc2dba078", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881741844, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json new file mode 100644 index 0000000000000..9174d0a8e6349 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json @@ -0,0 +1,758 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 06:14:50.599501+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "2", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_task_1_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695881690599, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881690599, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'run_another_data_task'}", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881691164, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 06:14:54.749222+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_run_another_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695881694749, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881694749, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881695389, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json new file mode 100644 index 0000000000000..b8163b82ebad4 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json @@ -0,0 +1,508 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=snowflake_operator", + "name": "snowflake_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 06:16:27.741292+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SnowflakeOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1", + "name": "snowflake_operator_transform_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695881787741, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881787741, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695881788450, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "FAILURE", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file From dcc2fabeeeeed3ffccd57687165bb5145919fb20 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Wed, 27 Sep 2023 23:30:27 -0700 Subject: [PATCH 37/60] fix mypy for 2.1.4 --- .../src/datahub_airflow_plugin/datahub_listener.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 2788f3bce325e..651fd07d9802c 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -300,7 +300,7 @@ def on_task_instance_running( if not hasattr(task_instance, "task"): # The type ignore is to placate mypy on Airflow 2.1.x. logger.warning( - f"No task set for task_id: {task_instance.task_id} - " + f"No task set for task_id: {task_instance.task_id} - " # type: ignore[attr-defined] f"dag_id: {task_instance.dag_id} - run_id {task_instance.run_id}" # type: ignore[attr-defined] ) return From 27391ffe9dc622390b02f5bada12222a47463372 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 10:36:06 -0700 Subject: [PATCH 38/60] update test golden --- .../integration/goldens/v2_simple_dag.json | 187 ++++++++++++++++++ 1 file changed, 187 insertions(+) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index e3b49688ac033..c19b154c8322a 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -161,6 +161,63 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 17:32:19.168164+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "2", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_task_1_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695922339168, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ] + } + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", @@ -194,6 +251,19 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ] + } + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", @@ -205,6 +275,23 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695922339168, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", @@ -317,5 +404,105 @@ "tags": [] } } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695922339632, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695922344075, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1695922344561, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-28 17:32:24.075062+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_run_another_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1695922344075, + "actor": "urn:li:corpuser:datahub" + } + } + } } ] \ No newline at end of file From 77c1133a700569facbb2cd60fdc320a01b0dcc81 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 12:32:39 -0700 Subject: [PATCH 39/60] fix build and test workflow --- .github/workflows/build-and-test.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml index f6320e1bd5c9f..efb492f7fec8a 100644 --- a/.github/workflows/build-and-test.yml +++ b/.github/workflows/build-and-test.yml @@ -26,7 +26,7 @@ jobs: matrix: command: [ - "./gradlew build -x :metadata-ingestion:build -x :metadata-ingestion:check -x docs-website:build -x :metadata-integration:java:spark-lineage:test -x :metadata-io:test -x :metadata-ingestion-modules:airflow-plugin:build -x :datahub-frontend:build -x :datahub-web-react:build --parallel", + "./gradlew build -x :metadata-ingestion:build -x :metadata-ingestion:check -x docs-website:build -x :metadata-integration:java:spark-lineage:test -x :metadata-io:test -x :metadata-ingestion-modules:airflow-plugin:build -x :metadata-ingestion-modules:airflow-plugin:check -x :datahub-frontend:build -x :datahub-web-react:build --parallel", "./gradlew :datahub-frontend:build :datahub-web-react:build --parallel", "./gradlew :metadata-ingestion-modules:airflow-plugin:build --parallel" ] @@ -51,7 +51,8 @@ jobs: java-version: 11 - uses: actions/setup-python@v4 with: - python-version: "3.7" + python-version: "3.10" + cache: pip - name: Gradle build (and test) run: | ${{ matrix.command }} @@ -81,7 +82,7 @@ jobs: - uses: actions/checkout@v3 - uses: actions/setup-python@v4 with: - python-version: "3.7" + python-version: "3.10" - name: Download YQ uses: chrisdickinson/setup-yq@v1.0.1 with: From 18370cdae47064a16c5603ac34fa2f87984209a8 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 13:52:53 -0700 Subject: [PATCH 40/60] handle subprocess kill timeout --- .../tests/integration/test_plugin.py | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index a36bac05e9a89..ab61ba23abf63 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -207,14 +207,16 @@ def _run_airflow( yield airflow_instance finally: - # Attempt a graceful shutdown. - print("Shutting down airflow...") - airflow_process.send_signal(signal.SIGINT) - airflow_process.wait(timeout=30) - - # If the graceful shutdown failed, kill the process. - airflow_process.kill() - airflow_process.wait(timeout=3) + try: + # Attempt a graceful shutdown. + print("Shutting down airflow...") + airflow_process.send_signal(signal.SIGINT) + airflow_process.wait(timeout=30) + except subprocess.TimeoutExpired: + # If the graceful shutdown failed, kill the process. + print("Hard shutting down airflow...") + airflow_process.kill() + airflow_process.wait(timeout=3) def check_golden_file( @@ -331,7 +333,7 @@ def test_airflow_plugin( ) print("Sleeping for a few seconds to let the plugin finish...") - time.sleep(5) + time.sleep(10) check_golden_file( pytestconfig=pytestconfig, From 19a36ce4a569409ded8a0f332a1feee832793a74 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 14:31:42 -0700 Subject: [PATCH 41/60] setup tox tests --- .../airflow-plugin/setup.cfg | 28 ++++++------ .../datahub_airflow_plugin/_airflow_shims.py | 2 + .../tests/integration/test_plugin.py | 4 ++ .../airflow-plugin/tests/unit/test_airflow.py | 17 ++++--- .../airflow-plugin/tox.ini | 44 +++++++++---------- 5 files changed, 49 insertions(+), 46 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/setup.cfg b/metadata-ingestion-modules/airflow-plugin/setup.cfg index 3e22ddc217908..c25256c5751b8 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.cfg +++ b/metadata-ingestion-modules/airflow-plugin/setup.cfg @@ -41,7 +41,7 @@ ignore_missing_imports = no [tool:pytest] asyncio_mode = auto -addopts = --cov=src --cov-report term-missing --cov-config setup.cfg --strict-markers -s -v +addopts = --cov=src --cov-report='' --cov-config setup.cfg --strict-markers -s -v markers = integration: marks tests to only run in integration (deselect with '-m "not integration"') @@ -49,23 +49,21 @@ testpaths = tests/unit tests/integration -[coverage:run] -# Because of some quirks in the way setup.cfg, coverage.py, pytest-cov, -# and tox interact, we should not uncomment the following line. -# See https://pytest-cov.readthedocs.io/en/latest/config.html and -# https://coverage.readthedocs.io/en/coverage-5.0/config.html. -# We also have some additional pytest/cov config options in tox.ini. -# source = src +# [coverage:run] +# # Because of some quirks in the way setup.cfg, coverage.py, pytest-cov, +# # and tox interact, we should not uncomment the following line. +# # See https://pytest-cov.readthedocs.io/en/latest/config.html and +# # https://coverage.readthedocs.io/en/coverage-5.0/config.html. +# # We also have some additional pytest/cov config options in tox.ini. +# # source = src -[coverage:paths] -# This is necessary for tox-based coverage to be counted properly. -source = - src - */site-packages +# [coverage:paths] +# # This is necessary for tox-based coverage to be counted properly. +# source = +# src +# */site-packages [coverage:report] -# The fail_under value ensures that at least some coverage data is collected. -# We override its value in the tox config. show_missing = true exclude_lines = pragma: no cover diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py index 4f0b7edc56690..10f014fbd586f 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_airflow_shims.py @@ -25,7 +25,9 @@ assert AIRFLOW_PATCHED +# Approach suggested by https://stackoverflow.com/a/11887885/5004662. AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) +HAS_AIRFLOW_STANDALONE_CMD = AIRFLOW_VERSION >= packaging.version.parse("2.2.0.dev0") HAS_AIRFLOW_LISTENER_API = AIRFLOW_VERSION >= packaging.version.parse("2.3.0.dev0") HAS_AIRFLOW_DAG_LISTENER_API = AIRFLOW_VERSION >= packaging.version.parse("2.5.0.dev0") diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index ab61ba23abf63..3fba72f1aa245 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -19,6 +19,7 @@ from datahub_airflow_plugin._airflow_shims import ( HAS_AIRFLOW_DAG_LISTENER_API, HAS_AIRFLOW_LISTENER_API, + HAS_AIRFLOW_STANDALONE_CMD, ) logger = logging.getLogger(__name__) @@ -156,6 +157,9 @@ def _run_airflow( "SQLALCHEMY_SILENCE_UBER_WARNING": "1", } + if not HAS_AIRFLOW_STANDALONE_CMD: + raise pytest.skip("Airflow standalone command is not available") + # Start airflow in a background subprocess. airflow_process = subprocess.Popen( ["airflow", "standalone"], diff --git a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py index 9fe1ec443cf38..d8620e74d7e30 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/unit/test_airflow.py @@ -14,18 +14,21 @@ import pytest from airflow.lineage import apply_lineage, prepare_lineage from airflow.models import DAG, Connection, DagBag, DagRun, TaskInstance -from datahub_provider import get_provider_info -from datahub_provider._airflow_shims import AIRFLOW_PATCHED, EmptyOperator -from datahub_provider.entities import Dataset, Urn -from datahub_provider.hooks.datahub import DatahubKafkaHook, DatahubRestHook -from datahub_provider.operators.datahub import DatahubEmitterOperator + +from datahub_airflow_plugin import get_provider_info +from datahub_airflow_plugin._airflow_shims import ( + AIRFLOW_PATCHED, + AIRFLOW_VERSION, + EmptyOperator, +) +from datahub_airflow_plugin.entities import Dataset, Urn +from datahub_airflow_plugin.hooks.datahub import DatahubKafkaHook, DatahubRestHook +from datahub_airflow_plugin.operators.datahub import DatahubEmitterOperator assert AIRFLOW_PATCHED # TODO: Remove default_view="tree" arg. Figure out why is default_view being picked as "grid" and how to fix it ? -# Approach suggested by https://stackoverflow.com/a/11887885/5004662. -AIRFLOW_VERSION = packaging.version.parse(airflow.version.version) lineage_mce = builder.make_lineage_mce( [ diff --git a/metadata-ingestion-modules/airflow-plugin/tox.ini b/metadata-ingestion-modules/airflow-plugin/tox.ini index 7e29ca336b47a..8d7a251058112 100644 --- a/metadata-ingestion-modules/airflow-plugin/tox.ini +++ b/metadata-ingestion-modules/airflow-plugin/tox.ini @@ -4,32 +4,28 @@ # and then run "tox" from this directory. [tox] -envlist = py3-quick,py3-full - -[gh-actions] -python = - 3.6: py3-full - 3.9: py3-full - -# Providing optional features that add dependencies from setup.py as deps here -# allows tox to recreate testenv when new dependencies are added to setup.py. -# Previous approach of using the tox global setting extras is not recommended -# as extras is only called when the testenv is created for the first time! -# see more here -> https://github.com/tox-dev/tox/issues/1105#issuecomment-448596282 +envlist = py38-airflow21, py38-airflow22, py310-airflow24, py310-airflow26, py30-airflow-27 [testenv] -deps = - -e ../../metadata-ingestion/[.dev] +use_develop = true +extras = dev,integration-tests,plugin-v1 +deps = + -e ../../metadata-ingestion/ + # Airflow version + airflow21: apache-airflow~=2.1.0 + airflow22: apache-airflow~=2.2.0 + airflow24: apache-airflow~=2.4.0 + airflow26: apache-airflow~=2.6.0 + airflow27: apache-airflow~=2.7.0 commands = - pytest --cov={envsitepackagesdir}/datahub --cov={envsitepackagesdir}/datahub_provider \ - py3-quick: -m 'not integration' --junit-xml=junit.quick.xml \ - py3-full: --cov-fail-under 65 --junit-xml=junit.full.xml \ - --continue-on-collection-errors \ - -vv + pytest --cov-append {posargs} -setenv = - AIRFLOW_HOME = /tmp/airflow/thisshouldnotexist-{envname} +# For Airflow 2.4+, add the plugin-v2 extra. +[testenv:py310-airflow24] +extras = dev,integration-tests,plugin-v2 -[testenv:py3-full] -deps = - ../../metadata-ingestion/.[dev] +[testenv:py310-airflow26] +extras = dev,integration-tests,plugin-v2 + +[testenv:py30-airflow-27] +extras = dev,integration-tests,plugin-v2 From 83a2621c3bff2a77e3abda3750e5a1ac3833190a Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 14:32:48 -0700 Subject: [PATCH 42/60] fix env list for tox --- metadata-ingestion-modules/airflow-plugin/tox.ini | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tox.ini b/metadata-ingestion-modules/airflow-plugin/tox.ini index 8d7a251058112..055cfea8c6eaf 100644 --- a/metadata-ingestion-modules/airflow-plugin/tox.ini +++ b/metadata-ingestion-modules/airflow-plugin/tox.ini @@ -4,7 +4,7 @@ # and then run "tox" from this directory. [tox] -envlist = py38-airflow21, py38-airflow22, py310-airflow24, py310-airflow26, py30-airflow-27 +envlist = py38-airflow21, py38-airflow22, py310-airflow24, py310-airflow26, py310-airflow-27 [testenv] use_develop = true @@ -27,5 +27,5 @@ extras = dev,integration-tests,plugin-v2 [testenv:py310-airflow26] extras = dev,integration-tests,plugin-v2 -[testenv:py30-airflow-27] +[testenv:py310-airflow-27] extras = dev,integration-tests,plugin-v2 From 7036c4376fb880e2e40d47c0bd26a6ef1c183931 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 15:36:01 -0700 Subject: [PATCH 43/60] slightly more robust tests --- .../airflow-plugin/tests/integration/test_plugin.py | 3 +++ metadata-ingestion-modules/airflow-plugin/tox.ini | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 3fba72f1aa245..923e3196aa619 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -170,6 +170,9 @@ def _run_airflow( _wait_for_airflow_healthy(airflow_port) print("Airflow is ready!") + # Sleep for a few seconds to make sure the other Airflow processes are ready. + time.sleep(3) + # Create an extra "airflow" user for easy testing. if IS_LOCAL: print("Creating an extra test user...") diff --git a/metadata-ingestion-modules/airflow-plugin/tox.ini b/metadata-ingestion-modules/airflow-plugin/tox.ini index 055cfea8c6eaf..81698ff8f0c4d 100644 --- a/metadata-ingestion-modules/airflow-plugin/tox.ini +++ b/metadata-ingestion-modules/airflow-plugin/tox.ini @@ -4,7 +4,7 @@ # and then run "tox" from this directory. [tox] -envlist = py38-airflow21, py38-airflow22, py310-airflow24, py310-airflow26, py310-airflow-27 +envlist = py38-airflow21, py38-airflow22, py310-airflow24, py310-airflow26, py310-airflow27 [testenv] use_develop = true @@ -27,5 +27,5 @@ extras = dev,integration-tests,plugin-v2 [testenv:py310-airflow26] extras = dev,integration-tests,plugin-v2 -[testenv:py310-airflow-27] +[testenv:py310-airflow27] extras = dev,integration-tests,plugin-v2 From 817c73e3350eb4ba3fc73371a4877fc3ddba7411 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 16:40:34 -0700 Subject: [PATCH 44/60] logging improvements --- .../src/datahub_airflow_plugin/_extractors.py | 20 ++++++++------ .../tests/integration/test_plugin.py | 27 ++++++++++++++----- 2 files changed, 32 insertions(+), 15 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py index 3242e8d9657bd..99b12468ca60d 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py @@ -114,9 +114,10 @@ def _sql_extractor_extract(self: "SqlExtractor") -> TaskMetadata: # Prepare to run the SQL parser. graph = self.context.get(_DATAHUB_GRAPH_CONTEXT_KEY, None) - database = getattr(self.operator, "database", None) - if not database: - database = self._get_database() + default_database = getattr(self.operator, "database", None) + if not default_database: + default_database = self.database + default_schema = self.default_schema # TODO: Add better handling for sql being a list of statements. if isinstance(sql, list): @@ -124,21 +125,24 @@ def _sql_extractor_extract(self: "SqlExtractor") -> TaskMetadata: sql = sql[0] # Run the SQL parser. + scheme = self.scheme + platform = OL_SCHEME_TWEAKS.get(scheme, scheme) self.log.debug( - "Running the SQL parser (%s): %s", + "Running the SQL parser %s (platform=%s, default db=%s, schema=%s): %s", "with graph client" if graph else "in offline mode", + platform, + default_database, + default_schema, sql, ) - scheme = self.scheme - platform = OL_SCHEME_TWEAKS.get(scheme, scheme) sql_parsing_result: SqlParsingResult = create_lineage_sql_parsed_result( query=sql, graph=graph, platform=platform, platform_instance=None, env=builder.DEFAULT_ENV, - database=database, - schema=self.default_schema, + database=default_database, + schema=default_schema, ) self.log.debug(f"Got sql lineage {sql_parsing_result}") diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 923e3196aa619..4362b9d429a1c 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -22,11 +22,14 @@ HAS_AIRFLOW_STANDALONE_CMD, ) -logger = logging.getLogger(__name__) - pytestmark = pytest.mark.integration + +logger = logging.getLogger(__name__) IS_LOCAL = os.environ.get("CI", "false") == "false" +DAGS_FOLDER = pathlib.Path(__file__).parent / "dags" +GOLDENS_FOLDER = pathlib.Path(__file__).parent / "goldens" + @dataclasses.dataclass class AirflowInstance: @@ -310,14 +313,11 @@ def test_airflow_plugin( # - Waits for the DAG to complete. # - Validates the metadata generated against a golden file. - dags_folder = pathlib.Path(__file__).parent / "dags" - goldens_folder = pathlib.Path(__file__).parent / "goldens" - - golden_path = goldens_folder / f"{golden_filename}.json" + golden_path = GOLDENS_FOLDER / f"{golden_filename}.json" dag_id = test_case.dag_id with _run_airflow( - tmp_path, dags_folder=dags_folder, is_v1=is_v1 + tmp_path, dags_folder=DAGS_FOLDER, is_v1=is_v1 ) as airflow_instance: print(f"Running DAG {dag_id}...") subprocess.check_call( @@ -360,3 +360,16 @@ def test_airflow_plugin( r"root\[\d+\]\['aspect'\]\['json'\]\['customProperties'\]\['openlineage_.*'\]", ], ) + + +if __name__ == "__main__": + # When run directly, just set up a local airflow instance. + import tempfile + + with _run_airflow( + tmp_path=pathlib.Path(tempfile.mkdtemp("airflow-plugin-test")), + dags_folder=DAGS_FOLDER, + is_v1=not HAS_AIRFLOW_LISTENER_API, + ) as airflow_instance: + input("Press enter to exit...") + print("quitting airflow") From 0b04b305495fe88ef8a748ca89a0c8d7ec516dca Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Thu, 28 Sep 2023 17:33:28 -0700 Subject: [PATCH 45/60] refactor get_platform_from_sqlalchemy_uri into dedicated file --- .../datahub/ingestion/source/kafka_connect.py | 4 +- .../ingestion/source/sql/sql_common.py | 48 ------------------- .../source/sql/sqlalchemy_uri_mapper.py | 48 +++++++++++++++++++ .../src/datahub/ingestion/source/superset.py | 6 ++- .../integrations/great_expectations/action.py | 4 +- .../src/datahub/utilities/sqlglot_lineage.py | 2 +- .../tests/unit/test_sql_common.py | 7 ++- 7 files changed, 62 insertions(+), 57 deletions(-) create mode 100644 metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py diff --git a/metadata-ingestion/src/datahub/ingestion/source/kafka_connect.py b/metadata-ingestion/src/datahub/ingestion/source/kafka_connect.py index f3344782917ab..5fae0ee5215a3 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/kafka_connect.py +++ b/metadata-ingestion/src/datahub/ingestion/source/kafka_connect.py @@ -28,7 +28,9 @@ ) from datahub.ingestion.api.source import MetadataWorkUnitProcessor, Source from datahub.ingestion.api.workunit import MetadataWorkUnit -from datahub.ingestion.source.sql.sql_common import get_platform_from_sqlalchemy_uri +from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( + get_platform_from_sqlalchemy_uri, +) from datahub.ingestion.source.state.stale_entity_removal_handler import ( StaleEntityRemovalHandler, StaleEntityRemovalSourceReport, diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py b/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py index 112defe76d957..056be6c2e50ac 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py @@ -1,12 +1,10 @@ import datetime import logging import traceback -from collections import OrderedDict from dataclasses import dataclass, field from typing import ( TYPE_CHECKING, Any, - Callable, Dict, Iterable, List, @@ -103,52 +101,6 @@ MISSING_COLUMN_INFO = "missing column information" -def _platform_alchemy_uri_tester_gen( - platform: str, opt_starts_with: Optional[str] = None -) -> Tuple[str, Callable[[str], bool]]: - return platform, lambda x: x.startswith( - platform if not opt_starts_with else opt_starts_with - ) - - -PLATFORM_TO_SQLALCHEMY_URI_TESTER_MAP: Dict[str, Callable[[str], bool]] = OrderedDict( - [ - _platform_alchemy_uri_tester_gen("athena", "awsathena"), - _platform_alchemy_uri_tester_gen("bigquery"), - _platform_alchemy_uri_tester_gen("clickhouse"), - _platform_alchemy_uri_tester_gen("druid"), - _platform_alchemy_uri_tester_gen("hana"), - _platform_alchemy_uri_tester_gen("hive"), - _platform_alchemy_uri_tester_gen("mongodb"), - _platform_alchemy_uri_tester_gen("mssql"), - _platform_alchemy_uri_tester_gen("mysql"), - _platform_alchemy_uri_tester_gen("oracle"), - _platform_alchemy_uri_tester_gen("pinot"), - _platform_alchemy_uri_tester_gen("presto"), - ( - "redshift", - lambda x: ( - x.startswith(("jdbc:postgres:", "postgresql")) - and x.find("redshift.amazonaws") > 0 - ) - or x.startswith("redshift"), - ), - # Don't move this before redshift. - _platform_alchemy_uri_tester_gen("postgres", "postgresql"), - _platform_alchemy_uri_tester_gen("snowflake"), - _platform_alchemy_uri_tester_gen("trino"), - _platform_alchemy_uri_tester_gen("vertica"), - ] -) - - -def get_platform_from_sqlalchemy_uri(sqlalchemy_uri: str) -> str: - for platform, tester in PLATFORM_TO_SQLALCHEMY_URI_TESTER_MAP.items(): - if tester(sqlalchemy_uri): - return platform - return "external" - - @dataclass class SQLSourceReport(StaleEntityRemovalSourceReport): tables_scanned: int = 0 diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py new file mode 100644 index 0000000000000..a1cf6122f4566 --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py @@ -0,0 +1,48 @@ +from collections import OrderedDict +from typing import Callable, Dict, Optional, Tuple + + +def _platform_alchemy_uri_tester_gen( + platform: str, opt_starts_with: Optional[str] = None +) -> Tuple[str, Callable[[str], bool]]: + return platform, lambda x: x.startswith( + platform if not opt_starts_with else opt_starts_with + ) + + +PLATFORM_TO_SQLALCHEMY_URI_TESTER_MAP: Dict[str, Callable[[str], bool]] = OrderedDict( + [ + _platform_alchemy_uri_tester_gen("athena", "awsathena"), + _platform_alchemy_uri_tester_gen("bigquery"), + _platform_alchemy_uri_tester_gen("clickhouse"), + _platform_alchemy_uri_tester_gen("druid"), + _platform_alchemy_uri_tester_gen("hana"), + _platform_alchemy_uri_tester_gen("hive"), + _platform_alchemy_uri_tester_gen("mongodb"), + _platform_alchemy_uri_tester_gen("mssql"), + _platform_alchemy_uri_tester_gen("mysql"), + _platform_alchemy_uri_tester_gen("oracle"), + _platform_alchemy_uri_tester_gen("pinot"), + _platform_alchemy_uri_tester_gen("presto"), + ( + "redshift", + lambda x: ( + x.startswith(("jdbc:postgres:", "postgresql")) + and x.find("redshift.amazonaws") > 0 + ) + or x.startswith("redshift"), + ), + # Don't move this before redshift. + _platform_alchemy_uri_tester_gen("postgres", "postgresql"), + _platform_alchemy_uri_tester_gen("snowflake"), + _platform_alchemy_uri_tester_gen("trino"), + _platform_alchemy_uri_tester_gen("vertica"), + ] +) + + +def get_platform_from_sqlalchemy_uri(sqlalchemy_uri: str) -> str: + for platform, tester in PLATFORM_TO_SQLALCHEMY_URI_TESTER_MAP.items(): + if tester(sqlalchemy_uri): + return platform + return "external" diff --git a/metadata-ingestion/src/datahub/ingestion/source/superset.py b/metadata-ingestion/src/datahub/ingestion/source/superset.py index 2a4563439b6ba..14bc4242d2a91 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/superset.py +++ b/metadata-ingestion/src/datahub/ingestion/source/superset.py @@ -21,7 +21,9 @@ ) from datahub.ingestion.api.source import MetadataWorkUnitProcessor, Source from datahub.ingestion.api.workunit import MetadataWorkUnit -from datahub.ingestion.source.sql import sql_common +from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( + get_platform_from_sqlalchemy_uri, +) from datahub.ingestion.source.state.stale_entity_removal_handler import ( StaleEntityRemovalHandler, StaleEntityRemovalSourceReport, @@ -202,7 +204,7 @@ def get_platform_from_database_id(self, database_id): sqlalchemy_uri = database_response.get("result", {}).get("sqlalchemy_uri") if sqlalchemy_uri is None: return database_response.get("result", {}).get("backend", "external") - return sql_common.get_platform_from_sqlalchemy_uri(sqlalchemy_uri) + return get_platform_from_sqlalchemy_uri(sqlalchemy_uri) @lru_cache(maxsize=None) def get_datasource_urn_from_id(self, datasource_id): diff --git a/metadata-ingestion/src/datahub/integrations/great_expectations/action.py b/metadata-ingestion/src/datahub/integrations/great_expectations/action.py index eabf62a4cda2b..f116550328819 100644 --- a/metadata-ingestion/src/datahub/integrations/great_expectations/action.py +++ b/metadata-ingestion/src/datahub/integrations/great_expectations/action.py @@ -35,7 +35,9 @@ from datahub.cli.cli_utils import get_boolean_env_variable from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.rest_emitter import DatahubRestEmitter -from datahub.ingestion.source.sql.sql_common import get_platform_from_sqlalchemy_uri +from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( + get_platform_from_sqlalchemy_uri, +) from datahub.metadata.com.linkedin.pegasus2avro.assertion import ( AssertionInfo, AssertionResult, diff --git a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py index 09dd9420422df..71e1bb2441084 100644 --- a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py +++ b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py @@ -242,7 +242,7 @@ def _table_level_lineage( - modified # ignore CTEs created in this statement - { - _TableName(database=None, schema=None, table=cte.alias_or_name) + _TableName(database=None, db_schema=None, table=cte.alias_or_name) for cte in statement.find_all(sqlglot.exp.CTE) } ) diff --git a/metadata-ingestion/tests/unit/test_sql_common.py b/metadata-ingestion/tests/unit/test_sql_common.py index 95af0e623e991..808b38192411d 100644 --- a/metadata-ingestion/tests/unit/test_sql_common.py +++ b/metadata-ingestion/tests/unit/test_sql_common.py @@ -4,12 +4,11 @@ import pytest from sqlalchemy.engine.reflection import Inspector -from datahub.ingestion.source.sql.sql_common import ( - PipelineContext, - SQLAlchemySource, +from datahub.ingestion.source.sql.sql_common import PipelineContext, SQLAlchemySource +from datahub.ingestion.source.sql.sql_config import SQLCommonConfig +from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( get_platform_from_sqlalchemy_uri, ) -from datahub.ingestion.source.sql.sql_config import SQLCommonConfig class _TestSQLAlchemyConfig(SQLCommonConfig): From f78780060d4313ea006d8ea022cb9d9483705fb8 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 13:21:14 -0700 Subject: [PATCH 46/60] close schema resolver --- .../src/datahub/utilities/sqlglot_lineage.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py index 71e1bb2441084..6f149548543d1 100644 --- a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py +++ b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py @@ -907,21 +907,22 @@ def create_lineage_sql_parsed_result( schema: Optional[str] = None, graph: Optional[DataHubGraph] = None, ) -> SqlParsingResult: + needs_close = False try: - schema_resolver = ( - graph._make_schema_resolver( + if graph: + schema_resolver = graph._make_schema_resolver( platform=platform, platform_instance=platform_instance, env=env, ) - if graph is not None - else SchemaResolver( + else: + needs_close = True + schema_resolver = SchemaResolver( platform=platform, platform_instance=platform_instance, env=env, graph=None, ) - ) return sqlglot_lineage( query, @@ -938,3 +939,6 @@ def create_lineage_sql_parsed_result( table_error=e, ), ) + finally: + if needs_close: + schema_resolver.close() From 1e8dafc81b82c740e68f05e86f0944949299f939 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 14:35:23 -0700 Subject: [PATCH 47/60] add support for arbitrary operators --- .../airflow-plugin/setup.py | 1 + .../src/datahub_airflow_plugin/_extractors.py | 58 +++++++++++++- .../tests/integration/dags/sqlite_operator.py | 75 +++++++++++++++++++ .../tests/integration/test_plugin.py | 11 +++ .../source/sql/sqlalchemy_uri_mapper.py | 1 + 5 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index fa5700fc8648c..7bfc105f303fd 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -108,6 +108,7 @@ def get_long_description(): # adding a bound to SQLAlchemy was the simplest solution. "sqlalchemy<1.4.42", "virtualenv", # needed by PythonVirtualenvOperator + "apache-airflow-providers-sqlite", } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py index 99b12468ca60d..f4f8475434413 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py @@ -4,6 +4,10 @@ from typing import TYPE_CHECKING, Optional import datahub.emitter.mce_builder as builder +from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator +from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( + get_platform_from_sqlalchemy_uri, +) from datahub.utilities.sqlglot_lineage import ( SqlParsingResult, create_lineage_sql_parsed_result, @@ -13,6 +17,7 @@ from openlineage.airflow.extractors import TaskMetadata from openlineage.airflow.extractors.snowflake_extractor import SnowflakeExtractor from openlineage.airflow.extractors.sql_extractor import SqlExtractor +from openlineage.airflow.utils import get_operator_class from openlineage.client.facet import ( ExtractionError, ExtractionErrorRunFacet, @@ -39,13 +44,28 @@ class ExtractorManager(OLExtractorManager): def __init__(self): super().__init__() + _sql_operator_overrides = [ + # The OL BigQuery extractor has some complex logic to fetch detect + # the BigQuery job_id and fetch lineage from there. However, it can't + # generate CLL, so we disable it and use our own extractor instead. + "BigQueryOperator", + "BigQueryExecuteQueryOperator", + # Athena also does something similar. + "AthenaOperator", + "AWSAthenaOperator", + # Additional types that OL doesn't support. This is only necessary because + # on older versions of Airflow, these operators don't inherit from SQLExecuteQueryOperator. + "SqliteOperator", + ] + for operator in _sql_operator_overrides: + self.task_to_extractor.extractors[operator] = GenericSqlExtractor + self._graph: Optional["DataHubGraph"] = None @contextlib.contextmanager def _patch_extractors(self): with contextlib.ExitStack() as stack: # Patch the SqlExtractor.extract() method. - # TODO: Make this work for Airflow 2.7+. stack.enter_context( unittest.mock.patch.object( SqlExtractor, @@ -86,12 +106,48 @@ def extract_metadata( ) def _get_extractor(self, task: "Operator") -> Optional[BaseExtractor]: + # By adding this, we can use the generic extractor as a fallback for + # any operator that inherits from SQLExecuteQueryOperator. + clazz = get_operator_class(task) + if SQLExecuteQueryOperator and issubclass(clazz, SQLExecuteQueryOperator): + self.task_to_extractor.extractors.setdefault( + clazz.__name__, GenericSqlExtractor + ) + extractor = super()._get_extractor(task) if extractor: extractor.set_context(_DATAHUB_GRAPH_CONTEXT_KEY, self._graph) return extractor +class GenericSqlExtractor(SqlExtractor): + # Note that the extract() method is patched elsewhere. + + @property + def default_schema(self): + return super().default_schema + + def _get_scheme(self) -> Optional[str]: + # Best effort conversion to DataHub platform names. + + with contextlib.suppress(Exception): + if self.hook: + if hasattr(self.hook, "get_uri"): + uri = self.hook.get_uri() + return get_platform_from_sqlalchemy_uri(uri) + + return self.conn.conn_type or super().dialect + + def _get_database(self) -> Optional[str]: + if self.conn: + # For BigQuery, the "database" is the project name. + if hasattr(self.conn, "project_id"): + return self.conn.project_id + + return self.conn.schema + return None + + def _sql_extractor_extract(self: "SqlExtractor") -> TaskMetadata: # Why not override the OL sql_parse method directly, instead of overriding # extract()? A few reasons: diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py new file mode 100644 index 0000000000000..77faec3c8935a --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py @@ -0,0 +1,75 @@ +from datetime import datetime + +from airflow import DAG +from airflow.providers.sqlite.operators.sqlite import SqliteOperator + +CONN_ID = "my_sqlite" + +COST_TABLE = "costs" +PROCESSED_TABLE = "processed_costs" + +with DAG( + "sqlite_operator", + start_date=datetime(2023, 1, 1), + schedule_interval=None, + catchup=False, +) as dag: + create_cost_table = SqliteOperator( + sqlite_conn_id=CONN_ID, + task_id="create_cost_table", + sql=""" + CREATE TABLE IF NOT EXISTS {{ params.table_name }} ( + id INTEGER PRIMARY KEY, + month TEXT NOT NULL, + total_cost REAL NOT NULL, + area REAL NOT NULL + ) + """, + params={"table_name": COST_TABLE}, + ) + + populate_cost_table = SqliteOperator( + sqlite_conn_id=CONN_ID, + task_id="populate_cost_table", + sql=""" + INSERT INTO {{ params.table_name }} (id, month, total_cost, area) + VALUES + (1, '2021-01', 100, 10), + (2, '2021-02', 200, 20), + (3, '2021-03', 300, 30) + """, + params={"table_name": COST_TABLE}, + ) + + transform_cost_table = SqliteOperator( + sqlite_conn_id=CONN_ID, + task_id="transform_cost_table", + sql=""" + CREATE TABLE IF NOT EXISTS {{ params.out_table_name }} AS + SELECT + id, + month, + total_cost, + area, + total_cost / area as cost_per_area + FROM {{ params.in_table_name }} + """, + params={ + "in_table_name": COST_TABLE, + "out_table_name": PROCESSED_TABLE, + }, + ) + + cleanup_tables = [] + for table_name in [COST_TABLE, PROCESSED_TABLE]: + cleanup_table = SqliteOperator( + sqlite_conn_id=CONN_ID, + task_id=f"cleanup_{table_name}", + sql=""" + DROP TABLE {{ params.table_name }} + """, + params={"table_name": table_name}, + ) + cleanup_tables.append(cleanup_table) + + create_cost_table >> populate_cost_table >> transform_cost_table >> cleanup_tables diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 4362b9d429a1c..b2fd885380a83 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -155,6 +155,11 @@ def _run_airflow( "insecure_mode": "true", }, ).get_uri(), + "AIRFLOW_CONN_MY_SQLITE": Connection( + conn_id="my_sqlite", + conn_type="sqlite", + host=str(tmp_path / "my_sqlite.db"), + ).get_uri(), # Convenience settings. "AIRFLOW__DATAHUB__LOG_LEVEL": "DEBUG", "SQLALCHEMY_SILENCE_UBER_WARNING": "1", @@ -313,6 +318,11 @@ def test_airflow_plugin( # - Waits for the DAG to complete. # - Validates the metadata generated against a golden file. + if not is_v1 and not test_case.success and not HAS_AIRFLOW_DAG_LISTENER_API: + # Saw a number of issues in CI where this would fail to emit the last events + # due to an error in the SQLAlchemy listener. This never happened locally for me. + pytest.skip("Cannot test failure cases without the Airflow DAG listener API") + golden_path = GOLDENS_FOLDER / f"{golden_filename}.json" dag_id = test_case.dag_id @@ -372,4 +382,5 @@ def test_airflow_plugin( is_v1=not HAS_AIRFLOW_LISTENER_API, ) as airflow_instance: input("Press enter to exit...") + breakpoint() print("quitting airflow") diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py index a1cf6122f4566..70384446cce0e 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py @@ -35,6 +35,7 @@ def _platform_alchemy_uri_tester_gen( # Don't move this before redshift. _platform_alchemy_uri_tester_gen("postgres", "postgresql"), _platform_alchemy_uri_tester_gen("snowflake"), + _platform_alchemy_uri_tester_gen("sqlite"), _platform_alchemy_uri_tester_gen("trino"), _platform_alchemy_uri_tester_gen("vertica"), ] From 9efb96df5a8359bb9f487eb8b23c488f147b9590 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 15:31:57 -0700 Subject: [PATCH 48/60] update tests --- .../src/datahub_airflow_plugin/_extractors.py | 6 +- .../datahub_listener.py | 2 +- .../goldens/v2_sqlite_operator.json | 1808 +++++++++++++++ .../v2_sqlite_operator_no_dag_listener.json | 2032 +++++++++++++++++ .../tests/integration/test_plugin.py | 5 +- .../airflow-plugin/tox.ini | 7 +- 6 files changed, 3849 insertions(+), 11 deletions(-) create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json create mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py index f4f8475434413..f84b7b56f6119 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/_extractors.py @@ -4,7 +4,6 @@ from typing import TYPE_CHECKING, Optional import datahub.emitter.mce_builder as builder -from airflow.providers.common.sql.operators.sql import SQLExecuteQueryOperator from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( get_platform_from_sqlalchemy_uri, ) @@ -17,7 +16,7 @@ from openlineage.airflow.extractors import TaskMetadata from openlineage.airflow.extractors.snowflake_extractor import SnowflakeExtractor from openlineage.airflow.extractors.sql_extractor import SqlExtractor -from openlineage.airflow.utils import get_operator_class +from openlineage.airflow.utils import get_operator_class, try_import_from_string from openlineage.client.facet import ( ExtractionError, ExtractionErrorRunFacet, @@ -109,6 +108,9 @@ def _get_extractor(self, task: "Operator") -> Optional[BaseExtractor]: # By adding this, we can use the generic extractor as a fallback for # any operator that inherits from SQLExecuteQueryOperator. clazz = get_operator_class(task) + SQLExecuteQueryOperator = try_import_from_string( + "airflow.providers.common.sql.operators.sql.SQLExecuteQueryOperator" + ) if SQLExecuteQueryOperator and issubclass(clazz, SQLExecuteQueryOperator): self.task_to_extractor.extractors.setdefault( clazz.__name__, GenericSqlExtractor diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 651fd07d9802c..8fc62d54d537d 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -213,7 +213,7 @@ def _extract_lineage( datajob.properties["datahub_sql_parser_error"] = str( sql_parsing_result.debug_info.error ) - else: + if not sql_parsing_result.debug_info.table_error: input_urns.extend(sql_parsing_result.in_tables) output_urns.extend(sql_parsing_result.out_tables) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json new file mode 100644 index 0000000000000..fd0384ecfc448 --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json @@ -0,0 +1,1808 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'create_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n '", + "task_id": "'create_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'populate_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=create_cost_table", + "name": "create_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:30:05.874635+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "5", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=create_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=create_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_create_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696026605874, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'create_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n '", + "task_id": "'create_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'populate_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=create_cost_table", + "name": "create_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026605874, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'populate_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "\"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"", + "task_id": "'populate_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'transform_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=populate_cost_table", + "name": "populate_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026606364, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:30:10.022225+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "4", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=populate_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=populate_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_populate_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696026610022, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'populate_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "\"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"", + "task_id": "'populate_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'transform_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=populate_cost_table", + "name": "populate_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026610022, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'cleanup_processed_costs', 'cleanup_costs'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)" + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026610696, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:30:14.379259+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "3", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_transform_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696026614379, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026614379, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)" + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026615486, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE costs\\n '", + "task_id": "'cleanup_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_costs", + "name": "cleanup_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:30:21.892294+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_costs&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_costs&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_cleanup_costs_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696026621892, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE costs\\n '", + "task_id": "'cleanup_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_costs", + "name": "cleanup_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026621892, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_processed_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE processed_costs\\n '", + "task_id": "'cleanup_processed_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE processed_costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_processed_costs", + "name": "cleanup_processed_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026623118, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:30:27.087997+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_processed_costs&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_processed_costs&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696026627087, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_processed_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE processed_costs\\n '", + "task_id": "'cleanup_processed_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE processed_costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_processed_costs", + "name": "cleanup_processed_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026627087, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696026628551, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json new file mode 100644 index 0000000000000..f427d7a89755d --- /dev/null +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json @@ -0,0 +1,2032 @@ +[ +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'create_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n '", + "task_id": "'create_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'populate_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=create_cost_table", + "name": "create_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:12:44.638271+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "5", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=create_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=create_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_create_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696025564638, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025564638, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'create_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n '", + "task_id": "'create_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'populate_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS costs (\\n id INTEGER PRIMARY KEY,\\n month TEXT NOT NULL,\\n total_cost REAL NOT NULL,\\n area REAL NOT NULL\\n )\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=create_cost_table", + "name": "create_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025565528, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'populate_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "\"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"", + "task_id": "'populate_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'transform_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=populate_cost_table", + "name": "populate_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:12:48.526035+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "4", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=populate_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=populate_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_populate_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696025568526, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025568526, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'populate_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "\"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"", + "task_id": "'populate_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'transform_cost_table'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=populate_cost_table", + "name": "populate_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025569709, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)" + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:12:52.839611+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "3", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_transform_cost_table_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696025572839, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025572839, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)" + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),month)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),month)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),total_cost)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),area)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),area)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD),total_cost)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD),cost_per_area)" + ], + "confidenceScore": 1.0 + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025574485, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE costs\\n '", + "task_id": "'cleanup_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_costs", + "name": "cleanup_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:12:59.130809+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_costs&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_costs&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_cleanup_costs_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696025579130, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025579130, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE costs\\n '", + "task_id": "'cleanup_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_costs", + "name": "cleanup_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025580994, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator", + "description": "None\n\n" + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_processed_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE processed_costs\\n '", + "task_id": "'cleanup_processed_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE processed_costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_processed_costs", + "name": "cleanup_processed_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceProperties", + "aspect": { + "json": { + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-29 22:13:04.443580+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "SqliteOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_processed_costs&dag_id=sqlite_operator&map_index=-1" + }, + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=cleanup_processed_costs&dag_id=sqlite_operator&map_index=-1", + "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696025584443, + "actor": "urn:li:corpuser:datahub" + } + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRelationships", + "aspect": { + "json": { + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "upstreamInstances": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceInput", + "aspect": { + "json": { + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025584443, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'cleanup_processed_costs'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n DROP TABLE processed_costs\\n '", + "task_id": "'cleanup_processed_costs'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "set()", + "inlets": "[]", + "outlets": "[]", + "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n DROP TABLE processed_costs\\n \"}", + "openlineage_run_facet_extractionError": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/ExtractionErrorRunFacet\", \"errors\": [{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"errorMessage\": \"Can only generate column-level lineage for select-like inner statements, not (outer statement type: )\", \"task\": \"datahub_sql_parser\"}], \"failedTasks\": 1, \"totalTasks\": 1}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=cleanup_processed_costs", + "name": "cleanup_processed_costs", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696025586670, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index b2fd885380a83..cd95b16e0256a 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -254,7 +254,7 @@ def check_golden_file( @dataclasses.dataclass class DagTestCase: dag_id: str - success: bool = False + success: bool = True v2_only: bool = False @@ -263,6 +263,7 @@ class DagTestCase: DagTestCase("simple_dag"), DagTestCase("basic_iolets"), DagTestCase("snowflake_operator", success=False, v2_only=True), + DagTestCase("sqlite_operator", v2_only=True), ] @@ -381,6 +382,6 @@ def test_airflow_plugin( dags_folder=DAGS_FOLDER, is_v1=not HAS_AIRFLOW_LISTENER_API, ) as airflow_instance: - input("Press enter to exit...") + # input("Press enter to exit...") breakpoint() print("quitting airflow") diff --git a/metadata-ingestion-modules/airflow-plugin/tox.ini b/metadata-ingestion-modules/airflow-plugin/tox.ini index 81698ff8f0c4d..2f05854940d10 100644 --- a/metadata-ingestion-modules/airflow-plugin/tox.ini +++ b/metadata-ingestion-modules/airflow-plugin/tox.ini @@ -21,11 +21,6 @@ commands = pytest --cov-append {posargs} # For Airflow 2.4+, add the plugin-v2 extra. -[testenv:py310-airflow24] +[testenv:py310-airflow{24,26,27}] extras = dev,integration-tests,plugin-v2 -[testenv:py310-airflow26] -extras = dev,integration-tests,plugin-v2 - -[testenv:py310-airflow27] -extras = dev,integration-tests,plugin-v2 From 4815c1a569643bee5319812c98a47acb7133a650 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 15:50:00 -0700 Subject: [PATCH 49/60] update desc generation --- .../src/datahub_airflow_plugin/client/airflow_generator.py | 2 +- .../airflow-plugin/tests/integration/dags/simple_dag.py | 1 + .../tests/integration/goldens/v1_basic_iolets.json | 3 +-- .../tests/integration/goldens/v1_simple_dag.json | 2 +- .../tests/integration/goldens/v2_basic_iolets.json | 3 +-- .../tests/integration/goldens/v2_simple_dag.json | 2 +- .../tests/integration/goldens/v2_snowflake_operator.json | 3 +-- .../tests/integration/goldens/v2_sqlite_operator.json | 3 +-- 8 files changed, 8 insertions(+), 11 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index c32ed4a8dc3f5..72f7dd5dfb8a4 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -143,7 +143,7 @@ def generate_dataflow( """ id = dag.dag_id orchestrator = "airflow" - description = f"{dag.description}\n\n{dag.doc_md or ''}" + description = "\n\n".join(filter(None, [dag.description, dag.doc_md])) or None data_flow = DataFlow( env=cluster, id=id, orchestrator=orchestrator, description=description ) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py index 1f4042d971423..1dd047f0a6dcc 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py @@ -10,6 +10,7 @@ start_date=datetime(2023, 1, 1), schedule_interval=None, catchup=False, + description="A simple DAG that runs a few fake data tasks.", ) as dag: task1 = BashOperator( task_id="task_1", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json index e5082afd704b5..ba33105318697 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", - "name": "basic_iolets", - "description": "None\n\n" + "name": "basic_iolets" } } }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json index c31be6f4eb5f6..e9df7575dc105 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -17,7 +17,7 @@ }, "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", "name": "simple_dag", - "description": "None\n\n" + "description": "A simple DAG that runs a few fake data tasks." } } }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json index 90155969c9241..d1800deb1b5af 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", - "name": "basic_iolets", - "description": "None\n\n" + "name": "basic_iolets" } } }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index c19b154c8322a..e72aee5a8da2f 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -17,7 +17,7 @@ }, "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", "name": "simple_dag", - "description": "None\n\n" + "description": "A simple DAG that runs a few fake data tasks." } } }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json index d3196e615656f..8df404e9849c1 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=snowflake_operator", - "name": "snowflake_operator", - "description": "None\n\n" + "name": "snowflake_operator" } } }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json index fd0384ecfc448..fa26eca2f5dba 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" + "name": "sqlite_operator" } } }, From 215b5a1f57f2600e2696ca3acd6e86598d4d5ee2 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 16:05:14 -0700 Subject: [PATCH 50/60] fix sqlglot create table lineage --- .../src/datahub/utilities/sqlglot_lineage.py | 7 +++++++ .../goldens/test_create_table_ddl.json | 8 ++++++++ .../unit/sql_parsing/test_sqlglot_lineage.py | 15 +++++++++++++++ 3 files changed, 30 insertions(+) create mode 100644 metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_table_ddl.json diff --git a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py index 6f149548543d1..4b3090eaaad31 100644 --- a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py +++ b/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py @@ -231,6 +231,13 @@ def _table_level_lineage( # In some cases like "MERGE ... then INSERT (col1, col2) VALUES (col1, col2)", # the `this` on the INSERT part isn't a table. if isinstance(expr.this, sqlglot.exp.Table) + } | { + # For CREATE DDL statements, the table name is nested inside + # a Schema object. + _TableName.from_sqlglot_table(expr.this.this) + for expr in statement.find_all(sqlglot.exp.Create) + if isinstance(expr.this, sqlglot.exp.Schema) + and isinstance(expr.this.this, sqlglot.exp.Table) } tables = ( diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_table_ddl.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_table_ddl.json new file mode 100644 index 0000000000000..4773974545bfa --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_table_ddl.json @@ -0,0 +1,8 @@ +{ + "query_type": "CREATE", + "in_tables": [], + "out_tables": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,costs,PROD)" + ], + "column_lineage": null +} \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py b/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py index 483c1ac4cc7f9..2a965a9bb1e61 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py @@ -274,6 +274,21 @@ def test_expand_select_star_basic(): ) +def test_create_table_ddl(): + assert_sql_result( + """ +CREATE TABLE IF NOT EXISTS costs ( + id INTEGER PRIMARY KEY, + month TEXT NOT NULL, + total_cost REAL NOT NULL, + area REAL NOT NULL +) +""", + dialect="sqlite", + expected_file=RESOURCE_DIR / "test_create_table_ddl.json", + ) + + def test_snowflake_column_normalization(): # Technically speaking this is incorrect since the column names are different and both quoted. From 841c22e113d0d6d03e71cea05db34aab6d6f1465 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 16:25:35 -0700 Subject: [PATCH 51/60] convert downstream set to list --- .../client/airflow_generator.py | 8 +- .../datahub_listener.py | 2 +- .../integration/goldens/v1_basic_iolets.json | 2 +- .../integration/goldens/v1_simple_dag.json | 4 +- .../v2_basic_iolets_no_dag_listener.json | 5 +- .../v2_simple_dag_no_dag_listener.json | 28 +----- .../goldens/v2_sqlite_operator.json | 50 +++-------- .../v2_sqlite_operator_no_dag_listener.json | 90 +++---------------- 8 files changed, 35 insertions(+), 154 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py index 72f7dd5dfb8a4..16585f70e820b 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/client/airflow_generator.py @@ -91,7 +91,7 @@ def _get_dependencies( ) # if the task triggers the subdag, link it to this node in the subdag - if subdag_task_id in _task_downstream_task_ids(upstream_task): + if subdag_task_id in sorted(_task_downstream_task_ids(upstream_task)): upstream_subdag_triggers.append(upstream_task_urn) # If the operator is an ExternalTaskSensor then we set the remote task as upstream. @@ -256,7 +256,11 @@ def generate_datajob( for k in try_keys: if hasattr(task, k): - job_property_bag[out_key] = repr(getattr(task, k)) + v = getattr(task, k) + if out_key == "downstream_task_ids": + # Generate these in a consistent order. + v = list(sorted(v)) + job_property_bag[out_key] = repr(v) break datajob.properties = job_property_bag diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 8fc62d54d537d..186f0ae545b6e 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -89,7 +89,7 @@ def wrapper(*args, **kwargs): ) thread.start() - thread.join(timeout=15) + thread.join(timeout=30) else: f(*args, **kwargs) except Exception as e: diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json index ba33105318697..353a655cc9590 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json @@ -70,7 +70,7 @@ "task_id": "'run_data_task'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]" }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json index e9df7575dc105..bda01983b67b3 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -71,7 +71,7 @@ "task_id": "'task_1'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", + "downstream_task_ids": "['run_another_data_task']", "inlets": "[]", "outlets": "[]" }, @@ -476,7 +476,7 @@ "task_id": "'run_another_data_task'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]" }, diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json index 0ae3536ecfed4..f48abb7dc881e 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=basic_iolets", - "name": "basic_iolets", - "description": "None\n\n" + "name": "basic_iolets" } } }, @@ -71,7 +70,7 @@ "task_id": "'run_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json index 9174d0a8e6349..4db0971442a2a 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json @@ -17,7 +17,7 @@ }, "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", "name": "simple_dag", - "description": "None\n\n" + "description": "A simple DAG that runs a few fake data tasks." } } }, @@ -71,7 +71,7 @@ "task_id": "'task_1'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", + "downstream_task_ids": "['run_another_data_task']", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -461,28 +461,6 @@ } } }, -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", - "changeType": "UPSERT", - "aspectName": "dataFlowInfo", - "aspect": { - "json": { - "customProperties": { - "_access_control": "None", - "catchup": "False", - "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", - "is_paused_upon_creation": "None", - "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", - "tags": "[]", - "timezone": "Timezone('UTC')" - }, - "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", - "name": "simple_dag", - "description": "None\n\n" - } - } -}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", @@ -533,7 +511,7 @@ "task_id": "'run_another_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json index fa26eca2f5dba..77172843fa14c 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json @@ -93,10 +93,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], - "outputDatasets": [], "inputDatajobs": [], "fineGrainedLineages": [] } @@ -193,11 +193,12 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRelationships", + "aspectName": "dataProcessInstanceOutput", "aspect": { "json": { - "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", - "upstreamInstances": [] + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] } } }, @@ -205,12 +206,11 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", + "aspectName": "dataProcessInstanceRelationships", "aspect": { "json": { - "inputs": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" - ] + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "upstreamInstances": [] } } }, @@ -638,36 +638,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'transform_cost_table'", - "execution_timeout": "None", - "sla": "None", - "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", - "task_id": "'transform_cost_table'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "{'cleanup_processed_costs', 'cleanup_costs'}", - "inlets": "[]", - "outlets": "[]", - "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", - "name": "transform_cost_table", - "type": { - "string": "COMMAND" - } - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", @@ -946,7 +916,7 @@ "task_id": "'transform_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", + "downstream_task_ids": "{'cleanup_processed_costs', 'cleanup_costs'}", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json index f427d7a89755d..d6664bfcbc134 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json @@ -16,8 +16,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" + "name": "sqlite_operator" } } }, @@ -72,7 +71,7 @@ "task_id": "'create_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'populate_cost_table'}", + "downstream_task_ids": "['populate_cost_table']", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -94,10 +93,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], - "outputDatasets": [], "inputDatajobs": [], "fineGrainedLineages": [] } @@ -195,10 +194,10 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", + "aspectName": "dataProcessInstanceOutput", "aspect": { "json": { - "inputs": [ + "outputs": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ] } @@ -264,23 +263,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" - ], - "outputDatasets": [], - "inputDatajobs": [], - "fineGrainedLineages": [] - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", @@ -357,28 +339,6 @@ } } }, -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", - "changeType": "UPSERT", - "aspectName": "dataFlowInfo", - "aspect": { - "json": { - "customProperties": { - "_access_control": "None", - "catchup": "False", - "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", - "is_paused_upon_creation": "None", - "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", - "tags": "[]", - "timezone": "Timezone('UTC')" - }, - "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" - } - } -}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", @@ -430,7 +390,7 @@ "task_id": "'populate_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'transform_cost_table'}", + "downstream_task_ids": "['transform_cost_table']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" @@ -788,7 +748,7 @@ "task_id": "'transform_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", + "downstream_task_ids": "['cleanup_costs', 'cleanup_processed_costs']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" @@ -1042,36 +1002,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'transform_cost_table'", - "execution_timeout": "None", - "sla": "None", - "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", - "task_id": "'transform_cost_table'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "{'cleanup_costs', 'cleanup_processed_costs'}", - "inlets": "[]", - "outlets": "[]", - "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", - "name": "transform_cost_table", - "type": { - "string": "COMMAND" - } - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", @@ -1378,7 +1308,7 @@ "task_id": "'cleanup_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1740,7 +1670,7 @@ "task_id": "'cleanup_processed_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", From e558cf17c8fd895a49d0af6155a36e741234129a Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 16:28:24 -0700 Subject: [PATCH 52/60] update more goldens --- .github/workflows/build-and-test.yml | 2 +- .../tests/integration/goldens/v2_basic_iolets.json | 4 ++-- .../tests/integration/goldens/v2_simple_dag.json | 4 ++-- .../tests/integration/goldens/v2_snowflake_operator.json | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml index efb492f7fec8a..3f409878b191f 100644 --- a/.github/workflows/build-and-test.yml +++ b/.github/workflows/build-and-test.yml @@ -26,9 +26,9 @@ jobs: matrix: command: [ + # metadata-ingestion and airflow-plugin each have dedicated build jobs "./gradlew build -x :metadata-ingestion:build -x :metadata-ingestion:check -x docs-website:build -x :metadata-integration:java:spark-lineage:test -x :metadata-io:test -x :metadata-ingestion-modules:airflow-plugin:build -x :metadata-ingestion-modules:airflow-plugin:check -x :datahub-frontend:build -x :datahub-web-react:build --parallel", "./gradlew :datahub-frontend:build :datahub-web-react:build --parallel", - "./gradlew :metadata-ingestion-modules:airflow-plugin:build --parallel" ] timezone: [ diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json index d1800deb1b5af..0464144643ded 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -70,10 +70,10 @@ "task_id": "'run_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", - "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" }, "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=basic_iolets&_flt_3_task_id=run_data_task", "name": "run_data_task", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index e72aee5a8da2f..38f0e08f2aca4 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -104,7 +104,7 @@ "task_id": "'task_1'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", + "downstream_task_ids": "['run_another_data_task']", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -342,7 +342,7 @@ "task_id": "'run_another_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json index 8df404e9849c1..f16e6e208c2c3 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json @@ -71,7 +71,7 @@ "task_id": "'transform_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" From 8c98aad62bb3b5255005cc39eb957ea85039e8a4 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 16:31:20 -0700 Subject: [PATCH 53/60] one more --- .../goldens/v2_sqlite_operator.json | 27 ++++--------------- 1 file changed, 5 insertions(+), 22 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json index 77172843fa14c..6b2b3b6605e89 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json @@ -71,7 +71,7 @@ "task_id": "'create_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'populate_cost_table'}", + "downstream_task_ids": "['populate_cost_table']", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -246,23 +246,6 @@ } } }, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" - ], - "outputDatasets": [], - "inputDatajobs": [], - "fineGrainedLineages": [] - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", @@ -353,7 +336,7 @@ "task_id": "'populate_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'transform_cost_table'}", + "downstream_task_ids": "['transform_cost_table']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" @@ -916,7 +899,7 @@ "task_id": "'transform_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'cleanup_processed_costs', 'cleanup_costs'}", + "downstream_task_ids": "['cleanup_costs', 'cleanup_processed_costs']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" @@ -1179,7 +1162,7 @@ "task_id": "'cleanup_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1465,7 +1448,7 @@ "task_id": "'cleanup_processed_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", From 6db6a3412611a37e065ac74cfe6aff6503a3b08a Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 17:50:42 -0700 Subject: [PATCH 54/60] fix issue with mcp ordering / dedup in test code --- .../datahub_listener.py | 25 ++++++++++++++++--- .../tests/integration/test_plugin.py | 2 ++ .../emitter/synchronized_file_emitter.py | 4 +++ .../datahub/testing/compare_metadata_json.py | 22 +++++++++------- 4 files changed, 41 insertions(+), 12 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 186f0ae545b6e..1632f4d059ec8 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -51,6 +51,7 @@ def hookimpl(f: _F) -> _F: # type: ignore[misc] # noqa: F811 _airflow_listener_initialized = False _airflow_listener: Optional["DataHubListener"] = None _RUN_IN_THREAD = True +_RUN_IN_THREAD_TIMEOUT = 30 def get_airflow_plugin_listener() -> Optional["DataHubListener"]: @@ -76,6 +77,8 @@ def get_airflow_plugin_listener() -> Optional["DataHubListener"]: def run_in_thread(f: _F) -> _F: + # This is also responsible for catching exceptions and logging them. + @functools.wraps(f) def wrapper(*args, **kwargs): try: @@ -89,7 +92,12 @@ def wrapper(*args, **kwargs): ) thread.start() - thread.join(timeout=30) + thread.join(timeout=_RUN_IN_THREAD_TIMEOUT) + if thread.is_alive(): + logger.warning( + f"Thread for {f.__name__} is still running after {_RUN_IN_THREAD_TIMEOUT} seconds. " + "Continuing without waiting for it to finish." + ) else: f(*args, **kwargs) except Exception as e: @@ -272,7 +280,9 @@ def _extract_lineage( for k, v in original_datajob.properties.items(): datajob.properties.setdefault(k, v) - # TODO: Deduplicate inlets/outlets. + # Deduplicate inlets/outlets. + datajob.inlets = list(sorted(set(datajob.inlets))) + datajob.outlets = list(sorted(set(datajob.outlets))) # Write all other OL facets as DataHub properties. if task_metadata: @@ -365,6 +375,10 @@ def on_task_instance_running( self.emitter.flush() + logger.debug( + f"DataHub listener finished processing notification about task instance start for {task_instance.task_id}" + ) + def on_task_instance_finish( self, task_instance: "TaskInstance", status: InstanceRunResult ) -> None: @@ -412,8 +426,10 @@ def on_task_instance_success( logger.debug( f"DataHub listener got notification about task instance success for {task_instance.task_id}" ) - self.on_task_instance_finish(task_instance, status=InstanceRunResult.SUCCESS) + logger.debug( + f"DataHub listener finished processing task instance success for {task_instance.task_id}" + ) @hookimpl @run_in_thread @@ -428,6 +444,9 @@ def on_task_instance_failed( # TODO: Handle UP_FOR_RETRY state. self.on_task_instance_finish(task_instance, status=InstanceRunResult.FAILURE) + logger.debug( + f"DataHub listener finished processing task instance failure for {task_instance.task_id}" + ) def on_dag_start(self, dag_run: "DagRun") -> None: dag = dag_run.dag diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index cd95b16e0256a..729fa015ea81e 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -162,6 +162,7 @@ def _run_airflow( ).get_uri(), # Convenience settings. "AIRFLOW__DATAHUB__LOG_LEVEL": "DEBUG", + "AIRFLOW__DATAHUB__DEBUG_EMITTER": "True", "SQLALCHEMY_SILENCE_UBER_WARNING": "1", } @@ -248,6 +249,7 @@ def check_golden_file( update_golden=update_golden, copy_output=False, ignore_paths=ignore_paths, + ignore_order=False, ) diff --git a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py index 0a1855e553f35..f82882f1a87cc 100644 --- a/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py +++ b/metadata-ingestion/src/datahub/emitter/synchronized_file_emitter.py @@ -1,3 +1,4 @@ +import logging import pathlib from typing import Callable, Optional, Union @@ -13,6 +14,8 @@ MetadataChangeProposal, ) +logger = logging.getLogger(__name__) + class SynchronizedFileEmitter(Closeable, Emitter): """ @@ -40,6 +43,7 @@ def emit( else: metadata = [] + logger.debug("Emitting metadata: %s", item) metadata.append(item) write_metadata_file(self._filename, metadata) diff --git a/metadata-ingestion/src/datahub/testing/compare_metadata_json.py b/metadata-ingestion/src/datahub/testing/compare_metadata_json.py index 5c52e1ab4f0b3..54f6a6e984c00 100644 --- a/metadata-ingestion/src/datahub/testing/compare_metadata_json.py +++ b/metadata-ingestion/src/datahub/testing/compare_metadata_json.py @@ -40,6 +40,7 @@ def assert_metadata_files_equal( update_golden: bool, copy_output: bool, ignore_paths: Sequence[str] = (), + ignore_order: bool = True, ) -> None: golden_exists = os.path.isfile(golden_path) @@ -65,7 +66,7 @@ def assert_metadata_files_equal( write_metadata_file(pathlib.Path(temp.name), golden_metadata) golden = load_json_file(temp.name) - diff = diff_metadata_json(output, golden, ignore_paths) + diff = diff_metadata_json(output, golden, ignore_paths, ignore_order=ignore_order) if diff and update_golden: if isinstance(diff, MCPDiff): diff.apply_delta(golden) @@ -91,16 +92,19 @@ def diff_metadata_json( output: MetadataJson, golden: MetadataJson, ignore_paths: Sequence[str] = (), + ignore_order: bool = True, ) -> Union[DeepDiff, MCPDiff]: ignore_paths = (*ignore_paths, *default_exclude_paths, r"root\[\d+].delta_info") try: - golden_map = get_aspects_by_urn(golden) - output_map = get_aspects_by_urn(output) - return MCPDiff.create( - golden=golden_map, - output=output_map, - ignore_paths=ignore_paths, - ) + if ignore_order: + golden_map = get_aspects_by_urn(golden) + output_map = get_aspects_by_urn(output) + return MCPDiff.create( + golden=golden_map, + output=output_map, + ignore_paths=ignore_paths, + ) + # if ignore_order is False, always use DeepDiff except CannotCompareMCPs as e: logger.info(f"{e}, falling back to MCE diff") except AssertionError as e: @@ -111,5 +115,5 @@ def diff_metadata_json( golden, output, exclude_regex_paths=ignore_paths, - ignore_order=True, + ignore_order=ignore_order, ) From 50e1130c6f5e2f7f5b104065c90c3e362f86336e Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 29 Sep 2023 18:01:24 -0700 Subject: [PATCH 55/60] start fixing more tests --- .../integration/goldens/v1_basic_iolets.json | 17 +- .../integration/goldens/v1_simple_dag.json | 56 +- ...v2_snowflake_operator_no_dag_listener.json | 508 ------------------ 3 files changed, 43 insertions(+), 538 deletions(-) delete mode 100644 metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json index 353a655cc9590..26aa2afaa831a 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_basic_iolets.json @@ -220,7 +220,7 @@ "task_id": "'run_data_task'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]" }, @@ -363,26 +363,23 @@ "json": { "customProperties": { "run_id": "manual_run_test", - "duration": "0.16349", - "start_date": "2023-09-27 22:07:50.874653+00:00", - "end_date": "2023-09-27 22:07:51.038143+00:00", + "duration": "0.176536", + "start_date": "2023-09-30 00:49:56.670239+00:00", + "end_date": "2023-09-30 00:49:56.846775+00:00", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "1", - "hostname": "1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.ip6.arpa", "max_tries": "0", "external_executor_id": "None", - "pid": "60941", "state": "success", "operator": "BashOperator", "priority_weight": "1", - "unixname": "hsheth", "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets" }, "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets", "name": "basic_iolets_run_data_task_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695852470874, + "time": 1696034996670, "actor": "urn:li:corpuser:datahub" } } @@ -503,7 +500,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695852470874, + "timestampMillis": 1696034996670, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -520,7 +517,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695852471038, + "timestampMillis": 1696034996846, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json index bda01983b67b3..b2e3a1fe47da7 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v1_simple_dag.json @@ -185,7 +185,7 @@ "task_id": "'task_1'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", + "downstream_task_ids": "['run_another_data_task']", "inlets": "[]", "outlets": "[]" }, @@ -292,26 +292,23 @@ "json": { "customProperties": { "run_id": "manual_run_test", - "duration": "0.166158", - "start_date": "2023-09-27 21:47:53.144328+00:00", - "end_date": "2023-09-27 21:47:53.310486+00:00", + "duration": "0.175983", + "start_date": "2023-09-30 00:48:58.943850+00:00", + "end_date": "2023-09-30 00:48:59.119833+00:00", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "1", - "hostname": "aluminum-b.local", "max_tries": "0", "external_executor_id": "None", - "pid": "55422", "state": "success", "operator": "BashOperator", "priority_weight": "2", - "unixname": "hsheth", "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag" }, "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=task_1&dag_id=simple_dag", "name": "simple_dag_task_1_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695851273144, + "time": 1696034938943, "actor": "urn:li:corpuser:datahub" } } @@ -396,7 +393,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695851273144, + "timestampMillis": 1696034938943, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -413,7 +410,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695851273310, + "timestampMillis": 1696034939119, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -426,6 +423,28 @@ } } }, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "None", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "A simple DAG that runs a few fake data tasks." + } + } +}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", @@ -554,7 +573,7 @@ "task_id": "'run_another_data_task'", "trigger_rule": "'all_success'", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]" }, @@ -625,26 +644,23 @@ "json": { "customProperties": { "run_id": "manual_run_test", - "duration": "0.125855", - "start_date": "2023-09-27 21:47:56.482097+00:00", - "end_date": "2023-09-27 21:47:56.607952+00:00", + "duration": "0.129888", + "start_date": "2023-09-30 00:49:02.158752+00:00", + "end_date": "2023-09-30 00:49:02.288640+00:00", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "1", - "hostname": "aluminum-b.local", "max_tries": "0", "external_executor_id": "None", - "pid": "55429", "state": "success", "operator": "BashOperator", "priority_weight": "1", - "unixname": "hsheth", "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag" }, "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag", "name": "simple_dag_run_another_data_task_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695851276482, + "time": 1696034942158, "actor": "urn:li:corpuser:datahub" } } @@ -669,7 +685,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695851276482, + "timestampMillis": 1696034942158, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -686,7 +702,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695851276607, + "timestampMillis": 1696034942288, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json deleted file mode 100644 index b8163b82ebad4..0000000000000 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator_no_dag_listener.json +++ /dev/null @@ -1,508 +0,0 @@ -[ -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", - "changeType": "UPSERT", - "aspectName": "dataFlowInfo", - "aspect": { - "json": { - "customProperties": { - "_access_control": "None", - "catchup": "False", - "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/snowflake_operator.py'", - "is_paused_upon_creation": "None", - "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", - "tags": "[]", - "timezone": "Timezone('UTC')" - }, - "externalUrl": "http://airflow.example.com/tree?dag_id=snowflake_operator", - "name": "snowflake_operator", - "description": "None\n\n" - } - } -}, -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", - "changeType": "UPSERT", - "aspectName": "ownership", - "aspect": { - "json": { - "owners": [ - { - "owner": "urn:li:corpuser:airflow", - "type": "DEVELOPER", - "source": { - "type": "SERVICE" - } - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:airflow" - } - } - } -}, -{ - "entityType": "dataFlow", - "entityUrn": "urn:li:dataFlow:(airflow,snowflake_operator,prod)", - "changeType": "UPSERT", - "aspectName": "globalTags", - "aspect": { - "json": { - "tags": [] - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'transform_cost_table'", - "execution_timeout": "None", - "sla": "None", - "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", - "task_id": "'transform_cost_table'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "set()", - "inlets": "[]", - "outlets": "[]", - "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", - "name": "transform_cost_table", - "type": { - "string": "COMMAND" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" - ], - "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" - ], - "inputDatajobs": [], - "fineGrainedLineages": [ - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" - ], - "confidenceScore": 1.0 - } - ] - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "ownership", - "aspect": { - "json": { - "owners": [ - { - "owner": "urn:li:corpuser:airflow", - "type": "DEVELOPER", - "source": { - "type": "SERVICE" - } - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:airflow" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "globalTags", - "aspect": { - "json": { - "tags": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceProperties", - "aspect": { - "json": { - "customProperties": { - "run_id": "manual_run_test", - "duration": "None", - "start_date": "2023-09-28 06:16:27.741292+00:00", - "end_date": "None", - "execution_date": "2023-09-27 21:34:38+00:00", - "try_number": "0", - "max_tries": "0", - "external_executor_id": "None", - "state": "running", - "operator": "SnowflakeOperator", - "priority_weight": "1", - "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1" - }, - "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=transform_cost_table&dag_id=snowflake_operator&map_index=-1", - "name": "snowflake_operator_transform_cost_table_manual_run_test", - "type": "BATCH_AD_HOC", - "created": { - "time": 1695881787741, - "actor": "urn:li:corpuser:datahub" - } - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRelationships", - "aspect": { - "json": { - "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "upstreamInstances": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", - "aspect": { - "json": { - "inputs": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" - ] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceOutput", - "aspect": { - "json": { - "outputs": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" - ] - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695881787741, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInfo", - "aspect": { - "json": { - "customProperties": { - "depends_on_past": "False", - "email": "None", - "label": "'transform_cost_table'", - "execution_timeout": "None", - "sla": "None", - "sql": "'\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", - "task_id": "'transform_cost_table'", - "trigger_rule": "", - "wait_for_downstream": "False", - "downstream_task_ids": "set()", - "inlets": "[]", - "outlets": "[]", - "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" - }, - "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=snowflake_operator&_flt_3_task_id=transform_cost_table", - "name": "transform_cost_table", - "type": { - "string": "COMMAND" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "dataJobInputOutput", - "aspect": { - "json": { - "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)" - ], - "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)" - ], - "inputDatajobs": [], - "fineGrainedLineages": [ - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),id)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),id)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),month)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),month)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),total_cost)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),area)" - ], - "confidenceScore": 1.0 - }, - { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),area)", - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD),total_cost)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD),cost_per_area)" - ], - "confidenceScore": 1.0 - } - ] - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,datahub_test_database.datahub_test_schema.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "ownership", - "aspect": { - "json": { - "owners": [ - { - "owner": "urn:li:corpuser:airflow", - "type": "DEVELOPER", - "source": { - "type": "SERVICE" - } - } - ], - "lastModified": { - "time": 0, - "actor": "urn:li:corpuser:airflow" - } - } - } -}, -{ - "entityType": "dataJob", - "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,snowflake_operator,prod),transform_cost_table)", - "changeType": "UPSERT", - "aspectName": "globalTags", - "aspect": { - "json": { - "tags": [] - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:3161034cc84e16a7c5e1906225734747", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1695881788450, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "FAILURE", - "nativeResultType": "airflow" - } - } - } -} -] \ No newline at end of file From e27729a12e50af5ca12a9b29ddfc3f3cca9ea3de Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Sat, 30 Sep 2023 17:28:10 -0700 Subject: [PATCH 56/60] update v2 tests --- .../datahub_listener.py | 4 +- .../integration/goldens/v2_basic_iolets.json | 101 +--- .../integration/goldens/v2_simple_dag.json | 346 ++++++++---- .../goldens/v2_snowflake_operator.json | 10 +- .../goldens/v2_sqlite_operator.json | 513 +++++++++--------- 5 files changed, 516 insertions(+), 458 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 1632f4d059ec8..3743932a7503b 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -281,8 +281,8 @@ def _extract_lineage( datajob.properties.setdefault(k, v) # Deduplicate inlets/outlets. - datajob.inlets = list(sorted(set(datajob.inlets))) - datajob.outlets = list(sorted(set(datajob.outlets))) + datajob.inlets = list(sorted(set(datajob.inlets), key=lambda x: str(x))) + datajob.outlets = list(sorted(set(datajob.outlets), key=lambda x: str(x))) # Write all other OL facets as DataHub properties. if task_metadata: diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json index 0464144643ded..2e733c2ad40a9 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets.json @@ -91,9 +91,9 @@ "aspect": { "json": { "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ @@ -107,7 +107,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -118,7 +118,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -129,7 +129,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -215,25 +215,22 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-27 22:41:25.543449+00:00", + "start_date": "2023-09-30 01:13:14.266272+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", - "hostname": "aluminum-b.local", "max_tries": "0", "external_executor_id": "None", - "pid": "74129", "state": "running", "operator": "BashOperator", "priority_weight": "1", - "unixname": "hsheth", "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1" }, "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_data_task&dag_id=basic_iolets&map_index=-1", "name": "basic_iolets_run_data_task_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695854485543, + "time": 1696036394266, "actor": "urn:li:corpuser:datahub" } } @@ -259,9 +256,9 @@ "aspect": { "json": { "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ] } @@ -283,7 +280,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -294,7 +291,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -305,7 +302,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -354,7 +351,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695854485543, + "timestampMillis": 1696036394266, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -380,7 +377,7 @@ "task_id": "'run_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -401,18 +398,12 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" ], @@ -421,28 +412,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", @@ -454,17 +423,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", @@ -487,17 +445,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", @@ -531,28 +478,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", @@ -594,7 +519,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695854486150, + "timestampMillis": 1696036394833, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json index 38f0e08f2aca4..454c509279e11 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag.json @@ -55,39 +55,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", @@ -125,13 +92,10 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" ], "inputDatajobs": [], @@ -161,6 +125,51 @@ } } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", @@ -171,7 +180,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-28 17:32:19.168164+00:00", + "start_date": "2023-09-30 06:53:58.219003+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -186,7 +195,7 @@ "name": "simple_dag_task_1_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695922339168, + "time": 1696056838219, "actor": "urn:li:corpuser:datahub" } } @@ -218,6 +227,19 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceOutput", + "aspect": { + "json": { + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" + ] + } + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", @@ -255,18 +277,70 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceOutput", + "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "outputs": [ + "timestampMillis": 1696056838219, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'task_1'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'task_1'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "['run_another_data_task']", + "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", + "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=task_1", + "name": "task_1", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" + ], + "outputDatasets": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" - ] + ], + "inputDatajobs": [], + "fineGrainedLineages": [] } } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -276,19 +350,24 @@ } }, { - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", + "aspectName": "status", "aspect": { "json": { - "timestampMillis": 1695922339168, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 + "removed": false + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false } } }, @@ -326,6 +405,26 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056838648, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", @@ -407,20 +506,31 @@ }, { "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fdbbbcd638bc0e91bbf8d7775efbecaf", + "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", + "aspectName": "dataProcessInstanceProperties", "aspect": { "json": { - "timestampMillis": 1695922339632, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" + "customProperties": { + "run_id": "manual_run_test", + "duration": "None", + "start_date": "2023-09-30 06:54:02.407515+00:00", + "end_date": "None", + "execution_date": "2023-09-27 21:34:38+00:00", + "try_number": "0", + "max_tries": "0", + "external_executor_id": "None", + "state": "running", + "operator": "BashOperator", + "priority_weight": "1", + "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" + "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", + "name": "simple_dag_run_another_data_task_manual_run_test", + "type": "BATCH_AD_HOC", + "created": { + "time": 1696056842407, + "actor": "urn:li:corpuser:datahub" } } } @@ -444,7 +554,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695922344075, + "timestampMillis": 1696056842407, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -455,52 +565,100 @@ } }, { - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", + "aspectName": "dataJobInfo", "aspect": { "json": { - "timestampMillis": 1695922344561, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'run_another_data_task'", + "execution_timeout": "None", + "sla": "None", + "task_id": "'run_another_data_task'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "[]", + "inlets": "[]", + "outlets": "[]", + "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_lock_for_execution\": true, \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_on_exit_code\": [99], \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"wait_for_past_depends_before_skipping\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=simple_dag&_flt_3_task_id=run_another_data_task", + "name": "run_another_data_task", + "type": { + "string": "COMMAND" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "dataJobInputOutput", + "aspect": { + "json": { + "inputDatasets": [], + "outputDatasets": [], + "inputDatajobs": [ + "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),task_1)" + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "ownership", + "aspect": { + "json": { + "owners": [ + { + "owner": "urn:li:corpuser:airflow", + "type": "DEVELOPER", + "source": { + "type": "SERVICE" + } + } + ], + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:airflow" } } } }, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,simple_dag,prod),run_another_data_task)", + "changeType": "UPSERT", + "aspectName": "globalTags", + "aspect": { + "json": { + "tags": [] + } + } +}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:888f71b79d9a0b162fe44acad7b2c2ae", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceProperties", + "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "customProperties": { - "run_id": "manual_run_test", - "duration": "None", - "start_date": "2023-09-28 17:32:24.075062+00:00", - "end_date": "None", - "execution_date": "2023-09-27 21:34:38+00:00", - "try_number": "0", - "max_tries": "0", - "external_executor_id": "None", - "state": "running", - "operator": "BashOperator", - "priority_weight": "1", - "log_url": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1" + "timestampMillis": 1696056842831, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" }, - "externalUrl": "http://airflow.example.com/log?execution_date=2023-09-27T21%3A34%3A38%2B00%3A00&task_id=run_another_data_task&dag_id=simple_dag&map_index=-1", - "name": "simple_dag_run_another_data_task_manual_run_test", - "type": "BATCH_AD_HOC", - "created": { - "time": 1695922344075, - "actor": "urn:li:corpuser:datahub" + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" } } } diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json index f16e6e208c2c3..affc395d421da 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_snowflake_operator.json @@ -225,7 +225,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-27 23:13:23.272262+00:00", + "start_date": "2023-09-30 06:55:36.844976+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -240,7 +240,7 @@ "name": "snowflake_operator_transform_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695856403272, + "time": 1696056936844, "actor": "urn:li:corpuser:datahub" } } @@ -313,7 +313,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695856403272, + "timestampMillis": 1696056936844, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -340,7 +340,7 @@ "task_id": "'transform_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE OR REPLACE TABLE processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" @@ -491,7 +491,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695856404172, + "timestampMillis": 1696056938096, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json index 6b2b3b6605e89..1a32b38ce055d 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator.json @@ -157,7 +157,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:30:05.874635+00:00", + "start_date": "2023-09-30 06:56:24.632190+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -172,20 +172,21 @@ "name": "sqlite_operator_create_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696026605874, + "time": 1696056984632, "actor": "urn:li:corpuser:datahub" } } } }, { - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", "changeType": "UPSERT", - "aspectName": "status", + "aspectName": "dataProcessInstanceRelationships", "aspect": { "json": { - "removed": false + "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", + "upstreamInstances": [] } } }, @@ -202,15 +203,31 @@ } } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRelationships", + "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "parentTemplate": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", - "upstreamInstances": [] + "timestampMillis": 1696056984632, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 } } }, @@ -231,7 +248,7 @@ "task_id": "'create_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'populate_cost_table'}", + "downstream_task_ids": "['populate_cost_table']", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -247,30 +264,18 @@ } }, { - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", + "aspectName": "dataJobInputOutput", "aspect": { "json": { - "timestampMillis": 1696026605874, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 + "inputDatasets": [], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] } } }, @@ -319,6 +324,26 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056984947, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", @@ -378,26 +403,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:fbeed1180fa0434e02ac6f75ace87869", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026606364, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" - } - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", @@ -442,7 +447,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:30:10.022225+00:00", + "start_date": "2023-09-30 06:56:28.605901+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -457,23 +462,12 @@ "name": "sqlite_operator_populate_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696026610022, + "time": 1696056988605, "actor": "urn:li:corpuser:datahub" } } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", @@ -499,6 +493,34 @@ } } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056988605, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", @@ -516,7 +538,7 @@ "task_id": "'populate_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'transform_cost_table'}", + "downstream_task_ids": "['transform_cost_table']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" @@ -537,7 +559,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [], @@ -559,34 +580,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026610022, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", @@ -621,6 +614,56 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056989098, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "['cleanup_costs', 'cleanup_processed_costs']", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", @@ -709,26 +752,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026610696, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" - } - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", @@ -784,7 +807,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:30:14.379259+00:00", + "start_date": "2023-09-30 06:56:32.888165+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -799,7 +822,7 @@ "name": "sqlite_operator_transform_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696026614379, + "time": 1696056992888, "actor": "urn:li:corpuser:datahub" } } @@ -818,50 +841,50 @@ } }, { - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", "changeType": "UPSERT", - "aspectName": "status", + "aspectName": "dataProcessInstanceInput", "aspect": { "json": { - "removed": false + "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ] } } }, { - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", "changeType": "UPSERT", - "aspectName": "status", + "aspectName": "dataProcessInstanceOutput", "aspect": { "json": { - "removed": false + "outputs": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" + ] } } }, { - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceOutput", + "aspectName": "status", "aspect": { "json": { - "outputs": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" - ] + "removed": false } } }, { - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:64e5ff8f552e857b607832731e09808b", + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", + "aspectName": "status", "aspect": { "json": { - "inputs": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" - ] + "removed": false } } }, @@ -872,7 +895,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696026614379, + "timestampMillis": 1696056992888, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -920,11 +943,9 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" ], "inputDatajobs": [ @@ -1058,28 +1079,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", @@ -1132,7 +1131,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696026615486, + "timestampMillis": 1696056993744, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1250,7 +1249,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:30:21.892294+00:00", + "start_date": "2023-09-30 06:56:37.745717+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -1265,23 +1264,12 @@ "name": "sqlite_operator_cleanup_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696026621892, + "time": 1696056997745, "actor": "urn:li:corpuser:datahub" } } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", @@ -1307,6 +1295,34 @@ } } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056997745, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", @@ -1324,7 +1340,7 @@ "task_id": "'cleanup_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1347,7 +1363,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [], @@ -1369,34 +1384,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026621892, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", @@ -1431,6 +1418,26 @@ } } }, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696056998672, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "COMPLETE", + "result": { + "type": "SUCCESS", + "nativeResultType": "airflow" + } + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", @@ -1492,26 +1499,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:07285de22276959612189d51336cc21a", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026623118, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "COMPLETE", - "result": { - "type": "SUCCESS", - "nativeResultType": "airflow" - } - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", @@ -1556,7 +1543,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:30:27.087997+00:00", + "start_date": "2023-09-30 06:56:42.645806+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -1571,23 +1558,12 @@ "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696026627087, + "time": 1696057002645, "actor": "urn:li:corpuser:datahub" } } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", @@ -1613,6 +1589,34 @@ } } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + } +}, +{ + "entityType": "dataProcessInstance", + "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", + "changeType": "UPSERT", + "aspectName": "dataProcessInstanceRunEvent", + "aspect": { + "json": { + "timestampMillis": 1696057002645, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "status": "STARTED", + "attempt": 1 + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", @@ -1630,7 +1634,7 @@ "task_id": "'cleanup_processed_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1653,7 +1657,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" ], "outputDatasets": [], @@ -1675,34 +1678,6 @@ } } }, -{ - "entityType": "dataProcessInstance", - "entityUrn": "urn:li:dataProcessInstance:bab908abccf3cd6607b50fdaf3003372", - "changeType": "UPSERT", - "aspectName": "dataProcessInstanceRunEvent", - "aspect": { - "json": { - "timestampMillis": 1696026627087, - "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" - }, - "status": "STARTED", - "attempt": 1 - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", @@ -1744,7 +1719,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696026628551, + "timestampMillis": 1696057003759, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" From 15f5122377b78b1bd6ad11479e5b7e1d21ad75aa Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Sat, 30 Sep 2023 17:28:17 -0700 Subject: [PATCH 57/60] update v2.4 tests --- .../v2_basic_iolets_no_dag_listener.json | 98 ++-------- .../v2_simple_dag_no_dag_listener.json | 78 ++++---- .../v2_sqlite_operator_no_dag_listener.json | 181 +++++++++--------- 3 files changed, 132 insertions(+), 225 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json index f48abb7dc881e..44b288efda954 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_basic_iolets_no_dag_listener.json @@ -91,9 +91,9 @@ "aspect": { "json": { "inputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ @@ -107,7 +107,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -118,7 +118,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -129,7 +129,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -215,7 +215,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-28 06:15:41.234925+00:00", + "start_date": "2023-09-30 06:59:52.401211+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -230,7 +230,7 @@ "name": "basic_iolets_run_data_task_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695881741234, + "time": 1696057192401, "actor": "urn:li:corpuser:datahub" } } @@ -256,9 +256,9 @@ "aspect": { "json": { "inputs": [ + "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ] } @@ -280,7 +280,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -291,7 +291,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -302,7 +302,7 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -351,7 +351,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881741234, + "timestampMillis": 1696057192401, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -377,7 +377,7 @@ "task_id": "'run_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableB', env='DEV', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableC', env='PROD', platform_instance='cloud'), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None), Dataset(platform='snowflake', name='mydb.schema.tableE', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"task_id\": \"run_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'This is where you might run your data tooling.'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"env\": \"DEV\", \"name\": \"mydb.schema.tableB\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableC\", \"platform\": \"snowflake\", \"platform_instance\": \"cloud\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}, {\"env\": \"PROD\", \"name\": \"mydb.schema.tableE\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -398,18 +398,12 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)" ], @@ -418,28 +412,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableB,DEV)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", @@ -451,17 +423,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", @@ -484,17 +445,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,cloud.mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", @@ -528,28 +478,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableE,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,basic_iolets,prod),run_data_task)", @@ -591,7 +519,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881741844, + "timestampMillis": 1696057192982, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json index 4db0971442a2a..73b5765e96b7d 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_simple_dag_no_dag_listener.json @@ -180,7 +180,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-28 06:14:50.599501+00:00", + "start_date": "2023-09-30 06:58:56.105026+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -195,7 +195,7 @@ "name": "simple_dag_task_1_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695881690599, + "time": 1696057136105, "actor": "urn:li:corpuser:datahub" } } @@ -280,7 +280,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881690599, + "timestampMillis": 1696057136105, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -306,7 +306,7 @@ "task_id": "'task_1'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'run_another_data_task'}", + "downstream_task_ids": "['run_another_data_task']", "inlets": "[Dataset(platform='snowflake', name='mydb.schema.tableA', env='PROD', platform_instance=None), Urn(_urn='urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)')]", "outlets": "[Dataset(platform='snowflake', name='mydb.schema.tableD', env='PROD', platform_instance=None)]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 1'\", \"dag\": \"<>\", \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"task_id\": \"task_1\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 1'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [\"run_another_data_task\"], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableA\", \"platform\": \"snowflake\"}, {\"_urn\": \"urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)\"}], \"outlets\": [{\"env\": \"PROD\", \"name\": \"mydb.schema.tableD\", \"platform\": \"snowflake\"}], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"task_1\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -327,13 +327,10 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)" ], "inputDatajobs": [], @@ -363,39 +360,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableA,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableC,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:snowflake,mydb.schema.tableD,PROD)", @@ -448,7 +412,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881691164, + "timestampMillis": 1696057136612, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -461,6 +425,28 @@ } } }, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/simple_dag.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=simple_dag", + "name": "simple_dag", + "description": "A simple DAG that runs a few fake data tasks." + } + } +}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,simple_dag,prod)", @@ -584,7 +570,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-28 06:14:54.749222+00:00", + "start_date": "2023-09-30 06:58:59.567004+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -599,7 +585,7 @@ "name": "simple_dag_run_another_data_task_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1695881694749, + "time": 1696057139567, "actor": "urn:li:corpuser:datahub" } } @@ -624,7 +610,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881694749, + "timestampMillis": 1696057139567, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -650,7 +636,7 @@ "task_id": "'run_another_data_task'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "openlineage_run_facet_unknownSourceAttribute": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/BaseFacet\", \"unknownItems\": [{\"name\": \"BashOperator\", \"properties\": {\"_BaseOperator__from_mapped\": false, \"_BaseOperator__init_kwargs\": {\"bash_command\": \"echo 'task 2'\", \"dag\": \"<>\", \"task_id\": \"run_another_data_task\"}, \"_BaseOperator__instantiated\": true, \"_dag\": \"<>\", \"_log\": \"<>\", \"append_env\": false, \"bash_command\": \"echo 'task 2'\", \"depends_on_past\": false, \"do_xcom_push\": true, \"downstream_task_ids\": [], \"email_on_failure\": true, \"email_on_retry\": true, \"executor_config\": {}, \"ignore_first_depends_on_past\": true, \"inlets\": [], \"outlets\": [], \"output_encoding\": \"utf-8\", \"owner\": \"airflow\", \"params\": \"<>\", \"pool\": \"default_pool\", \"pool_slots\": 1, \"priority_weight\": 1, \"queue\": \"default\", \"retries\": 0, \"retry_delay\": \"<>\", \"retry_exponential_backoff\": false, \"skip_exit_code\": 99, \"start_date\": \"<>\", \"task_group\": \"<>\", \"task_id\": \"run_another_data_task\", \"trigger_rule\": \"all_success\", \"upstream_task_ids\": [\"task_1\"], \"wait_for_downstream\": false, \"weight_rule\": \"downstream\"}, \"type\": \"operator\"}]}" @@ -720,7 +706,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1695881695389, + "timestampMillis": 1696057140164, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json index d6664bfcbc134..c082be693e30c 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/goldens/v2_sqlite_operator_no_dag_listener.json @@ -157,7 +157,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:12:44.638271+00:00", + "start_date": "2023-09-30 07:00:45.832554+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -172,7 +172,7 @@ "name": "sqlite_operator_create_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696025564638, + "time": 1696057245832, "actor": "urn:li:corpuser:datahub" } } @@ -221,7 +221,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025564638, + "timestampMillis": 1696057245832, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -248,7 +248,7 @@ "task_id": "'create_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'populate_cost_table'}", + "downstream_task_ids": "['populate_cost_table']", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -264,13 +264,18 @@ } }, { - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),create_cost_table)", "changeType": "UPSERT", - "aspectName": "status", + "aspectName": "dataJobInputOutput", "aspect": { "json": { - "removed": false + "inputDatasets": [], + "outputDatasets": [ + "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" + ], + "inputDatajobs": [], + "fineGrainedLineages": [] } } }, @@ -326,7 +331,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025565528, + "timestampMillis": 1696057246734, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -339,6 +344,27 @@ } } }, +{ + "entityType": "dataFlow", + "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", + "changeType": "UPSERT", + "aspectName": "dataFlowInfo", + "aspect": { + "json": { + "customProperties": { + "_access_control": "None", + "catchup": "False", + "fileloc": "'/Users/hsheth/projects/datahub/metadata-ingestion-modules/airflow-plugin/tests/integration/dags/sqlite_operator.py'", + "is_paused_upon_creation": "None", + "start_date": "DateTime(2023, 1, 1, 0, 0, 0, tzinfo=Timezone('UTC'))", + "tags": "[]", + "timezone": "Timezone('UTC')" + }, + "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", + "name": "sqlite_operator" + } + } +}, { "entityType": "dataFlow", "entityUrn": "urn:li:dataFlow:(airflow,sqlite_operator,prod)", @@ -476,7 +502,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:12:48.526035+00:00", + "start_date": "2023-09-30 07:00:49.653938+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -491,7 +517,7 @@ "name": "sqlite_operator_populate_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696025568526, + "time": 1696057249653, "actor": "urn:li:corpuser:datahub" } } @@ -540,7 +566,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025568526, + "timestampMillis": 1696057249653, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -567,7 +593,7 @@ "task_id": "'populate_cost_table'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "{'transform_cost_table'}", + "downstream_task_ids": "['transform_cost_table']", "inlets": "[]", "outlets": "[]", "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n INSERT INTO costs (id, month, total_cost, area)\\n VALUES\\n (1, '2021-01', 100, 10),\\n (2, '2021-02', 200, 20),\\n (3, '2021-03', 300, 30)\\n \"}" @@ -588,7 +614,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [], @@ -610,17 +635,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),populate_cost_table)", @@ -662,7 +676,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025569709, + "timestampMillis": 1696057250831, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -692,8 +706,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" + "name": "sqlite_operator" } } }, @@ -904,7 +917,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:12:52.839611+00:00", + "start_date": "2023-09-30 07:00:53.989264+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -919,7 +932,7 @@ "name": "sqlite_operator_transform_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696025572839, + "time": 1696057253989, "actor": "urn:li:corpuser:datahub" } } @@ -992,7 +1005,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025572839, + "timestampMillis": 1696057253989, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1002,6 +1015,36 @@ } } }, +{ + "entityType": "dataJob", + "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", + "changeType": "UPSERT", + "aspectName": "dataJobInfo", + "aspect": { + "json": { + "customProperties": { + "depends_on_past": "False", + "email": "None", + "label": "'transform_cost_table'", + "execution_timeout": "None", + "sla": "None", + "sql": "'\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n '", + "task_id": "'transform_cost_table'", + "trigger_rule": "", + "wait_for_downstream": "False", + "downstream_task_ids": "['cleanup_costs', 'cleanup_processed_costs']", + "inlets": "[]", + "outlets": "[]", + "openlineage_job_facet_sql": "{\"_producer\": \"https://github.com/OpenLineage/OpenLineage/tree/1.2.0/integration/airflow\", \"_schemaURL\": \"https://raw.githubusercontent.com/OpenLineage/OpenLineage/main/spec/OpenLineage.json#/definitions/SqlJobFacet\", \"query\": \"\\n CREATE TABLE IF NOT EXISTS processed_costs AS\\n SELECT\\n id,\\n month,\\n total_cost,\\n area,\\n total_cost / area as cost_per_area\\n FROM costs\\n \"}" + }, + "externalUrl": "http://airflow.example.com/taskinstance/list/?flt1_dag_id_equals=sqlite_operator&_flt_3_task_id=transform_cost_table", + "name": "transform_cost_table", + "type": { + "string": "COMMAND" + } + } + } +}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),transform_cost_table)", @@ -1010,11 +1053,9 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" ], "inputDatajobs": [ @@ -1148,28 +1189,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", @@ -1222,7 +1241,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025574485, + "timestampMillis": 1696057255628, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1252,8 +1271,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" + "name": "sqlite_operator" } } }, @@ -1396,7 +1414,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:12:59.130809+00:00", + "start_date": "2023-09-30 07:01:00.421177+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -1411,7 +1429,7 @@ "name": "sqlite_operator_cleanup_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696025579130, + "time": 1696057260421, "actor": "urn:li:corpuser:datahub" } } @@ -1460,7 +1478,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025579130, + "timestampMillis": 1696057260421, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1487,7 +1505,7 @@ "task_id": "'cleanup_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1510,7 +1528,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ], "outputDatasets": [], @@ -1532,17 +1549,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_costs)", @@ -1584,7 +1590,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025580994, + "timestampMillis": 1696057262258, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1614,8 +1620,7 @@ "timezone": "Timezone('UTC')" }, "externalUrl": "http://airflow.example.com/tree?dag_id=sqlite_operator", - "name": "sqlite_operator", - "description": "None\n\n" + "name": "sqlite_operator" } } }, @@ -1758,7 +1763,7 @@ "customProperties": { "run_id": "manual_run_test", "duration": "None", - "start_date": "2023-09-29 22:13:04.443580+00:00", + "start_date": "2023-09-30 07:01:05.540192+00:00", "end_date": "None", "execution_date": "2023-09-27 21:34:38+00:00", "try_number": "0", @@ -1773,7 +1778,7 @@ "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1696025584443, + "time": 1696057265540, "actor": "urn:li:corpuser:datahub" } } @@ -1822,7 +1827,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025584443, + "timestampMillis": 1696057265540, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1849,7 +1854,7 @@ "task_id": "'cleanup_processed_costs'", "trigger_rule": "", "wait_for_downstream": "False", - "downstream_task_ids": "set()", + "downstream_task_ids": "[]", "inlets": "[]", "outlets": "[]", "datahub_sql_parser_error": "Can only generate column-level lineage for select-like inner statements, not (outer statement type: )", @@ -1872,7 +1877,6 @@ "aspect": { "json": { "inputDatasets": [ - "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)" ], "outputDatasets": [], @@ -1894,17 +1898,6 @@ } } }, -{ - "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.processed_costs,PROD)", - "changeType": "UPSERT", - "aspectName": "status", - "aspect": { - "json": { - "removed": false - } - } -}, { "entityType": "dataJob", "entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(airflow,sqlite_operator,prod),cleanup_processed_costs)", @@ -1946,7 +1939,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1696025586670, + "timestampMillis": 1696057267631, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" From 5e2405bea265cb039ddaab1bbea338599f71115e Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 3 Oct 2023 17:17:29 -0700 Subject: [PATCH 58/60] review --- docs/how/updating-datahub.md | 2 ++ docs/lineage/airflow.md | 8 ++++---- .../src/datahub_airflow_plugin/datahub_plugin.py | 9 +++++++++ .../tests/integration/test_plugin.py | 3 +++ metadata-ingestion/setup.py | 16 ++++++---------- .../source/sql/sqlalchemy_uri_mapper.py | 4 +--- 6 files changed, 25 insertions(+), 17 deletions(-) diff --git a/docs/how/updating-datahub.md b/docs/how/updating-datahub.md index bd9fa92a7afef..4df8d435cf1c4 100644 --- a/docs/how/updating-datahub.md +++ b/docs/how/updating-datahub.md @@ -5,8 +5,10 @@ This file documents any backwards-incompatible changes in DataHub and assists pe ## Next ### Breaking Changes + - #8810 - Removed support for SQLAlchemy 1.3.x. Only SQLAlchemy 1.4.x is supported now. - #8853 - The Airflow plugin no longer supports Airflow 2.0.x or Python 3.7. See the docs for more details. +- #8853 - Introduced the Airflow plugin v2. If you're using Airflow 2.3+, the v2 plugin will be enabled by default, and so you'll need to switch your requirements to include `pip install 'acryl-datahub-airflow-plugin[plugin-v2]'`. To continue using the v1 plugin, set the `DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN` environment variable to `true`. ### Potential Downtime diff --git a/docs/lineage/airflow.md b/docs/lineage/airflow.md index f1d3763e11323..19ed1598d4c5a 100644 --- a/docs/lineage/airflow.md +++ b/docs/lineage/airflow.md @@ -15,10 +15,10 @@ The DataHub Airflow plugin supports: There's two actively supported implementations of the plugin, with different Airflow version support. -| Approach | Airflow Version | Notes | -| --------- | --------------- | -------------------------------------------------------------------------------------- | -| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | -| Plugin v1 | 2.1+ | Supported. No automatic lineage extraction; may not extract lineage if the task fails. | +| Approach | Airflow Version | Notes | +| --------- | --------------- | --------------------------------------------------------------------------- | +| Plugin v2 | 2.3+ | Recommended. Requires Python 3.8+ | +| Plugin v1 | 2.1+ | No automatic lineage extraction; may not extract lineage if the task fails. | If you're using Airflow older than 2.1, it's possible to use the v1 plugin with older versions of `acryl-datahub-airflow-plugin`. See the [compatibility section](#compatibility) for more details. diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py index a9d319edf41d1..c96fab31647f5 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin.py @@ -18,6 +18,15 @@ "DATAHUB_AIRFLOW_PLUGIN_USE_V1_PLUGIN", "false" ).lower() in ("true", "1") +if _USE_AIRFLOW_LISTENER_INTERFACE: + try: + from openlineage.airflow.utils import try_import_from_string # noqa: F401 + except ImportError: + # If v2 plugin dependencies are not installed, we fall back to v1. + logger.debug("Falling back to v1 plugin due to missing dependencies.") + _USE_AIRFLOW_LISTENER_INTERFACE = False + + with contextlib.suppress(Exception): if not os.getenv("DATAHUB_AIRFLOW_PLUGIN_SKIP_FORK_PATCH", "false").lower() in ( "true", diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index 729fa015ea81e..ca15cb3cf0bcb 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -208,6 +208,9 @@ def _run_airflow( env=environment, ) + # Load the admin user's password. This is generated by the + # `airflow standalone` command, and is different from the + # airflow user that we create when running locally. airflow_username = "admin" airflow_password = (airflow_home / "standalone_admin_password.txt").read_text() diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 52db3ffd616a3..95554cb2695ac 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -1,4 +1,3 @@ -import os import sys from typing import Dict, Set @@ -9,14 +8,6 @@ exec(fp.read(), package_metadata) -def get_long_description(): - root = os.path.dirname(__file__) - with open(os.path.join(root, "README.md")) as f: - description = f.read() - - return description - - base_requirements = { # Typing extension should be >=3.10.0.2 ideally but we can't restrict due to a Airflow 2.1 dependency conflict. "typing_extensions>=3.7.4.3", @@ -668,7 +659,12 @@ def get_long_description(): }, license="Apache License 2.0", description="A CLI to work with DataHub metadata", - long_description=get_long_description(), + long_description="""\ +The `acryl-datahub` package contains a CLI and SDK for interacting with DataHub, +as well as an integration framework for pulling/pushing metadata from external systems. + +See the [DataHub docs](https://datahubproject.io/docs/metadata-ingestion). +""", long_description_content_type="text/markdown", classifiers=[ "Development Status :: 5 - Production/Stable", diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py index 70384446cce0e..b6a463837228d 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/sqlalchemy_uri_mapper.py @@ -5,9 +5,7 @@ def _platform_alchemy_uri_tester_gen( platform: str, opt_starts_with: Optional[str] = None ) -> Tuple[str, Callable[[str], bool]]: - return platform, lambda x: x.startswith( - platform if not opt_starts_with else opt_starts_with - ) + return platform, lambda x: x.startswith(opt_starts_with or platform) PLATFORM_TO_SQLALCHEMY_URI_TESTER_MAP: Dict[str, Callable[[str], bool]] = OrderedDict( From 3b49da83b0c97a6f02d06502f586f654babba092 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 3 Oct 2023 17:28:53 -0700 Subject: [PATCH 59/60] finishing touches --- .../datahub_airflow_plugin/datahub_listener.py | 15 +++++++++++++++ .../datahub_airflow_plugin/datahub_plugin_v22.py | 14 ++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py index 3743932a7503b..a3f5cb489e29f 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_listener.py @@ -4,6 +4,7 @@ import threading from typing import TYPE_CHECKING, Callable, Dict, List, Optional, TypeVar, cast +import airflow import datahub.emitter.mce_builder as builder from datahub.api.entities.datajob import DataJob from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult @@ -14,6 +15,7 @@ FineGrainedLineageDownstreamTypeClass, FineGrainedLineageUpstreamTypeClass, ) +from datahub.telemetry import telemetry from datahub.utilities.sqlglot_lineage import SqlParsingResult from datahub.utilities.urns.dataset_urn import DatasetUrn from openlineage.airflow.listener import TaskHolder @@ -73,6 +75,19 @@ def get_airflow_plugin_listener() -> Optional["DataHubListener"]: OpenLineagePlugin.listeners = [] + telemetry.telemetry_instance.ping( + "airflow-plugin-init", + { + "airflow-version": airflow.__version__, + "datahub-airflow-plugin": "v2", + "datahub-airflow-plugin-dag-events": HAS_AIRFLOW_DAG_LISTENER_API, + "capture_executions": plugin_config.capture_executions, + "capture_tags": plugin_config.capture_tags_info, + "capture_ownership": plugin_config.capture_ownership_info, + "enable_extractors": plugin_config.enable_extractors, + "disable_openlineage_plugin": plugin_config.disable_openlineage_plugin, + }, + ) return _airflow_listener diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py index e8ff02d48a651..046fbb5efaa03 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/datahub_plugin_v22.py @@ -3,11 +3,13 @@ import traceback from typing import Any, Callable, Iterable, List, Optional, Union +import airflow from airflow.lineage import PIPELINE_OUTLETS from airflow.models.baseoperator import BaseOperator from airflow.utils.module_loading import import_string from cattr import structure from datahub.api.entities.dataprocess.dataprocess_instance import InstanceRunResult +from datahub.telemetry import telemetry from datahub_airflow_plugin._airflow_shims import ( MappedOperator, @@ -318,5 +320,17 @@ def _patch_datahub_policy(): _patch_policy(settings) + plugin_config = get_lineage_config() + telemetry.telemetry_instance.ping( + "airflow-plugin-init", + { + "airflow-version": airflow.__version__, + "datahub-airflow-plugin": "v1", + "capture_executions": plugin_config.capture_executions, + "capture_tags": plugin_config.capture_tags_info, + "capture_ownership": plugin_config.capture_ownership_info, + }, + ) + _patch_datahub_policy() From d61e5e9ce14a625bad3430118cc5562ec24a7be5 Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Tue, 3 Oct 2023 18:48:17 -0700 Subject: [PATCH 60/60] update example + deps --- .../airflow-plugin/setup.py | 3 +++ .../example_dags/lineage_emission_dag.py | 22 ++++--------------- .../tests/integration/test_plugin.py | 2 +- 3 files changed, 8 insertions(+), 19 deletions(-) diff --git a/metadata-ingestion-modules/airflow-plugin/setup.py b/metadata-ingestion-modules/airflow-plugin/setup.py index 7bfc105f303fd..a5af881022d8c 100644 --- a/metadata-ingestion-modules/airflow-plugin/setup.py +++ b/metadata-ingestion-modules/airflow-plugin/setup.py @@ -107,6 +107,9 @@ def get_long_description(): # of splitting this into integration-test-old and integration-test-new, # adding a bound to SQLAlchemy was the simplest solution. "sqlalchemy<1.4.42", + # To avoid https://github.com/snowflakedb/snowflake-connector-python/issues/1188, + # we need https://github.com/snowflakedb/snowflake-connector-python/pull/1193 + "snowflake-connector-python>=2.7.10", "virtualenv", # needed by PythonVirtualenvOperator "apache-airflow-providers-sqlite", } diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py index f40295c6bb883..0d7cdb6b6e90a 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/example_dags/lineage_emission_dag.py @@ -2,12 +2,11 @@ This example demonstrates how to emit lineage to DataHub within an Airflow DAG. """ - from datetime import timedelta import datahub.emitter.mce_builder as builder from airflow import DAG -from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator +from airflow.operators.bash import BashOperator from airflow.utils.dates import days_ago from datahub_airflow_plugin.operators.datahub import DatahubEmitterOperator @@ -33,23 +32,10 @@ catchup=False, default_view="tree", ) as dag: - # This example shows a SnowflakeOperator followed by a lineage emission. However, the - # same DatahubEmitterOperator can be used to emit lineage in any context. - - sql = """CREATE OR REPLACE TABLE `mydb.schema.tableC` AS - WITH some_table AS ( - SELECT * FROM `mydb.schema.tableA` - ), - some_other_table AS ( - SELECT id, some_column FROM `mydb.schema.tableB` - ) - SELECT * FROM some_table - LEFT JOIN some_other_table ON some_table.unique_id=some_other_table.id""" - transformation_task = SnowflakeOperator( - task_id="snowflake_transformation", + transformation_task = BashOperator( + task_id="transformation_task", dag=dag, - snowflake_conn_id="snowflake_default", - sql=sql, + bash_command="echo 'This is where you might run your data tooling.'", ) emit_lineage_task = DatahubEmitterOperator( diff --git a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py index ca15cb3cf0bcb..a2b7fd151a1e4 100644 --- a/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py +++ b/metadata-ingestion-modules/airflow-plugin/tests/integration/test_plugin.py @@ -79,7 +79,7 @@ class NotReadyError(Exception): @tenacity.retry( reraise=True, wait=tenacity.wait_fixed(1), - stop=tenacity.stop_after_delay(60), + stop=tenacity.stop_after_delay(90), retry=tenacity.retry_if_exception_type(NotReadyError), ) def _wait_for_dag_finish(