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 32bbe88481636..197ae5298aa83 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 @@ -7,7 +7,7 @@ from datahub.ingestion.source.sql.sqlalchemy_uri_mapper import ( get_platform_from_sqlalchemy_uri, ) -from datahub.utilities.sqlglot_lineage import ( +from datahub.sql_parsing.sqlglot_lineage import ( SqlParsingResult, create_lineage_sql_parsed_result, ) 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 e1d53be7bae6b..706920d9df62f 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 @@ -106,9 +106,9 @@ def _get_dependencies( job_id=task.external_task_id, data_flow_urn=str( DataFlowUrn.create_from_ids( - orchestrator=flow_urn.get_orchestrator_name(), + orchestrator=flow_urn.orchestrator, flow_id=task.external_dag_id, - env=flow_urn.get_env(), + env=flow_urn.cluster, ) ), ) 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 a7f588a166dde..23a8dd569bf47 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 @@ -16,8 +16,8 @@ FineGrainedLineageDownstreamTypeClass, FineGrainedLineageUpstreamTypeClass, ) +from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult from datahub.telemetry import telemetry -from datahub.utilities.sqlglot_lineage import SqlParsingResult from openlineage.airflow.listener import TaskHolder from openlineage.airflow.utils import redact_with_exclusions from openlineage.client.serde import Serde diff --git a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/entities.py b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/entities.py index 5a4bcb0097a8c..30b35ac6d6198 100644 --- a/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/entities.py +++ b/metadata-ingestion-modules/airflow-plugin/src/datahub_airflow_plugin/entities.py @@ -55,7 +55,7 @@ def entities_to_dataset_urn_list(iolets: List[str]) -> List[DatasetUrn]: dataset_urn_list: List[DatasetUrn] = [] for let in iolets: if guess_entity_type(let) == "dataset": - dataset_urn_list.append(DatasetUrn.create_from_string(let)) + dataset_urn_list.append(DatasetUrn.from_string(let)) return dataset_urn_list @@ -63,5 +63,5 @@ def entities_to_datajob_urn_list(inlets: List[str]) -> List[DataJobUrn]: datajob_urn_list: List[DataJobUrn] = [] for let in inlets: if guess_entity_type(let) == "dataJob": - datajob_urn_list.append(DataJobUrn.create_from_string(let)) + datajob_urn_list.append(DataJobUrn.from_string(let)) return datajob_urn_list 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 3965ee4a10ad0..313abad9c5546 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 @@ -212,7 +212,7 @@ "name": "sqlite_operator_create_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223533895, + "time": 1707253651059, "actor": "urn:li:corpuser:datahub" } } @@ -261,7 +261,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223533895, + "timestampMillis": 1707253651059, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -442,7 +442,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223534302, + "timestampMillis": 1707253651425, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -492,10 +492,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "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)" ], @@ -576,7 +576,7 @@ "name": "sqlite_operator_populate_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223539348, + "time": 1707253655698, "actor": "urn:li:corpuser:datahub" } } @@ -598,10 +598,10 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", + "aspectName": "dataProcessInstanceOutput", "aspect": { "json": { - "inputs": [ + "outputs": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ] } @@ -625,7 +625,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223539348, + "timestampMillis": 1707253655698, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -672,10 +672,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "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)" ], @@ -735,7 +735,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223540058, + "timestampMillis": 1707253656320, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -939,7 +939,7 @@ "name": "sqlite_operator_transform_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223548187, + "time": 1707253660584, "actor": "urn:li:corpuser:datahub" } } @@ -1012,7 +1012,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223548187, + "timestampMillis": 1707253660584, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1248,7 +1248,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223549416, + "timestampMillis": 1707253661682, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1384,7 +1384,7 @@ "name": "sqlite_operator_cleanup_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223557795, + "time": 1707253669241, "actor": "urn:li:corpuser:datahub" } } @@ -1433,7 +1433,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223557795, + "timestampMillis": 1707253669241, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1545,7 +1545,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223559079, + "timestampMillis": 1707253670409, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1681,7 +1681,7 @@ "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223564459, + "time": 1707253675107, "actor": "urn:li:corpuser:datahub" } } @@ -1730,7 +1730,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223564459, + "timestampMillis": 1707253675107, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1842,7 +1842,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223566107, + "timestampMillis": 1707253676482, "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 a9f9fbac56fff..60beff71c46c6 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 @@ -212,7 +212,7 @@ "name": "sqlite_operator_create_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223251992, + "time": 1707253281415, "actor": "urn:li:corpuser:datahub" } } @@ -261,7 +261,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223251992, + "timestampMillis": 1707253281415, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -442,7 +442,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223253042, + "timestampMillis": 1707253282244, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -549,10 +549,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "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)" ], @@ -633,7 +633,7 @@ "name": "sqlite_operator_populate_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223258947, + "time": 1707253286225, "actor": "urn:li:corpuser:datahub" } } @@ -655,10 +655,10 @@ "entityType": "dataProcessInstance", "entityUrn": "urn:li:dataProcessInstance:04e1badac1eacd1c41123d07f579fa92", "changeType": "UPSERT", - "aspectName": "dataProcessInstanceInput", + "aspectName": "dataProcessInstanceOutput", "aspect": { "json": { - "inputs": [ + "outputs": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,public.costs,PROD)" ] } @@ -682,7 +682,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223258947, + "timestampMillis": 1707253286225, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -729,10 +729,10 @@ "aspectName": "dataJobInputOutput", "aspect": { "json": { - "inputDatasets": [ + "inputDatasets": [], + "outputDatasets": [ "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)" ], @@ -792,7 +792,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223260414, + "timestampMillis": 1707253287414, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1053,7 +1053,7 @@ "name": "sqlite_operator_transform_cost_table_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223266595, + "time": 1707253293513, "actor": "urn:li:corpuser:datahub" } } @@ -1126,7 +1126,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223266595, + "timestampMillis": 1707253293513, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1362,7 +1362,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223268728, + "timestampMillis": 1707253295443, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1555,7 +1555,7 @@ "name": "sqlite_operator_cleanup_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223275045, + "time": 1707253301697, "actor": "urn:li:corpuser:datahub" } } @@ -1604,7 +1604,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223275045, + "timestampMillis": 1707253301697, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1716,7 +1716,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223277378, + "timestampMillis": 1707253303779, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -1909,7 +1909,7 @@ "name": "sqlite_operator_cleanup_processed_costs_manual_run_test", "type": "BATCH_AD_HOC", "created": { - "time": 1701223282010, + "time": 1707253308368, "actor": "urn:li:corpuser:datahub" } } @@ -1958,7 +1958,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223282010, + "timestampMillis": 1707253308368, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" @@ -2070,7 +2070,7 @@ "aspectName": "dataProcessInstanceRunEvent", "aspect": { "json": { - "timestampMillis": 1701223284766, + "timestampMillis": 1707253310722, "partitionSpec": { "type": "FULL_TABLE", "partition": "FULL_TABLE_SNAPSHOT" diff --git a/metadata-ingestion/setup.cfg b/metadata-ingestion/setup.cfg index b3fc53ccfaf58..25ece8ac11ef0 100644 --- a/metadata-ingestion/setup.cfg +++ b/metadata-ingestion/setup.cfg @@ -3,6 +3,9 @@ max-complexity = 20 ignore = # Ignore: line length issues, since black's formatter will take care of them. E501, + # Ignore compound statements, since they're used for ellipsis by black + # See https://github.com/psf/black/issues/3887 + E704, # Ignore: 1 blank line required before class docstring. D203, # See https://stackoverflow.com/a/57074416. @@ -68,6 +71,8 @@ disallow_untyped_defs = yes disallow_untyped_defs = yes [mypy-datahub.ingestion.run.*] disallow_untyped_defs = yes +[mypy-datahub.sql_parsing.*] +disallow_untyped_defs = yes [mypy-datahub.utilities.*] disallow_untyped_defs = yes 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 6a2f733dcf8f7..e1f99169ffb93 100644 --- a/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py +++ b/metadata-ingestion/src/datahub/api/entities/dataprocess/dataprocess_instance.py @@ -74,7 +74,7 @@ class DataProcessInstance: ) def __post_init__(self): - self.urn = DataProcessInstanceUrn.create_from_id( + self.urn = DataProcessInstanceUrn( id=DataProcessInstanceKey( cluster=self.cluster, orchestrator=self.orchestrator, diff --git a/metadata-ingestion/src/datahub/emitter/mce_builder.py b/metadata-ingestion/src/datahub/emitter/mce_builder.py index fe9ecee8f80d0..9098a1373eae7 100644 --- a/metadata-ingestion/src/datahub/emitter/mce_builder.py +++ b/metadata-ingestion/src/datahub/emitter/mce_builder.py @@ -6,6 +6,7 @@ import os import re import time +from datetime import datetime from enum import Enum from typing import ( TYPE_CHECKING, @@ -18,6 +19,7 @@ TypeVar, Union, get_type_hints, + overload, ) import typing_inspect @@ -81,6 +83,22 @@ def get_sys_time() -> int: return int(time.time() * 1000) +@overload +def make_ts_millis(ts: None) -> None: + ... + + +@overload +def make_ts_millis(ts: datetime) -> int: + ... + + +def make_ts_millis(ts: Optional[datetime]) -> Optional[int]: + if ts is None: + return None + return int(ts.timestamp() * 1000) + + def make_data_platform_urn(platform: str) -> str: if platform.startswith("urn:li:dataPlatform:"): return platform diff --git a/metadata-ingestion/src/datahub/emitter/sql_parsing_builder.py b/metadata-ingestion/src/datahub/emitter/sql_parsing_builder.py index a8fe4f0df83cf..a57886a0ba699 100644 --- a/metadata-ingestion/src/datahub/emitter/sql_parsing_builder.py +++ b/metadata-ingestion/src/datahub/emitter/sql_parsing_builder.py @@ -20,8 +20,8 @@ UpstreamClass, UpstreamLineageClass, ) +from datahub.sql_parsing.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult from datahub.utilities.file_backed_collections import FileBackedDict -from datahub.utilities.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult logger = logging.getLogger(__name__) @@ -47,11 +47,14 @@ class LineageEdge: def gen_upstream_aspect(self) -> UpstreamClass: return UpstreamClass( - auditStamp=AuditStampClass( - time=int(self.audit_stamp.timestamp() * 1000), actor=self.actor or "" - ) - if self.audit_stamp - else None, + auditStamp=( + AuditStampClass( + time=int(self.audit_stamp.timestamp() * 1000), + actor=self.actor or "", + ) + if self.audit_stamp + else None + ), dataset=self.upstream_urn, type=self.type, ) @@ -133,9 +136,9 @@ def process_sql_parsing_result( self._lineage_map[downstream_urn] = _merge_lineage_data( downstream_urn=downstream_urn, upstream_urns=result.in_tables, - column_lineage=result.column_lineage - if include_column_lineage - else None, + column_lineage=( + result.column_lineage if include_column_lineage else None + ), upstream_edges=self._lineage_map.get(downstream_urn, {}), query_timestamp=query_timestamp, is_view_ddl=is_view_ddl, @@ -143,7 +146,7 @@ def process_sql_parsing_result( ) if self.generate_usage_statistics and query_timestamp is not None: - upstream_fields = _compute_upstream_fields(result) + upstream_fields = compute_upstream_fields(result) for upstream_urn in upstreams_to_ingest: self._usage_aggregator.aggregate_event( resource=upstream_urn, @@ -237,9 +240,11 @@ def _merge_lineage_data( upstream_urn=upstream_urn, audit_stamp=query_timestamp, actor=user, - type=DatasetLineageTypeClass.VIEW - if is_view_ddl - else DatasetLineageTypeClass.TRANSFORMED, + type=( + DatasetLineageTypeClass.VIEW + if is_view_ddl + else DatasetLineageTypeClass.TRANSFORMED + ), ), ) if query_timestamp and ( # Use the most recent query @@ -261,7 +266,7 @@ def _merge_lineage_data( return upstream_edges -def _compute_upstream_fields( +def compute_upstream_fields( result: SqlParsingResult, ) -> Dict[DatasetUrn, Set[DatasetUrn]]: upstream_fields: Dict[DatasetUrn, Set[DatasetUrn]] = defaultdict(set) diff --git a/metadata-ingestion/src/datahub/ingestion/graph/client.py b/metadata-ingestion/src/datahub/ingestion/graph/client.py index d64f756dddc13..35ddc727dadbe 100644 --- a/metadata-ingestion/src/datahub/ingestion/graph/client.py +++ b/metadata-ingestion/src/datahub/ingestion/graph/client.py @@ -50,11 +50,11 @@ from datahub.ingestion.source.state.entity_removal_state import ( GenericCheckpointState, ) - from datahub.utilities.sqlglot_lineage import ( + from datahub.sql_parsing.schema_resolver import ( GraphQLSchemaMetadata, SchemaResolver, - SqlParsingResult, ) + from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult logger = logging.getLogger(__name__) @@ -1003,7 +1003,7 @@ def _make_schema_resolver( env: str, include_graph: bool = True, ) -> "SchemaResolver": - from datahub.utilities.sqlglot_lineage import SchemaResolver + from datahub.sql_parsing.schema_resolver import SchemaResolver return SchemaResolver( platform=platform, @@ -1055,7 +1055,7 @@ def parse_sql_lineage( default_db: Optional[str] = None, default_schema: Optional[str] = None, ) -> "SqlParsingResult": - from datahub.utilities.sqlglot_lineage import sqlglot_lineage + from datahub.sql_parsing.sqlglot_lineage import sqlglot_lineage # Cache the schema resolver to make bulk parsing faster. schema_resolver = self._make_schema_resolver( diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py index b8bc07b9a3559..c36b150d3220f 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py @@ -110,6 +110,7 @@ GlobalTagsClass, TagAssociationClass, ) +from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.utilities.file_backed_collections import FileBackedDict from datahub.utilities.hive_schema_to_avro import ( HiveColumnToAvroConverter, @@ -118,7 +119,6 @@ from datahub.utilities.mapping import Constants from datahub.utilities.perf_timer import PerfTimer from datahub.utilities.registries.domain_registry import DomainRegistry -from datahub.utilities.sqlglot_lineage import SchemaResolver logger: logging.Logger = logging.getLogger(__name__) @@ -484,9 +484,11 @@ def get_dataplatform_instance_aspect( ) -> MetadataWorkUnit: aspect = DataPlatformInstanceClass( platform=make_data_platform_urn(self.platform), - instance=make_dataplatform_instance_urn(self.platform, project_id) - if self.config.include_data_platform_instance - else None, + instance=( + make_dataplatform_instance_urn(self.platform, project_id) + if self.config.include_data_platform_instance + else None + ), ) return MetadataChangeProposalWrapper( entityUrn=dataset_urn, aspect=aspect @@ -539,11 +541,13 @@ def gen_dataset_containers( domain_config=self.config.domain, schema_container_key=schema_container_key, database_container_key=database_container_key, - external_url=BQ_EXTERNAL_DATASET_URL_TEMPLATE.format( - project=project_id, dataset=dataset - ) - if self.config.include_external_url - else None, + external_url=( + BQ_EXTERNAL_DATASET_URL_TEMPLATE.format( + project=project_id, dataset=dataset + ) + if self.config.include_external_url + else None + ), tags=tags_joined, ) @@ -1071,19 +1075,27 @@ def gen_dataset_workunits( name=datahub_dataset_name.get_table_display_name(), description=table.comment, qualifiedName=str(datahub_dataset_name), - created=TimeStamp(time=int(table.created.timestamp() * 1000)) - if table.created is not None - else None, - lastModified=TimeStamp(time=int(table.last_altered.timestamp() * 1000)) - if table.last_altered is not None - else TimeStamp(time=int(table.created.timestamp() * 1000)) - if table.created is not None - else None, - externalUrl=BQ_EXTERNAL_TABLE_URL_TEMPLATE.format( - project=project_id, dataset=dataset_name, table=table.name - ) - if self.config.include_external_url - else None, + created=( + TimeStamp(time=int(table.created.timestamp() * 1000)) + if table.created is not None + else None + ), + lastModified=( + TimeStamp(time=int(table.last_altered.timestamp() * 1000)) + if table.last_altered is not None + else ( + TimeStamp(time=int(table.created.timestamp() * 1000)) + if table.created is not None + else None + ) + ), + externalUrl=( + BQ_EXTERNAL_TABLE_URL_TEMPLATE.format( + project=project_id, dataset=dataset_name, table=table.name + ) + if self.config.include_external_url + else None + ), ) if custom_properties: dataset_properties.customProperties.update(custom_properties) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py index 7db36867b4e69..5a3e6088e651f 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py @@ -59,14 +59,11 @@ UpstreamLineageClass, ) from datahub.specific.dataset import DatasetPatchBuilder +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult, sqlglot_lineage from datahub.utilities import memory_footprint from datahub.utilities.file_backed_collections import FileBackedDict from datahub.utilities.perf_timer import PerfTimer -from datahub.utilities.sqlglot_lineage import ( - SchemaResolver, - SqlParsingResult, - sqlglot_lineage, -) from datahub.utilities.urns.dataset_urn import DatasetUrn logger: logging.Logger = logging.getLogger(__name__) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py index ccc64184f3346..b5c077b7cda39 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py @@ -59,9 +59,10 @@ USAGE_EXTRACTION_USAGE_AGGREGATION, ) from datahub.metadata.schema_classes import OperationClass, OperationTypeClass +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import sqlglot_lineage from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedDict from datahub.utilities.perf_timer import PerfTimer -from datahub.utilities.sqlglot_lineage import SchemaResolver, sqlglot_lineage logger: logging.Logger = logging.getLogger(__name__) diff --git a/metadata-ingestion/src/datahub/ingestion/source/datahub/datahub_api_reader.py b/metadata-ingestion/src/datahub/ingestion/source/datahub/datahub_api_reader.py index 7ee36736723b2..6986aac0a7757 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/datahub/datahub_api_reader.py +++ b/metadata-ingestion/src/datahub/ingestion/source/datahub/datahub_api_reader.py @@ -7,7 +7,7 @@ from datahub.ingestion.graph.filters import RemovedStatusFilter from datahub.ingestion.source.datahub.config import DataHubSourceConfig from datahub.ingestion.source.datahub.report import DataHubSourceReport -from datahub.metadata._schema_classes import _Aspect +from datahub.metadata.schema_classes import _Aspect logger = logging.getLogger(__name__) diff --git a/metadata-ingestion/src/datahub/ingestion/source/dbt/dbt_common.py b/metadata-ingestion/src/datahub/ingestion/source/dbt/dbt_common.py index 985c9118f3422..a5afe42d65371 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/dbt/dbt_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/dbt/dbt_common.py @@ -107,15 +107,16 @@ UpstreamLineageClass, ViewPropertiesClass, ) -from datahub.utilities.mapping import Constants, OperationProcessor -from datahub.utilities.sqlglot_lineage import ( +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import ( SchemaInfo, - SchemaResolver, SqlParsingDebugInfo, SqlParsingResult, - detach_ctes, + infer_output_schema, sqlglot_lineage, ) +from datahub.sql_parsing.sqlglot_utils import detach_ctes +from datahub.utilities.mapping import Constants, OperationProcessor from datahub.utilities.time import datetime_to_ts_millis from datahub.utilities.topological_sort import topological_sort @@ -951,9 +952,11 @@ def _infer_schemas_and_update_cll(self, all_nodes_map: Dict[str, DBTNode]) -> No ): schema_fields = [ SchemaField( - fieldPath=column.name.lower() - if self.config.convert_column_urns_to_lowercase - else column.name, + fieldPath=( + column.name.lower() + if self.config.convert_column_urns_to_lowercase + else column.name + ), type=column.datahub_data_type or SchemaFieldDataType(type=NullTypeClass()), nativeDataType=column.data_type, @@ -1029,17 +1032,8 @@ def _infer_schemas_and_update_cll(self, all_nodes_map: Dict[str, DBTNode]) -> No # If we didn't fetch the schema from the graph, use the inferred schema. inferred_schema_fields = None - if sql_result and sql_result.column_lineage: - inferred_schema_fields = [ - SchemaField( - fieldPath=column_lineage.downstream.column, - type=column_lineage.downstream.column_type - or SchemaFieldDataType(type=NullTypeClass()), - nativeDataType=column_lineage.downstream.native_column_type - or "", - ) - for column_lineage in sql_result.column_lineage - ] + if sql_result: + inferred_schema_fields = infer_output_schema(sql_result) # Conditionally add the inferred schema to the schema resolver. if ( @@ -1541,9 +1535,9 @@ def _translate_dbt_name_to_upstream_urn(dbt_name: str) -> str: downstreams=[ mce_builder.make_schema_field_urn(node_urn, downstream) ], - confidenceScore=node.cll_debug_info.confidence - if node.cll_debug_info - else None, + confidenceScore=( + node.cll_debug_info.confidence if node.cll_debug_info else None + ), ) for downstream, upstreams in itertools.groupby( node.upstream_cll, lambda x: x.downstream_col diff --git a/metadata-ingestion/src/datahub/ingestion/source/metabase.py b/metadata-ingestion/src/datahub/ingestion/source/metabase.py index a65bb76345f4b..07582871cfc8d 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/metabase.py +++ b/metadata-ingestion/src/datahub/ingestion/source/metabase.py @@ -41,8 +41,8 @@ OwnershipClass, OwnershipTypeClass, ) +from datahub.sql_parsing.sqlglot_lineage import create_lineage_sql_parsed_result from datahub.utilities import config_clean -from datahub.utilities.sqlglot_lineage import create_lineage_sql_parsed_result logger = logging.getLogger(__name__) @@ -160,9 +160,11 @@ def setup_session(self) -> None: None, { "username": self.config.username, - "password": self.config.password.get_secret_value() - if self.config.password - else None, + "password": ( + self.config.password.get_secret_value() + if self.config.password + else None + ), }, ) @@ -493,9 +495,11 @@ def construct_card_custom_properties(self, card_details: dict) -> Dict: metrics, dimensions = [], [] for meta_data in result_metadata: display_name = meta_data.get("display_name", "") or "" - metrics.append(display_name) if "aggregation" in meta_data.get( - "field_ref", "" - ) else dimensions.append(display_name) + ( + metrics.append(display_name) + if "aggregation" in meta_data.get("field_ref", "") + else dimensions.append(display_name) + ) filters = (card_details.get("dataset_query", {}).get("query", {})).get( "filter", [] diff --git a/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/native_sql_parser.py b/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/native_sql_parser.py index 56c9a4abe18ad..52a7cb6fed0ee 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/native_sql_parser.py +++ b/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/native_sql_parser.py @@ -3,9 +3,11 @@ import sqlparse -import datahub.utilities.sqlglot_lineage as sqlglot_l from datahub.ingestion.api.common import PipelineContext -from datahub.utilities.sqlglot_lineage import SqlParsingResult +from datahub.sql_parsing.sqlglot_lineage import ( + SqlParsingResult, + create_lineage_sql_parsed_result, +) SPECIAL_CHARACTERS = ["#(lf)", "(lf)"] @@ -67,7 +69,7 @@ def parse_custom_sql( logger.debug(f"Processing native query = {sql_query}") - return sqlglot_l.create_lineage_sql_parsed_result( + return create_lineage_sql_parsed_result( query=sql_query, default_schema=schema, default_db=database, diff --git a/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/resolver.py b/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/resolver.py index 930841f1f0df2..72f9c2167cab9 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/resolver.py +++ b/metadata-ingestion/src/datahub/ingestion/source/powerbi/m_query/resolver.py @@ -27,7 +27,7 @@ IdentifierAccessor, ) from datahub.ingestion.source.powerbi.rest_api_wrapper.data_classes import Table -from datahub.utilities.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult +from datahub.sql_parsing.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult logger = logging.getLogger(__name__) @@ -199,9 +199,11 @@ def parse_custom_sql( return Lineage( upstreams=dataplatform_tables, - column_lineage=parsed_result.column_lineage - if parsed_result.column_lineage is not None - else [], + column_lineage=( + parsed_result.column_lineage + if parsed_result.column_lineage is not None + else [] + ), ) @@ -525,12 +527,12 @@ def resolve_to_data_platform_table_list( # From supported_resolver enum get respective resolver like AmazonRedshift or Snowflake or Oracle or NativeQuery and create instance of it # & also pass additional information that will be need to generate urn - table_qualified_name_creator: AbstractDataPlatformTableCreator = ( - supported_resolver.get_table_full_name_creator()( - ctx=ctx, - config=config, - platform_instance_resolver=platform_instance_resolver, - ) + table_qualified_name_creator: ( + AbstractDataPlatformTableCreator + ) = supported_resolver.get_table_full_name_creator()( + ctx=ctx, + config=config, + platform_instance_resolver=platform_instance_resolver, ) lineage.append(table_qualified_name_creator.create_lineage(f_detail)) diff --git a/metadata-ingestion/src/datahub/ingestion/source/powerbi/powerbi.py b/metadata-ingestion/src/datahub/ingestion/source/powerbi/powerbi.py index cdf7c975c0614..0c21d735324f1 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/powerbi/powerbi.py +++ b/metadata-ingestion/src/datahub/ingestion/source/powerbi/powerbi.py @@ -81,8 +81,8 @@ UpstreamLineageClass, ViewPropertiesClass, ) +from datahub.sql_parsing.sqlglot_lineage import ColumnLineageInfo from datahub.utilities.dedup_list import deduplicate_list -from datahub.utilities.sqlglot_lineage import ColumnLineageInfo # Logger instance logger = logging.getLogger(__name__) diff --git a/metadata-ingestion/src/datahub/ingestion/source/redshift/lineage.py b/metadata-ingestion/src/datahub/ingestion/source/redshift/lineage.py index 898e6db0b14b0..7120c313c3d41 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/redshift/lineage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/redshift/lineage.py @@ -4,7 +4,7 @@ from dataclasses import dataclass, field from datetime import datetime from enum import Enum -from typing import Dict, List, Optional, Set, Tuple, Union, cast +from typing import Dict, List, Optional, Set, Tuple, Union from urllib.parse import urlparse import humanfriendly @@ -12,7 +12,7 @@ import sqlglot import datahub.emitter.mce_builder as builder -import datahub.utilities.sqlglot_lineage as sqlglot_l +import datahub.sql_parsing.sqlglot_lineage as sqlglot_l from datahub.emitter import mce_builder from datahub.emitter.mce_builder import make_dataset_urn_with_platform_instance from datahub.ingestion.api.common import PipelineContext @@ -31,7 +31,6 @@ from datahub.ingestion.source.state.redundant_run_skip_handler import ( RedundantLineageRunSkipHandler, ) -from datahub.metadata._schema_classes import SchemaFieldDataTypeClass from datahub.metadata.com.linkedin.pegasus2avro.dataset import ( FineGrainedLineage, FineGrainedLineageDownstreamType, @@ -48,9 +47,10 @@ UpstreamClass, UpstreamLineageClass, ) +from datahub.metadata.urns import DatasetUrn +from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.utilities import memory_footprint from datahub.utilities.dedup_list import deduplicate_list -from datahub.utilities.urns import dataset_urn logger: logging.Logger = logging.getLogger(__name__) @@ -137,9 +137,7 @@ def parse_alter_table_rename(default_schema: str, query: str) -> Tuple[str, str, def split_qualified_table_name(urn: str) -> Tuple[str, str, str]: - qualified_table_name = dataset_urn.DatasetUrn.create_from_string( - urn - ).get_entity_id()[1] + qualified_table_name = DatasetUrn.from_string(urn).name # -3 because platform instance is optional and that can cause the split to have more than 3 elements db, schema, table = qualified_table_name.split(".")[-3:] @@ -174,12 +172,10 @@ def _init_temp_table_schema( if self.context.graph is None: # to silent lint return - schema_resolver: sqlglot_l.SchemaResolver = ( - self.context.graph._make_schema_resolver( - platform=LineageDatasetPlatform.REDSHIFT.value, - platform_instance=self.config.platform_instance, - env=self.config.env, - ) + schema_resolver: SchemaResolver = self.context.graph._make_schema_resolver( + platform=LineageDatasetPlatform.REDSHIFT.value, + platform_instance=self.config.platform_instance, + env=self.config.env, ) dataset_vs_columns: Dict[str, List[SchemaField]] = {} @@ -201,7 +197,7 @@ def _init_temp_table_schema( if ( result is None or result.column_lineage is None - or result.query_type != sqlglot_l.QueryType.CREATE + or not result.query_type.is_create() or not result.out_tables ): logger.debug(f"Unsupported temp table query found: {table.query_text}") @@ -216,26 +212,18 @@ def _init_temp_table_schema( for table in self.temp_tables.values(): if ( table.parsed_result is None + or table.urn is None or table.parsed_result.column_lineage is None ): continue - for column_lineage in table.parsed_result.column_lineage: - if column_lineage.downstream.table not in dataset_vs_columns: - dataset_vs_columns[cast(str, column_lineage.downstream.table)] = [] - # Initialise the temp table urn, we later need this to merge CLL - - dataset_vs_columns[cast(str, column_lineage.downstream.table)].append( - SchemaField( - fieldPath=column_lineage.downstream.column, - type=cast( - SchemaFieldDataTypeClass, - column_lineage.downstream.column_type, - ), - nativeDataType=cast( - str, column_lineage.downstream.native_column_type - ), - ) - ) + + # Initialise the temp table urn, we later need this to merge CLL + downstream_urn = table.urn + if downstream_urn not in dataset_vs_columns: + dataset_vs_columns[downstream_urn] = [] + dataset_vs_columns[downstream_urn].extend( + sqlglot_l.infer_output_schema(table.parsed_result) or [] + ) # Add datasets, and it's respective fields in schema_resolver, so that later schema_resolver would be able # correctly generates the upstreams for temporary tables @@ -320,9 +308,11 @@ def _get_sources_from_query( return ( sources, - parsed_result.column_lineage - if self.config.include_view_column_lineage - else None, + ( + parsed_result.column_lineage + if self.config.include_view_column_lineage + else None + ), ) def _build_s3_path_from_row(self, filename: str) -> str: @@ -376,11 +366,11 @@ def _get_sources( platform=platform.value, name=path, env=self.config.env, - platform_instance=self.config.platform_instance_map.get( - platform.value - ) - if self.config.platform_instance_map is not None - else None, + platform_instance=( + self.config.platform_instance_map.get(platform.value) + if self.config.platform_instance_map is not None + else None + ), ) elif source_schema is not None and source_table is not None: platform = LineageDatasetPlatform.REDSHIFT @@ -545,11 +535,11 @@ def _get_target_lineage( platform=target_platform.value, name=target_path, env=self.config.env, - platform_instance=self.config.platform_instance_map.get( - target_platform.value - ) - if self.config.platform_instance_map is not None - else None, + platform_instance=( + self.config.platform_instance_map.get(target_platform.value) + if self.config.platform_instance_map is not None + else None + ), ) except ValueError as e: self.warn(logger, "non-s3-lineage", str(e)) @@ -803,11 +793,11 @@ def get_lineage( mce_builder.make_dataset_urn_with_platform_instance( upstream_platform, f"{schema.external_database}.{tablename}", - platform_instance=self.config.platform_instance_map.get( - upstream_platform - ) - if self.config.platform_instance_map - else None, + platform_instance=( + self.config.platform_instance_map.get(upstream_platform) + if self.config.platform_instance_map + else None + ), env=self.config.env, ), DatasetLineageTypeClass.COPY, diff --git a/metadata-ingestion/src/datahub/ingestion/source/redshift/redshift_schema.py b/metadata-ingestion/src/datahub/ingestion/source/redshift/redshift_schema.py index 0ea073c050502..db8ff4dda8665 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/redshift/redshift_schema.py +++ b/metadata-ingestion/src/datahub/ingestion/source/redshift/redshift_schema.py @@ -8,8 +8,8 @@ from datahub.ingestion.source.redshift.query import RedshiftQuery from datahub.ingestion.source.sql.sql_generic import BaseColumn, BaseTable from datahub.metadata.com.linkedin.pegasus2avro.schema import SchemaField +from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult from datahub.utilities.hive_schema_to_avro import get_schema_fields_for_hive_column -from datahub.utilities.sqlglot_lineage import SqlParsingResult logger: logging.Logger = logging.getLogger(__name__) @@ -368,26 +368,38 @@ def get_lineage_rows( while rows: for row in rows: yield LineageRow( - source_schema=row[field_names.index("source_schema")] - if "source_schema" in field_names - else None, - source_table=row[field_names.index("source_table")] - if "source_table" in field_names - else None, - target_schema=row[field_names.index("target_schema")] - if "target_schema" in field_names - else None, - target_table=row[field_names.index("target_table")] - if "target_table" in field_names - else None, + source_schema=( + row[field_names.index("source_schema")] + if "source_schema" in field_names + else None + ), + source_table=( + row[field_names.index("source_table")] + if "source_table" in field_names + else None + ), + target_schema=( + row[field_names.index("target_schema")] + if "target_schema" in field_names + else None + ), + target_table=( + row[field_names.index("target_table")] + if "target_table" in field_names + else None + ), # See https://docs.aws.amazon.com/redshift/latest/dg/r_STL_QUERYTEXT.html # for why we need to remove the \\n. - ddl=row[field_names.index("ddl")].replace("\\n", "\n") - if "ddl" in field_names - else None, - filename=row[field_names.index("filename")] - if "filename" in field_names - else None, + ddl=( + row[field_names.index("ddl")].replace("\\n", "\n") + if "ddl" in field_names + else None + ), + filename=( + row[field_names.index("filename")] + if "filename" in field_names + else None + ), ) rows = cursor.fetchmany() diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_lineage_v2.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_lineage_v2.py index 142dbbf12f010..47434f8b9a963 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_lineage_v2.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_lineage_v2.py @@ -46,12 +46,9 @@ UpstreamLineage, ) from datahub.metadata.schema_classes import DatasetLineageTypeClass, UpstreamClass +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult, sqlglot_lineage from datahub.utilities.perf_timer import PerfTimer -from datahub.utilities.sqlglot_lineage import ( - SchemaResolver, - SqlParsingResult, - sqlglot_lineage, -) from datahub.utilities.time import ts_millis_to_datetime logger: logging.Logger = logging.getLogger(__name__) @@ -107,16 +104,20 @@ def __init__( def get_time_window(self) -> Tuple[datetime, datetime]: if self.redundant_run_skip_handler: return self.redundant_run_skip_handler.suggest_run_time_window( - self.config.start_time - if not self.config.ignore_start_time_lineage - else ts_millis_to_datetime(0), + ( + self.config.start_time + if not self.config.ignore_start_time_lineage + else ts_millis_to_datetime(0) + ), self.config.end_time, ) else: return ( - self.config.start_time - if not self.config.ignore_start_time_lineage - else ts_millis_to_datetime(0), + ( + self.config.start_time + if not self.config.ignore_start_time_lineage + else ts_millis_to_datetime(0) + ), self.config.end_time, ) @@ -153,9 +154,11 @@ def get_workunits( if self.redundant_run_skip_handler: # Update the checkpoint state for this run. self.redundant_run_skip_handler.update_state( - self.config.start_time - if not self.config.ignore_start_time_lineage - else ts_millis_to_datetime(0), + ( + self.config.start_time + if not self.config.ignore_start_time_lineage + else ts_millis_to_datetime(0) + ), self.config.end_time, ) @@ -629,9 +632,11 @@ def _should_ingest_lineage(self) -> bool: if ( self.redundant_run_skip_handler and self.redundant_run_skip_handler.should_skip_this_run( - cur_start_time=self.config.start_time - if not self.config.ignore_start_time_lineage - else ts_millis_to_datetime(0), + cur_start_time=( + self.config.start_time + if not self.config.ignore_start_time_lineage + else ts_millis_to_datetime(0) + ), cur_end_time=self.config.end_time, ) ): diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_v2.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_v2.py index 50f50ec647434..87f8a30bbe05f 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_v2.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_v2.py @@ -133,10 +133,10 @@ TimeType, ) from datahub.metadata.com.linkedin.pegasus2avro.tag import TagProperties +from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.utilities.file_backed_collections import FileBackedDict from datahub.utilities.perf_timer import PerfTimer from datahub.utilities.registries.domain_registry import DomainRegistry -from datahub.utilities.sqlglot_lineage import SchemaResolver logger: logging.Logger = logging.getLogger(__name__) @@ -1091,9 +1091,11 @@ def gen_dataset_workunits( yield dpi_aspect subTypes = SubTypes( - typeNames=[DatasetSubTypes.VIEW] - if isinstance(table, SnowflakeView) - else [DatasetSubTypes.TABLE] + typeNames=( + [DatasetSubTypes.VIEW] + if isinstance(table, SnowflakeView) + else [DatasetSubTypes.TABLE] + ) ) yield MetadataChangeProposalWrapper( @@ -1139,27 +1141,37 @@ def get_dataset_properties( ) -> DatasetProperties: return DatasetProperties( name=table.name, - created=TimeStamp(time=int(table.created.timestamp() * 1000)) - if table.created is not None - else None, - lastModified=TimeStamp(time=int(table.last_altered.timestamp() * 1000)) - if table.last_altered is not None - else TimeStamp(time=int(table.created.timestamp() * 1000)) - if table.created is not None - else None, + created=( + TimeStamp(time=int(table.created.timestamp() * 1000)) + if table.created is not None + else None + ), + lastModified=( + TimeStamp(time=int(table.last_altered.timestamp() * 1000)) + if table.last_altered is not None + else ( + TimeStamp(time=int(table.created.timestamp() * 1000)) + if table.created is not None + else None + ) + ), description=table.comment, qualifiedName=f"{db_name}.{schema_name}.{table.name}", customProperties={}, - externalUrl=self.get_external_url_for_table( - table.name, - schema_name, - db_name, - SnowflakeObjectDomain.TABLE - if isinstance(table, SnowflakeTable) - else SnowflakeObjectDomain.VIEW, - ) - if self.config.include_external_url - else None, + externalUrl=( + self.get_external_url_for_table( + table.name, + schema_name, + db_name, + ( + SnowflakeObjectDomain.TABLE + if isinstance(table, SnowflakeTable) + else SnowflakeObjectDomain.VIEW + ), + ) + if self.config.include_external_url + else None + ), ) def gen_tag_workunits(self, tag: SnowflakeTag) -> Iterable[MetadataWorkUnit]: @@ -1203,21 +1215,25 @@ def gen_schema_metadata( nativeDataType=col.get_precise_native_type(), description=col.comment, nullable=col.is_nullable, - isPartOfKey=col.name in table.pk.column_names - if isinstance(table, SnowflakeTable) and table.pk is not None - else None, - globalTags=GlobalTags( - [ - TagAssociation( - make_tag_urn( - self.snowflake_identifier(tag.identifier()) + isPartOfKey=( + col.name in table.pk.column_names + if isinstance(table, SnowflakeTable) and table.pk is not None + else None + ), + globalTags=( + GlobalTags( + [ + TagAssociation( + make_tag_urn( + self.snowflake_identifier(tag.identifier()) + ) ) - ) - for tag in table.column_tags[col.name] - ] - ) - if col.name in table.column_tags - else None, + for tag in table.column_tags[col.name] + ] + ) + if col.name in table.column_tags + else None + ), ) for col in table.columns ], @@ -1288,9 +1304,11 @@ def classify_snowflake_table( self.classification_handler.classify_schema_fields( dataset_name, schema_metadata, - table.sample_data.to_dict(orient="list") - if table.sample_data is not None - else {}, + ( + table.sample_data.to_dict(orient="list") + if table.sample_data is not None + else {} + ), ) except Exception as e: logger.debug( @@ -1325,21 +1343,31 @@ def gen_database_containers( sub_types=[DatasetContainerSubTypes.DATABASE], domain_registry=self.domain_registry, domain_config=self.config.domain, - external_url=self.get_external_url_for_database(database.name) - if self.config.include_external_url - else None, + external_url=( + self.get_external_url_for_database(database.name) + if self.config.include_external_url + else None + ), description=database.comment, - created=int(database.created.timestamp() * 1000) - if database.created is not None - else None, - last_modified=int(database.last_altered.timestamp() * 1000) - if database.last_altered is not None - else int(database.created.timestamp() * 1000) - if database.created is not None - else None, - tags=[self.snowflake_identifier(tag.identifier()) for tag in database.tags] - if database.tags - else None, + created=( + int(database.created.timestamp() * 1000) + if database.created is not None + else None + ), + last_modified=( + int(database.last_altered.timestamp() * 1000) + if database.last_altered is not None + else ( + int(database.created.timestamp() * 1000) + if database.created is not None + else None + ) + ), + tags=( + [self.snowflake_identifier(tag.identifier()) for tag in database.tags] + if database.tags + else None + ), ) def gen_schema_containers( @@ -1371,20 +1399,30 @@ def gen_schema_containers( sub_types=[DatasetContainerSubTypes.SCHEMA], domain_registry=self.domain_registry, description=schema.comment, - external_url=self.get_external_url_for_schema(schema.name, db_name) - if self.config.include_external_url - else None, - created=int(schema.created.timestamp() * 1000) - if schema.created is not None - else None, - last_modified=int(schema.last_altered.timestamp() * 1000) - if schema.last_altered is not None - else int(schema.created.timestamp() * 1000) - if schema.created is not None - else None, - tags=[self.snowflake_identifier(tag.identifier()) for tag in schema.tags] - if schema.tags - else None, + external_url=( + self.get_external_url_for_schema(schema.name, db_name) + if self.config.include_external_url + else None + ), + created=( + int(schema.created.timestamp() * 1000) + if schema.created is not None + else None + ), + last_modified=( + int(schema.last_altered.timestamp() * 1000) + if schema.last_altered is not None + else ( + int(schema.created.timestamp() * 1000) + if schema.created is not None + else None + ) + ), + tags=( + [self.snowflake_identifier(tag.identifier()) for tag in schema.tags] + if schema.tags + else None + ), ) def get_tables_for_schema( 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 a831dfa50342d..582c73803cdca 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/sql_common.py @@ -96,17 +96,17 @@ UpstreamClass, ViewPropertiesClass, ) +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import ( + SqlParsingResult, + sqlglot_lineage, + view_definition_lineage_helper, +) from datahub.telemetry import telemetry from datahub.utilities.file_backed_collections import FileBackedDict from datahub.utilities.lossy_collections import LossyList from datahub.utilities.registries.domain_registry import DomainRegistry from datahub.utilities.sqlalchemy_query_combiner import SQLAlchemyQueryCombinerReport -from datahub.utilities.sqlglot_lineage import ( - SchemaResolver, - SqlParsingResult, - sqlglot_lineage, - view_definition_lineage_helper, -) if TYPE_CHECKING: from datahub.ingestion.source.ge_data_profiler import ( diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/teradata.py b/metadata-ingestion/src/datahub/ingestion/source/sql/teradata.py index 4ea8dbe236c53..4f2fc799ecc30 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/teradata.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/teradata.py @@ -17,7 +17,6 @@ ) # This import verifies that the dependencies are available. -import teradatasqlalchemy # noqa: F401 import teradatasqlalchemy.types as custom_types from pydantic.fields import Field from sqlalchemy import create_engine, inspect @@ -53,12 +52,13 @@ from datahub.ingestion.source.usage.usage_common import BaseUsageConfig from datahub.ingestion.source_report.ingestion_stage import IngestionStageReport from datahub.ingestion.source_report.time_window import BaseTimeWindowReport -from datahub.metadata._schema_classes import SchemaMetadataClass from datahub.metadata.com.linkedin.pegasus2avro.schema import ( BytesTypeClass, TimeTypeClass, ) -from datahub.utilities.sqlglot_lineage import SchemaResolver, sqlglot_lineage +from datahub.metadata.schema_classes import SchemaMetadataClass +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import sqlglot_lineage logger: logging.Logger = logging.getLogger(__name__) @@ -573,9 +573,9 @@ def __init__(self, config: TeradataConfig, ctx: PipelineContext): self.graph: Optional[DataHubGraph] = ctx.graph self.builder: SqlParsingBuilder = SqlParsingBuilder( - usage_config=self.config.usage - if self.config.include_usage_statistics - else None, + usage_config=( + self.config.usage if self.config.include_usage_statistics else None + ), generate_lineage=True, generate_usage_statistics=self.config.include_usage_statistics, generate_operations=self.config.usage.include_operational_stats, @@ -782,9 +782,11 @@ def cache_tables_and_views(self) -> None: create_timestamp=entry.CreateTimeStamp, last_alter_name=entry.LastAlterName, last_alter_timestamp=entry.LastAlterTimeStamp, - request_text=entry.RequestText.strip() - if entry.object_type == "View" and entry.RequestText - else None, + request_text=( + entry.RequestText.strip() + if entry.object_type == "View" and entry.RequestText + else None + ), ) if table.database not in self._tables_cache: self._tables_cache[table.database] = [] @@ -836,9 +838,9 @@ def gen_lineage_from_query( sql=query.replace("(NOT CASESPECIFIC)", ""), schema_resolver=self.schema_resolver, default_db=None, - default_schema=default_database - if default_database - else self.config.default_db, + default_schema=( + default_database if default_database else self.config.default_db + ), ) if result.debug_info.table_error: logger.debug( diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql_queries.py b/metadata-ingestion/src/datahub/ingestion/source/sql_queries.py index c3d6657c81fa7..138430a68fcb9 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql_queries.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql_queries.py @@ -35,7 +35,8 @@ from datahub.ingestion.api.workunit import MetadataWorkUnit from datahub.ingestion.graph.client import DataHubGraph from datahub.ingestion.source.usage.usage_common import BaseUsageConfig -from datahub.utilities.sqlglot_lineage import SchemaResolver, sqlglot_lineage +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import sqlglot_lineage logger = logging.getLogger(__name__) @@ -219,9 +220,11 @@ def create( ) -> "QueryEntry": return cls( query=entry_dict["query"], - timestamp=datetime.fromtimestamp(entry_dict["timestamp"], tz=timezone.utc) - if "timestamp" in entry_dict - else None, + timestamp=( + datetime.fromtimestamp(entry_dict["timestamp"], tz=timezone.utc) + if "timestamp" in entry_dict + else None + ), user=make_user_urn(entry_dict["user"]) if "user" in entry_dict else None, operation_type=entry_dict.get("operation_type"), downstream_tables=[ diff --git a/metadata-ingestion/src/datahub/ingestion/source/tableau.py b/metadata-ingestion/src/datahub/ingestion/source/tableau.py index d761eb1c15f0d..c2091cacc1c4b 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/tableau.py +++ b/metadata-ingestion/src/datahub/ingestion/source/tableau.py @@ -142,12 +142,12 @@ SubTypesClass, ViewPropertiesClass, ) -from datahub.utilities import config_clean -from datahub.utilities.sqlglot_lineage import ( +from datahub.sql_parsing.sqlglot_lineage import ( ColumnLineageInfo, SqlParsingResult, create_lineage_sql_parsed_result, ) +from datahub.utilities import config_clean from datahub.utilities.urns.dataset_urn import DatasetUrn logger: logging.Logger = logging.getLogger(__name__) @@ -737,11 +737,13 @@ def _authenticate(self) -> None: def get_data_platform_instance(self) -> DataPlatformInstanceClass: return DataPlatformInstanceClass( platform=builder.make_data_platform_urn(self.platform), - instance=builder.make_dataplatform_instance_urn( - self.platform, self.config.platform_instance - ) - if self.config.platform_instance - else None, + instance=( + builder.make_dataplatform_instance_urn( + self.platform, self.config.platform_instance + ) + if self.config.platform_instance + else None + ), ) def get_connection_object_page( @@ -2088,9 +2090,11 @@ def emit_sheets_as_charts( description="", title=sheet.get(c.NAME) or "", lastModified=last_modified, - externalUrl=sheet_external_url - if self.config.ingest_external_links_for_charts - else None, + externalUrl=( + sheet_external_url + if self.config.ingest_external_links_for_charts + else None + ), inputs=sorted(datasource_urn), customProperties=self.get_custom_props_from_dict(sheet, [c.LUID]), ) @@ -2365,9 +2369,11 @@ def emit_dashboard( title=title, charts=chart_urns, lastModified=last_modified, - dashboardUrl=dashboard_external_url - if self.config.ingest_external_links_for_dashboards - else None, + dashboardUrl=( + dashboard_external_url + if self.config.ingest_external_links_for_dashboards + else None + ), customProperties=self.get_custom_props_from_dict(dashboard, [c.LUID]), ) dashboard_snapshot.aspects.append(dashboard_info_class) @@ -2498,9 +2504,11 @@ def emit_project_containers(self) -> Iterable[MetadataWorkUnit]: name=project.name, description=project.description, sub_types=[c.PROJECT], - parent_container_key=self.gen_project_key(project.parent_id) - if project.parent_id - else None, + parent_container_key=( + self.gen_project_key(project.parent_id) + if project.parent_id + else None + ), ) if ( project.parent_id is not None diff --git a/metadata-ingestion/src/datahub/ingestion/source/tableau_common.py b/metadata-ingestion/src/datahub/ingestion/source/tableau_common.py index f0f106dd0f20a..289f638c5cb98 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/tableau_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/tableau_common.py @@ -33,7 +33,7 @@ TagAssociationClass, UpstreamClass, ) -from datahub.utilities.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult +from datahub.sql_parsing.sqlglot_lineage import ColumnLineageInfo, SqlParsingResult logger = logging.getLogger(__name__) @@ -495,15 +495,17 @@ def tableau_field_to_schema_field(field, ingest_tags): field.get("description", ""), field.get("formula") ), nativeDataType=nativeDataType, - globalTags=get_tags_from_params( - [ - field.get("role", ""), - field.get("__typename", ""), - field.get("aggregation", ""), - ] - ) - if ingest_tags - else None, + globalTags=( + get_tags_from_params( + [ + field.get("role", ""), + field.get("__typename", ""), + field.get("aggregation", ""), + ] + ) + if ingest_tags + else None + ), ) return schema_field diff --git a/metadata-ingestion/src/datahub/ingestion/source/unity/source.py b/metadata-ingestion/src/datahub/ingestion/source/unity/source.py index 7a47b1181ae36..ac77cd7e5e2be 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/unity/source.py +++ b/metadata-ingestion/src/datahub/ingestion/source/unity/source.py @@ -103,15 +103,15 @@ UpstreamClass, UpstreamLineageClass, ) -from datahub.utilities.file_backed_collections import FileBackedDict -from datahub.utilities.hive_schema_to_avro import get_schema_fields_for_hive_column -from datahub.utilities.registries.domain_registry import DomainRegistry -from datahub.utilities.sqlglot_lineage import ( - SchemaResolver, +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import ( SqlParsingResult, sqlglot_lineage, view_definition_lineage_helper, ) +from datahub.utilities.file_backed_collections import FileBackedDict +from datahub.utilities.hive_schema_to_avro import get_schema_fields_for_hive_column +from datahub.utilities.registries.domain_registry import DomainRegistry logger: logging.Logger = logging.getLogger(__name__) @@ -338,14 +338,16 @@ def _gen_notebook_workunits(self, notebook: Notebook) -> Iterable[MetadataWorkUn externalUrl=urljoin( self.config.workspace_url, f"#notebook/{notebook.id}" ), - created=TimeStampClass(int(notebook.created_at.timestamp() * 1000)) - if notebook.created_at - else None, - lastModified=TimeStampClass( - int(notebook.modified_at.timestamp() * 1000) - ) - if notebook.modified_at - else None, + created=( + TimeStampClass(int(notebook.created_at.timestamp() * 1000)) + if notebook.created_at + else None + ), + lastModified=( + TimeStampClass(int(notebook.modified_at.timestamp() * 1000)) + if notebook.modified_at + else None + ), ), SubTypesClass(typeNames=[DatasetSubTypes.NOTEBOOK]), BrowsePathsClass(paths=notebook.path.split("/")), @@ -583,9 +585,9 @@ def _generate_lineage_aspect( if upstreams: return UpstreamLineageClass( upstreams=upstreams, - fineGrainedLineages=finegrained_lineages - if self.config.include_column_lineage - else None, + fineGrainedLineages=( + finegrained_lineages if self.config.include_column_lineage else None + ), ) else: return None @@ -662,9 +664,11 @@ def gen_catalog_containers(self, catalog: Catalog) -> Iterable[MetadataWorkUnit] name=catalog.name, sub_types=[DatasetContainerSubTypes.CATALOG], domain_urn=domain_urn, - parent_container_key=self.gen_metastore_key(catalog.metastore) - if self.config.include_metastore and catalog.metastore - else None, + parent_container_key=( + self.gen_metastore_key(catalog.metastore) + if self.config.include_metastore and catalog.metastore + else None + ), description=catalog.comment, owner_urn=self.get_owner_urn(catalog.owner), external_url=f"{self.external_url_base}/{catalog.name}", @@ -804,11 +808,13 @@ def _create_data_platform_instance_aspect( if self.config.ingest_data_platform_instance_aspect: return DataPlatformInstanceClass( platform=make_data_platform_urn(self.platform), - instance=make_dataplatform_instance_urn( - self.platform, self.platform_instance_name - ) - if self.platform_instance_name - else None, + instance=( + make_dataplatform_instance_urn( + self.platform, self.platform_instance_name + ) + if self.platform_instance_name + else None + ), ) return None diff --git a/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py b/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py index 4547f9f368198..e6117d70d0c02 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py @@ -133,19 +133,20 @@ def add_read_entry( query: Optional[str], fields: List[str], user_email_pattern: AllowDenyPattern = AllowDenyPattern.allow_all(), + count: int = 1, ) -> None: if user_email and not user_email_pattern.allowed(user_email): return - self.readCount += 1 + self.readCount += count if user_email is not None: - self.userFreq[user_email] += 1 + self.userFreq[user_email] += count if query: self.queryCount += 1 - self.queryFreq[query] += 1 + self.queryFreq[query] += count for column in fields: - self.columnFreq[column] += 1 + self.columnFreq[column] += count def make_usage_workunit( self, @@ -240,6 +241,7 @@ def aggregate_event( query: Optional[str], user: Optional[str], fields: List[str], + count: int = 1, ) -> None: floored_ts: datetime = get_time_bucket(start_time, self.config.bucket_duration) self.aggregation[floored_ts].setdefault( @@ -252,6 +254,7 @@ def aggregate_event( user, query, fields, + count=count, ) def generate_workunits( diff --git a/metadata-ingestion/src/datahub/ingestion/transformer/add_dataset_ownership.py b/metadata-ingestion/src/datahub/ingestion/transformer/add_dataset_ownership.py index 73cb8e4d6739b..5112a443768db 100644 --- a/metadata-ingestion/src/datahub/ingestion/transformer/add_dataset_ownership.py +++ b/metadata-ingestion/src/datahub/ingestion/transformer/add_dataset_ownership.py @@ -14,8 +14,11 @@ from datahub.ingestion.transformer.dataset_transformer import ( DatasetOwnershipTransformer, ) -from datahub.metadata._schema_classes import OwnershipTypeClass -from datahub.metadata.schema_classes import OwnerClass, OwnershipClass +from datahub.metadata.schema_classes import ( + OwnerClass, + OwnershipClass, + OwnershipTypeClass, +) class AddDatasetOwnershipConfig(TransformerSemanticsConfigModel): @@ -73,9 +76,11 @@ def transform_aspect( in_ownership_aspect: Optional[OwnershipClass] = cast(OwnershipClass, aspect) out_ownership_aspect: OwnershipClass = OwnershipClass( owners=[], - lastModified=in_ownership_aspect.lastModified - if in_ownership_aspect is not None - else None, + lastModified=( + in_ownership_aspect.lastModified + if in_ownership_aspect is not None + else None + ), ) # Check if user want to keep existing ownerships diff --git a/metadata-ingestion/src/datahub/ingestion/transformer/extract_ownership_from_tags.py b/metadata-ingestion/src/datahub/ingestion/transformer/extract_ownership_from_tags.py index 6266e0bca6c61..c4eba0e011de3 100644 --- a/metadata-ingestion/src/datahub/ingestion/transformer/extract_ownership_from_tags.py +++ b/metadata-ingestion/src/datahub/ingestion/transformer/extract_ownership_from_tags.py @@ -8,9 +8,9 @@ from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.transformer.dataset_transformer import DatasetTagsTransformer -from datahub.metadata._schema_classes import MetadataChangeProposalClass from datahub.metadata.schema_classes import ( GlobalTagsClass, + MetadataChangeProposalClass, OwnerClass, OwnershipClass, OwnershipTypeClass, diff --git a/metadata-ingestion/src/datahub/sql_parsing/__init__.py b/metadata-ingestion/src/datahub/sql_parsing/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/metadata-ingestion/src/datahub/sql_parsing/_models.py b/metadata-ingestion/src/datahub/sql_parsing/_models.py new file mode 100644 index 0000000000000..bbd3a9c9d11f8 --- /dev/null +++ b/metadata-ingestion/src/datahub/sql_parsing/_models.py @@ -0,0 +1,78 @@ +import functools +from typing import Any, Optional + +import sqlglot +from pydantic import BaseModel + +from datahub.configuration.pydantic_migration_helpers import PYDANTIC_VERSION_2 +from datahub.metadata.schema_classes import SchemaFieldDataTypeClass + + +class _ParserBaseModel( + BaseModel, + arbitrary_types_allowed=True, + json_encoders={ + SchemaFieldDataTypeClass: lambda v: v.to_obj(), + }, +): + def json(self, *args: Any, **kwargs: Any) -> str: + if PYDANTIC_VERSION_2: + return super().model_dump_json(*args, **kwargs) # type: ignore + else: + return super().json(*args, **kwargs) + + +@functools.total_ordering +class _FrozenModel(_ParserBaseModel, frozen=True): + def __lt__(self, other: "_FrozenModel") -> bool: + # TODO: The __fields__ attribute is deprecated in Pydantic v2. + for field in self.__fields__: + self_v = getattr(self, field) + other_v = getattr(other, field) + if self_v != other_v: + return self_v < other_v + + return False + + +class _TableName(_FrozenModel): + database: Optional[str] = None + db_schema: Optional[str] = None + table: str + + def as_sqlglot_table(self) -> sqlglot.exp.Table: + return sqlglot.exp.Table( + catalog=( + sqlglot.exp.Identifier(this=self.database) if self.database else None + ), + db=sqlglot.exp.Identifier(this=self.db_schema) if self.db_schema else None, + this=sqlglot.exp.Identifier(this=self.table), + ) + + def qualified( + self, + dialect: sqlglot.Dialect, + default_db: Optional[str] = None, + default_schema: Optional[str] = None, + ) -> "_TableName": + database = self.database or default_db + db_schema = self.db_schema or default_schema + + return _TableName( + database=database, + db_schema=db_schema, + table=self.table, + ) + + @classmethod + def from_sqlglot_table( + cls, + table: sqlglot.exp.Table, + default_db: Optional[str] = None, + default_schema: Optional[str] = None, + ) -> "_TableName": + return cls( + database=table.catalog or default_db, + db_schema=table.db or default_schema, + table=table.this.name, + ) diff --git a/metadata-ingestion/src/datahub/sql_parsing/schema_resolver.py b/metadata-ingestion/src/datahub/sql_parsing/schema_resolver.py new file mode 100644 index 0000000000000..ce43fb5da57c0 --- /dev/null +++ b/metadata-ingestion/src/datahub/sql_parsing/schema_resolver.py @@ -0,0 +1,255 @@ +import contextlib +import pathlib +from typing import Dict, List, Optional, Protocol, Set, Tuple + +from typing_extensions import TypedDict + +from datahub.emitter.mce_builder import ( + DEFAULT_ENV, + make_dataset_urn_with_platform_instance, +) +from datahub.ingestion.api.closeable import Closeable +from datahub.ingestion.graph.client import DataHubGraph +from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier +from datahub.metadata.schema_classes import SchemaFieldClass, SchemaMetadataClass +from datahub.metadata.urns import DataPlatformUrn +from datahub.sql_parsing._models import _TableName +from datahub.sql_parsing.sql_parsing_common import PLATFORMS_WITH_CASE_SENSITIVE_TABLES +from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedDict +from datahub.utilities.urns.field_paths import get_simple_field_path_from_v2_field_path + +# A lightweight table schema: column -> type mapping. +SchemaInfo = Dict[str, str] + + +class GraphQLSchemaField(TypedDict): + fieldPath: str + nativeDataType: str + + +class GraphQLSchemaMetadata(TypedDict): + fields: List[GraphQLSchemaField] + + +class SchemaResolverInterface(Protocol): + @property + def platform(self) -> str: + ... + + def resolve_table(self, table: _TableName) -> Tuple[str, Optional[SchemaInfo]]: + ... + + def __hash__(self) -> int: + # Mainly to make lru_cache happy in methods that accept a schema resolver. + return id(self) + + +class SchemaResolver(Closeable, SchemaResolverInterface): + def __init__( + self, + *, + platform: str, + platform_instance: Optional[str] = None, + env: str = DEFAULT_ENV, + graph: Optional[DataHubGraph] = None, + _cache_filename: Optional[pathlib.Path] = None, + ): + # Also supports platform with an urn prefix. + self._platform = DataPlatformUrn(platform).platform_name + self.platform_instance = platform_instance + self.env = env + + self.graph = graph + + # Init cache, potentially restoring from a previous run. + shared_conn = None + if _cache_filename: + shared_conn = ConnectionWrapper(filename=_cache_filename) + self._schema_cache: FileBackedDict[Optional[SchemaInfo]] = FileBackedDict( + shared_connection=shared_conn, + extra_columns={"is_missing": lambda v: v is None}, + ) + + @property + def platform(self) -> str: + return self._platform + + def get_urns(self) -> Set[str]: + return set(k for k, v in self._schema_cache.items() if v is not None) + + def schema_count(self) -> int: + return int( + self._schema_cache.sql_query( + f"SELECT COUNT(*) FROM {self._schema_cache.tablename} WHERE is_missing" + )[0][0] + ) + + def get_urn_for_table(self, table: _TableName, lower: bool = False) -> str: + # TODO: Validate that this is the correct 2/3 layer hierarchy for the platform. + + table_name = ".".join( + filter(None, [table.database, table.db_schema, table.table]) + ) + + platform_instance = self.platform_instance + + if lower: + table_name = table_name.lower() + platform_instance = platform_instance.lower() if platform_instance else None + + if self.platform == "bigquery": + # Normalize shard numbers and other BigQuery weirdness. + with contextlib.suppress(IndexError): + table_name = BigqueryTableIdentifier.from_string_name( + table_name + ).get_table_name() + + urn = make_dataset_urn_with_platform_instance( + platform=self.platform, + platform_instance=platform_instance, + env=self.env, + name=table_name, + ) + return urn + + def resolve_table(self, table: _TableName) -> Tuple[str, Optional[SchemaInfo]]: + urn = self.get_urn_for_table(table) + + schema_info = self._resolve_schema_info(urn) + if schema_info: + return urn, schema_info + + urn_lower = self.get_urn_for_table(table, lower=True) + if urn_lower != urn: + schema_info = self._resolve_schema_info(urn_lower) + if schema_info: + return urn_lower, schema_info + + if self._prefers_urn_lower(): + return urn_lower, None + else: + return urn, None + + def _prefers_urn_lower(self) -> bool: + return self.platform not in PLATFORMS_WITH_CASE_SENSITIVE_TABLES + + def has_urn(self, urn: str) -> bool: + return self._schema_cache.get(urn) is not None + + def _resolve_schema_info(self, urn: str) -> Optional[SchemaInfo]: + if urn in self._schema_cache: + return self._schema_cache[urn] + + # TODO: For bigquery partitioned tables, add the pseudo-column _PARTITIONTIME + # or _PARTITIONDATE where appropriate. + + if self.graph: + schema_info = self._fetch_schema_info(self.graph, urn) + if schema_info: + self._save_to_cache(urn, schema_info) + return schema_info + + self._save_to_cache(urn, None) + return None + + def add_schema_metadata( + self, urn: str, schema_metadata: SchemaMetadataClass + ) -> None: + schema_info = self._convert_schema_aspect_to_info(schema_metadata) + self._save_to_cache(urn, schema_info) + + def add_raw_schema_info(self, urn: str, schema_info: SchemaInfo) -> None: + self._save_to_cache(urn, schema_info) + + def add_graphql_schema_metadata( + self, urn: str, schema_metadata: GraphQLSchemaMetadata + ) -> None: + schema_info = self.convert_graphql_schema_metadata_to_info(schema_metadata) + self._save_to_cache(urn, schema_info) + + def with_temp_tables( + self, temp_tables: Dict[str, Optional[List[SchemaFieldClass]]] + ) -> SchemaResolverInterface: + extra_schemas = { + urn: ( + self._convert_schema_field_list_to_info(fields) + if fields is not None + else None + ) + for urn, fields in temp_tables.items() + } + + return _SchemaResolverWithExtras( + base_resolver=self, extra_schemas=extra_schemas + ) + + def _save_to_cache(self, urn: str, schema_info: Optional[SchemaInfo]) -> None: + self._schema_cache[urn] = schema_info + + def _fetch_schema_info(self, graph: DataHubGraph, urn: str) -> Optional[SchemaInfo]: + aspect = graph.get_aspect(urn, SchemaMetadataClass) + if not aspect: + return None + + return self._convert_schema_aspect_to_info(aspect) + + @classmethod + def _convert_schema_aspect_to_info( + cls, schema_metadata: SchemaMetadataClass + ) -> SchemaInfo: + return cls._convert_schema_field_list_to_info(schema_metadata.fields) + + @classmethod + def _convert_schema_field_list_to_info( + cls, schema_fields: List[SchemaFieldClass] + ) -> SchemaInfo: + return { + get_simple_field_path_from_v2_field_path(col.fieldPath): ( + # The actual types are more of a "nice to have". + col.nativeDataType + or "str" + ) + for col in schema_fields + # TODO: We can't generate lineage to columns nested within structs yet. + if "." not in get_simple_field_path_from_v2_field_path(col.fieldPath) + } + + @classmethod + def convert_graphql_schema_metadata_to_info( + cls, schema: GraphQLSchemaMetadata + ) -> SchemaInfo: + return { + get_simple_field_path_from_v2_field_path(field["fieldPath"]): ( + # The actual types are more of a "nice to have". + field["nativeDataType"] + or "str" + ) + for field in schema["fields"] + # TODO: We can't generate lineage to columns nested within structs yet. + if "." not in get_simple_field_path_from_v2_field_path(field["fieldPath"]) + } + + def close(self) -> None: + self._schema_cache.close() + + +class _SchemaResolverWithExtras(SchemaResolverInterface): + def __init__( + self, + base_resolver: SchemaResolver, + extra_schemas: Dict[str, Optional[SchemaInfo]], + ): + self._base_resolver = base_resolver + self._extra_schemas = extra_schemas + + @property + def platform(self) -> str: + return self._base_resolver.platform + + def resolve_table(self, table: _TableName) -> Tuple[str, Optional[SchemaInfo]]: + urn = self._base_resolver.get_urn_for_table( + table, lower=self._base_resolver._prefers_urn_lower() + ) + if urn in self._extra_schemas: + return urn, self._extra_schemas[urn] + return self._base_resolver.resolve_table(table) diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator_v2.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator_v2.py new file mode 100644 index 0000000000000..cb23c9244cd86 --- /dev/null +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator_v2.py @@ -0,0 +1,902 @@ +import dataclasses +import enum +import itertools +import json +import logging +import pathlib +import tempfile +from collections import defaultdict +from datetime import datetime, timezone +from typing import Callable, Dict, Iterable, List, Optional, Set, cast + +import datahub.metadata.schema_classes as models +from datahub.emitter.mce_builder import get_sys_time, make_ts_millis +from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.emitter.sql_parsing_builder import compute_upstream_fields +from datahub.ingestion.api.report import Report +from datahub.ingestion.graph.client import DataHubGraph +from datahub.ingestion.source.usage.usage_common import BaseUsageConfig, UsageAggregator +from datahub.metadata.urns import ( + CorpUserUrn, + DataPlatformUrn, + DatasetUrn, + QueryUrn, + SchemaFieldUrn, +) +from datahub.sql_parsing.schema_resolver import SchemaResolver, SchemaResolverInterface +from datahub.sql_parsing.sql_parsing_common import QueryType +from datahub.sql_parsing.sqlglot_lineage import ( + ColumnLineageInfo, + ColumnRef, + SqlParsingResult, + infer_output_schema, + sqlglot_lineage, +) +from datahub.sql_parsing.sqlglot_utils import generate_hash +from datahub.utilities.file_backed_collections import ( + ConnectionWrapper, + FileBackedDict, + FileBackedList, +) +from datahub.utilities.lossy_collections import LossyDict, LossyList +from datahub.utilities.ordered_set import OrderedSet + +logger = logging.getLogger(__name__) +QueryId = str +UrnStr = str + +_DEFAULT_USER_URN = CorpUserUrn("_ingestion") +_MISSING_SESSION_ID = "__MISSING_SESSION_ID" + + +class QueryLogSetting(enum.Enum): + DISABLED = "DISABLED" + STORE_ALL = "STORE_ALL" + STORE_FAILED = "STORE_FAILED" + + +@dataclasses.dataclass +class ViewDefinition: + # TODO view urn? + + view_definition: str + default_db: Optional[str] = None + default_schema: Optional[str] = None + + +@dataclasses.dataclass +class QueryMetadata: + query_id: QueryId + + # raw_query_string: str + formatted_query_string: str + + session_id: str # will be _MISSING_SESSION_ID if not present + query_type: QueryType + lineage_type: str # from models.DatasetLineageTypeClass + latest_timestamp: Optional[datetime] + actor: Optional[CorpUserUrn] + + upstreams: List[UrnStr] # this is direct upstreams, which may be temp tables + column_lineage: List[ColumnLineageInfo] + confidence_score: float + + def make_created_audit_stamp(self) -> models.AuditStampClass: + return models.AuditStampClass( + time=make_ts_millis(self.latest_timestamp) or 0, + actor=(self.actor or _DEFAULT_USER_URN).urn(), + ) + + def make_last_modified_audit_stamp(self) -> models.AuditStampClass: + return models.AuditStampClass( + time=make_ts_millis(self.latest_timestamp or datetime.now(tz=timezone.utc)) + or 0, + actor=(self.actor or _DEFAULT_USER_URN).urn(), + ) + + +@dataclasses.dataclass +class SqlAggregatorReport(Report): + _aggregator: "SqlParsingAggregator" + query_log_path: Optional[str] = None + + num_observed_queries: int = 0 + num_observed_queries_failed: int = 0 + num_observed_queries_column_failed: int = 0 + observed_query_parse_failures = LossyList[str]() + + num_view_definitions: int = 0 + num_views_failed: int = 0 + num_views_column_failed: int = 0 + views_parse_failures = LossyDict[UrnStr, str]() + + num_queries_with_temp_tables_in_session: int = 0 + + num_unique_query_fingerprints: Optional[int] = None + + # Lineage-related. + num_urns_with_lineage: Optional[int] = None + num_temp_sessions: Optional[int] = None + num_inferred_temp_schemas: Optional[int] = None + queries_with_temp_upstreams: LossyDict[QueryId, LossyList] = dataclasses.field( + default_factory=LossyDict + ) + + num_queries_entities_generated: int = 0 + + # Usage-related. + usage_skipped_missing_timestamp: int = 0 + + def compute_stats(self) -> None: + self.num_unique_query_fingerprints = len(self._aggregator._query_map) + + self.num_urns_with_lineage = len(self._aggregator._lineage_map) + self.num_temp_sessions = len(self._aggregator._temp_lineage_map) + self.num_inferred_temp_schemas = len(self._aggregator._inferred_temp_schemas) + + return super().compute_stats() + + +class SqlParsingAggregator: + def __init__( + self, + *, + platform: str, + platform_instance: Optional[str], + env: str, + graph: Optional[DataHubGraph] = None, + generate_lineage: bool = True, + generate_queries: bool = True, + generate_usage_statistics: bool = False, + generate_operations: bool = False, + usage_config: Optional[BaseUsageConfig] = None, + is_temp_table: Optional[Callable[[UrnStr], bool]] = None, + query_log: QueryLogSetting = QueryLogSetting.DISABLED, + ) -> None: + self.platform = DataPlatformUrn(platform) + self.platform_instance = platform_instance + self.env = env + + self.generate_lineage = generate_lineage + self.generate_queries = generate_queries + self.generate_usage_statistics = generate_usage_statistics + self.generate_operations = generate_operations + if self.generate_queries and not self.generate_lineage: + raise ValueError("Queries will only be generated if lineage is enabled") + + self.usage_config = usage_config + if self.generate_usage_statistics and self.usage_config is None: + raise ValueError("Usage statistics generation requires a usage config") + + self.report = SqlAggregatorReport(_aggregator=self) + + # can be used by BQ where we have a "temp_table_dataset_prefix" + self.is_temp_table = is_temp_table + + self.query_log = query_log + + # Set up the schema resolver. + self._schema_resolver: SchemaResolver + if graph is None: + self._schema_resolver = SchemaResolver( + platform=self.platform.platform_name, + platform_instance=self.platform_instance, + env=self.env, + ) + else: + self._schema_resolver = None # type: ignore + self._initialize_schema_resolver_from_graph(graph) + + # Initialize internal data structures. + # This leans pretty heavily on the our query fingerprinting capabilities. + # In particular, it must be true that if two queries have the same fingerprint, + # they must generate the same lineage. + + self._shared_connection: Optional[ConnectionWrapper] = None + if self.query_log != QueryLogSetting.DISABLED: + # Initialize and log a file to store the queries. + query_log_path = pathlib.Path(tempfile.mkdtemp()) / "query_log.db" + self.report.query_log_path = str(query_log_path) + + # By providing a filename explicitly here, we also ensure that the file + # is not automatically deleted on exit. + self._shared_connection = ConnectionWrapper(filename=query_log_path) + + # Stores the logged queries. + self._logged_queries = FileBackedList[str]( + shared_connection=self._shared_connection, tablename="stored_queries" + ) + + # Map of query_id -> QueryMetadata + self._query_map = FileBackedDict[QueryMetadata]( + shared_connection=self._shared_connection, tablename="query_map" + ) + + # Map of downstream urn -> { query ids } + self._lineage_map = FileBackedDict[OrderedSet[QueryId]]( + shared_connection=self._shared_connection, tablename="lineage_map" + ) + + # Map of view urn -> view definition + self._view_definitions = FileBackedDict[ViewDefinition]( + shared_connection=self._shared_connection, tablename="view_definitions" + ) + + # Map of session ID -> {temp table name -> query id} + # Needs to use the query_map to find the info about the query. + # This assumes that a temp table is created at most once per session. + self._temp_lineage_map = FileBackedDict[Dict[UrnStr, QueryId]]( + shared_connection=self._shared_connection, tablename="temp_lineage_map" + ) + + # Map of query ID -> schema fields, only for query IDs that generate temp tables. + self._inferred_temp_schemas = FileBackedDict[List[models.SchemaFieldClass]]( + shared_connection=self._shared_connection, tablename="inferred_temp_schemas" + ) + + # Usage aggregator. This will only be initialized if usage statistics are enabled. + # TODO: Replace with FileBackedDict. + self._usage_aggregator: Optional[UsageAggregator[UrnStr]] = None + if self.generate_usage_statistics: + assert self.usage_config is not None + self._usage_aggregator = UsageAggregator(config=self.usage_config) + + @property + def _need_schemas(self) -> bool: + return self.generate_lineage or self.generate_usage_statistics + + def register_schema( + self, urn: DatasetUrn, schema: models.SchemaMetadataClass + ) -> None: + # If lineage or usage is enabled, adds the schema to the schema resolver + # by putting the condition in here, we can avoid all the conditional + # logic that we previously needed in each source + + if self._need_schemas: + self._schema_resolver.add_schema_metadata(str(urn), schema) + + def _initialize_schema_resolver_from_graph(self, graph: DataHubGraph) -> None: + # requires a graph instance + # if no schemas are currently registered in the schema resolver + # and we need the schema resolver (e.g. lineage or usage is enabled) + # then use the graph instance to fetch all schemas for the + # platform/instance/env combo + if not self._need_schemas: + return + + if ( + self._schema_resolver is not None + and self._schema_resolver.schema_count() > 0 + ): + # TODO: Have a mechanism to override this, e.g. when table ingestion is enabled but view ingestion is not. + logger.info( + "Not fetching any schemas from the graph, since " + f"there are {self._schema_resolver.schema_count()} schemas already registered." + ) + return + + # TODO: The initialize_schema_resolver_from_datahub method should take in a SchemaResolver + # that it can populate or add to, rather than creating a new one and dropping any schemas + # that were already loaded into the existing one. + self._schema_resolver = graph.initialize_schema_resolver_from_datahub( + platform=self.platform.urn(), + platform_instance=self.platform_instance, + env=self.env, + ) + + def add_view_definition( + self, + view_urn: DatasetUrn, + view_definition: str, + default_db: Optional[str] = None, + default_schema: Optional[str] = None, + ) -> None: + """Add a view definition to the aggregator. + + View definitions always contribute to lineage, but do not count towards + usage statistics or operations. + + The actual processing of view definitions is deferred until output time, + since all schemas will be registered at that point. + """ + + self.report.num_view_definitions += 1 + + self._view_definitions[str(view_urn)] = ViewDefinition( + view_definition=view_definition, + default_db=default_db, + default_schema=default_schema, + ) + + def add_observed_query( + self, + query: str, + default_db: Optional[str] = None, + default_schema: Optional[str] = None, + query_timestamp: Optional[datetime] = None, + user: Optional[CorpUserUrn] = None, + session_id: Optional[ + str + ] = None, # can only see temp tables with the same session + usage_multiplier: int = 1, + is_known_temp_table: bool = False, + require_out_table_schema: bool = False, + ) -> None: + """Add an observed query to the aggregator. + + This will always generate usage. If it's a mutation query, it will also generate + lineage and an operation. If it's a temp table, the lineage gets logged in a separate + map, which will get used in subsequent queries with the same session ID. + + This assumes that queries come in order of increasing timestamps. + """ + + self.report.num_observed_queries += 1 + + # All queries with no session ID are assumed to be part of the same session. + session_id = session_id or _MISSING_SESSION_ID + + # Load in the temp tables for this session. + schema_resolver: SchemaResolverInterface = ( + self._make_schema_resolver_for_session(session_id) + ) + + # Run the SQL parser. + parsed = self._run_sql_parser( + query, + default_db=default_db, + default_schema=default_schema, + schema_resolver=schema_resolver, + ) + if parsed.debug_info.error: + self.report.observed_query_parse_failures.append( + str(parsed.debug_info.error) + ) + if parsed.debug_info.table_error: + self.report.num_observed_queries_failed += 1 + return # we can't do anything with this query + elif parsed.debug_info.error: + self.report.num_observed_queries_column_failed += 1 + + # Register the query's usage. + if not self._usage_aggregator: + pass # usage is not enabled + elif query_timestamp is None: + self.report.usage_skipped_missing_timestamp += 1 + else: + # TODO: We need a full list of columns referenced, not just the out tables. + upstream_fields = compute_upstream_fields(parsed) + for upstream_urn in parsed.in_tables: + self._usage_aggregator.aggregate_event( + resource=upstream_urn, + start_time=query_timestamp, + query=query, + user=user.urn() if user else None, + fields=sorted(upstream_fields.get(upstream_urn, [])), + count=usage_multiplier, + ) + + if not parsed.out_tables: + return + out_table = parsed.out_tables[0] + query_fingerprint = parsed.query_fingerprint + assert query_fingerprint is not None + + # Register the query. + self._add_to_query_map( + QueryMetadata( + query_id=query_fingerprint, + formatted_query_string=query, # TODO replace with formatted query string + session_id=session_id, + query_type=parsed.query_type, + lineage_type=models.DatasetLineageTypeClass.TRANSFORMED, + latest_timestamp=query_timestamp, + actor=user, + upstreams=parsed.in_tables, + column_lineage=parsed.column_lineage or [], + confidence_score=parsed.debug_info.confidence, + ) + ) + + # Register the query's lineage. + if ( + is_known_temp_table + or ( + parsed.query_type.is_create() + and parsed.query_type_props.get("temporary") + ) + or (self.is_temp_table and self.is_temp_table(out_table)) + or ( + require_out_table_schema + and not self._schema_resolver.has_urn(out_table) + ) + ): + # Infer the schema of the output table and track it for later. + inferred_schema = infer_output_schema(parsed) + if inferred_schema is not None: + self._inferred_temp_schemas[query_fingerprint] = inferred_schema + + # Also track the lineage for the temp table, for merging purposes later. + self._temp_lineage_map.for_mutation(session_id, {})[ + out_table + ] = query_fingerprint + + else: + # Non-temp tables immediately generate lineage. + self._lineage_map.for_mutation(out_table, OrderedSet()).add( + query_fingerprint + ) + + def _make_schema_resolver_for_session( + self, session_id: str + ) -> SchemaResolverInterface: + schema_resolver: SchemaResolverInterface = self._schema_resolver + if session_id in self._temp_lineage_map: + temp_table_schemas: Dict[str, Optional[List[models.SchemaFieldClass]]] = {} + for temp_table_urn, query_id in self._temp_lineage_map[session_id].items(): + temp_table_schemas[temp_table_urn] = self._inferred_temp_schemas.get( + query_id + ) + + if temp_table_schemas: + schema_resolver = self._schema_resolver.with_temp_tables( + temp_table_schemas + ) + self.report.num_queries_with_temp_tables_in_session += 1 + + return schema_resolver + + def _process_view_definition( + self, view_urn: UrnStr, view_definition: ViewDefinition + ) -> None: + # Run the SQL parser. + parsed = self._run_sql_parser( + view_definition.view_definition, + default_db=view_definition.default_db, + default_schema=view_definition.default_schema, + schema_resolver=self._schema_resolver, + ) + if parsed.debug_info.error: + self.report.views_parse_failures[view_urn] = str(parsed.debug_info.error) + if parsed.debug_info.table_error: + self.report.num_views_failed += 1 + return # we can't do anything with this query + elif parsed.debug_info.error: + self.report.num_views_column_failed += 1 + + # Note that in some cases, the view definition will be a SELECT statement + # instead of a CREATE VIEW ... AS SELECT statement. In those cases, we can't + # trust the parsed query type or downstream urn. + + query_fingerprint = self._view_query_id(view_urn) + + # Register the query. + self._add_to_query_map( + QueryMetadata( + query_id=query_fingerprint, + formatted_query_string=view_definition.view_definition, + session_id=_MISSING_SESSION_ID, + query_type=QueryType.CREATE_VIEW, + lineage_type=models.DatasetLineageTypeClass.VIEW, + latest_timestamp=None, + actor=None, + upstreams=parsed.in_tables, + column_lineage=parsed.column_lineage or [], + confidence_score=parsed.debug_info.confidence, + ) + ) + + # Register the query's lineage. + self._lineage_map.for_mutation(view_urn, OrderedSet()).add(query_fingerprint) + + def _run_sql_parser( + self, + query: str, + default_db: Optional[str], + default_schema: Optional[str], + schema_resolver: SchemaResolverInterface, + ) -> SqlParsingResult: + parsed = sqlglot_lineage( + query, + schema_resolver=schema_resolver, + default_db=default_db, + default_schema=default_schema, + ) + + # Conditionally log the query. + if self.query_log == QueryLogSetting.STORE_ALL or ( + self.query_log == QueryLogSetting.STORE_FAILED and parsed.debug_info.error + ): + self._logged_queries.append(query) + + # Also add some extra logging. + if parsed.debug_info.error: + logger.debug( + f"Error parsing query {query}: {parsed.debug_info.error}", + exc_info=parsed.debug_info.error, + ) + + return parsed + + def _add_to_query_map(self, new: QueryMetadata) -> None: + query_fingerprint = new.query_id + + if query_fingerprint in self._query_map: + current = self._query_map.for_mutation(query_fingerprint) + + # This assumes that queries come in order of increasing timestamps, + # so the current query is more authoritative than the previous one. + current.formatted_query_string = new.formatted_query_string + current.session_id = new.session_id + current.latest_timestamp = new.latest_timestamp or current.latest_timestamp + current.actor = new.actor or current.actor + + # An invariant of the fingerprinting is that if two queries have the + # same fingerprint, they must also have the same lineage. We overwrite + # here just in case more schemas got registered in the interim. + current.upstreams = new.upstreams + current.column_lineage = new.column_lineage + current.confidence_score = new.confidence_score + else: + self._query_map[query_fingerprint] = new + + """ + def add_lineage(self) -> None: + # A secondary mechanism for adding non-SQL-based lineage + # e.g. redshift external tables might use this when pointing at s3 + + # TODO Add this once we have a use case for it + pass + """ + + def gen_metadata(self) -> Iterable[MetadataChangeProposalWrapper]: + # diff from v1 - we generate operations here, and it also + # generates MCPWs instead of workunits + yield from self._gen_lineage_mcps() + yield from self._gen_usage_statistics_mcps() + yield from self._gen_operation_mcps() + + def _gen_lineage_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: + if not self.generate_lineage: + return + + # Process all views and inject them into the lineage map. + # The parsing of view definitions is deferred until this point + # to ensure the availability of all schema metadata. + for view_urn, view_definition in self._view_definitions.items(): + self._process_view_definition(view_urn, view_definition) + self._view_definitions.clear() + + # Generate lineage and queries. + queries_generated: Set[QueryId] = set() + for downstream_urn in self._lineage_map: + yield from self._gen_lineage_for_downstream( + downstream_urn, queries_generated=queries_generated + ) + + @classmethod + def _query_type_precedence(cls, query_type: str) -> int: + query_precedence = [ + models.DatasetLineageTypeClass.COPY, + models.DatasetLineageTypeClass.VIEW, + models.DatasetLineageTypeClass.TRANSFORMED, + ] + + idx = query_precedence.index(query_type) + if idx == -1: + return len(query_precedence) + return idx + + def _gen_lineage_for_downstream( + self, downstream_urn: str, queries_generated: Set[QueryId] + ) -> Iterable[MetadataChangeProposalWrapper]: + query_ids = self._lineage_map[downstream_urn] + queries: List[QueryMetadata] = [ + self._resolve_query_with_temp_tables(self._query_map[query_id]) + for query_id in query_ids + ] + + # Sort the queries by highest precedence first, then by latest timestamp. + # Tricky: by converting the timestamp to a number, we also can ignore the + # differences between naive and aware datetimes. + queries = sorted( + reversed(queries), + key=lambda query: ( + self._query_type_precedence(query.lineage_type), + -(make_ts_millis(query.latest_timestamp) or 0), + ), + ) + + queries_map: Dict[QueryId, QueryMetadata] = { + query.query_id: query for query in queries + } + + # mapping of upstream urn -> query id that produced it + upstreams: Dict[UrnStr, QueryId] = {} + # mapping of downstream column -> { upstream column -> query id that produced it } + cll: Dict[str, Dict[SchemaFieldUrn, QueryId]] = defaultdict(dict) + + for query in queries: + # Using setdefault to respect the precedence of queries. + + for upstream in query.upstreams: + upstreams.setdefault(upstream, query.query_id) + + for lineage_info in query.column_lineage: + for upstream_ref in lineage_info.upstreams: + cll[lineage_info.downstream.column].setdefault( + SchemaFieldUrn(upstream_ref.table, upstream_ref.column), + query.query_id, + ) + + # Finally, we can build our lineage edge. + required_queries = OrderedSet[QueryId]() + upstream_aspect = models.UpstreamLineageClass(upstreams=[]) + for upstream_urn, query_id in upstreams.items(): + required_queries.add(query_id) + + upstream_aspect.upstreams.append( + models.UpstreamClass( + dataset=upstream_urn, + type=queries_map[query_id].lineage_type, + query=( + self._query_urn(query_id) if self.generate_queries else None + ), + created=query.make_created_audit_stamp(), + auditStamp=models.AuditStampClass( + time=get_sys_time(), + actor=_DEFAULT_USER_URN.urn(), + ), + ) + ) + upstream_aspect.fineGrainedLineages = [] + for downstream_column, all_upstream_columns in cll.items(): + # Group by query ID. + for query_id, upstream_columns_for_query in itertools.groupby( + sorted(all_upstream_columns.items(), key=lambda x: x[1]), + key=lambda x: x[1], + ): + upstream_columns = [x[0] for x in upstream_columns_for_query] + required_queries.add(query_id) + + upstream_aspect.fineGrainedLineages.append( + models.FineGrainedLineageClass( + upstreamType=models.FineGrainedLineageUpstreamTypeClass.FIELD_SET, + upstreams=[ + upstream_column.urn() + for upstream_column in upstream_columns + ], + downstreamType=models.FineGrainedLineageDownstreamTypeClass.FIELD, + downstreams=[ + SchemaFieldUrn(downstream_urn, downstream_column).urn() + ], + query=( + self._query_urn(query_id) if self.generate_queries else None + ), + confidenceScore=queries_map[query_id].confidence_score, + ) + ) + + yield MetadataChangeProposalWrapper( + entityUrn=downstream_urn, + aspect=upstream_aspect, + ) + + if not self.generate_queries: + return + for query_id in required_queries: + # Avoid generating the same query twice. + if query_id in queries_generated: + continue + queries_generated.add(query_id) + self.report.num_queries_entities_generated += 1 + + query = queries_map[query_id] + yield from MetadataChangeProposalWrapper.construct_many( + entityUrn=self._query_urn(query_id), + aspects=[ + models.QueryPropertiesClass( + statement=models.QueryStatementClass( + value=query.formatted_query_string, + language=models.QueryLanguageClass.SQL, + ), + source=models.QuerySourceClass.SYSTEM, + created=query.make_created_audit_stamp(), + lastModified=query.make_last_modified_audit_stamp(), + ), + models.QuerySubjectsClass( + subjects=[ + models.QuerySubjectClass(entity=dataset_urn) + for dataset_urn in itertools.chain( + [downstream_urn], query.upstreams + ) + ] + ), + ], + ) + + @classmethod + def _query_urn(cls, query_id: QueryId) -> str: + return QueryUrn(query_id).urn() + + @classmethod + def _composite_query_id(cls, composed_of_queries: Iterable[QueryId]) -> str: + composed_of_queries = list(composed_of_queries) + combined = json.dumps(composed_of_queries) + return f"composite_{generate_hash(combined)}" + + @classmethod + def _view_query_id(cls, view_urn: UrnStr) -> str: + return f"view_{DatasetUrn.url_encode(view_urn)}" + + def _resolve_query_with_temp_tables( + self, + base_query: QueryMetadata, + ) -> QueryMetadata: + # TODO: Add special handling for COPY operations, which should mirror the schema + # of the thing being copied in order to generate CLL. + + session_id = base_query.session_id + + composed_of_queries = OrderedSet[QueryId]() + + @dataclasses.dataclass + class QueryLineageInfo: + upstreams: List[UrnStr] # this is direct upstreams, with *no temp tables* + column_lineage: List[ColumnLineageInfo] + confidence_score: float + + def _recurse_into_query( + query: QueryMetadata, recursion_path: List[QueryId] + ) -> QueryLineageInfo: + if query.query_id in recursion_path: + # This is a cycle, so we just return the query as-is. + return QueryLineageInfo( + upstreams=query.upstreams, + column_lineage=query.column_lineage, + confidence_score=query.confidence_score, + ) + recursion_path = [*recursion_path, query.query_id] + composed_of_queries.add(query.query_id) + + # Find all the temp tables that this query depends on. + temp_upstream_queries: Dict[UrnStr, QueryLineageInfo] = {} + for upstream in query.upstreams: + upstream_query_id = self._temp_lineage_map.get(session_id, {}).get( + upstream + ) + if upstream_query_id: + upstream_query = self._query_map.get(upstream_query_id) + if upstream_query: + temp_upstream_queries[upstream] = _recurse_into_query( + upstream_query, recursion_path + ) + + # Compute merged upstreams. + new_upstreams = OrderedSet[UrnStr]() + for upstream in query.upstreams: + if upstream in temp_upstream_queries: + new_upstreams.update(temp_upstream_queries[upstream].upstreams) + else: + new_upstreams.add(upstream) + + # Compute merged column lineage. + new_cll = [] + for lineage_info in query.column_lineage: + new_column_upstreams: List[ColumnRef] = [] + for existing_col_upstream in lineage_info.upstreams: + if existing_col_upstream.table in temp_upstream_queries: + new_column_upstreams.extend( + [ + temp_col_upstream + for temp_lineage_info in temp_upstream_queries[ + existing_col_upstream.table + ].column_lineage + for temp_col_upstream in temp_lineage_info.upstreams + if temp_lineage_info.downstream.column + == existing_col_upstream.column + ] + ) + else: + new_column_upstreams.append(existing_col_upstream) + + new_cll.append( + ColumnLineageInfo( + downstream=lineage_info.downstream, + upstreams=new_column_upstreams, + ) + ) + + # Compute merged confidence score. + new_confidence_score = min( + [ + query.confidence_score, + *[ + temp_upstream_query.confidence_score + for temp_upstream_query in temp_upstream_queries.values() + ], + ] + ) + + return QueryLineageInfo( + upstreams=list(new_upstreams), + column_lineage=new_cll, + confidence_score=new_confidence_score, + ) + + resolved_lineage_info = _recurse_into_query(base_query, []) + + # Fast path if there were no temp tables. + if len(composed_of_queries) == 1: + return base_query + + # If this query does actually depend on temp tables: + # - Clone the query into a new object + # - Generate a new composite fingerprint + # - Update the lineage info + # - Update the query text to combine the queries + + composite_query_id = self._composite_query_id(composed_of_queries) + self.report.queries_with_temp_upstreams.setdefault( + composite_query_id, LossyList() + ).extend(composed_of_queries) + + merged_query_text = ";\n\n".join( + [ + self._query_map[query_id].formatted_query_string + for query_id in reversed(composed_of_queries) + ] + ) + + resolved_query = dataclasses.replace( + base_query, + query_id=composite_query_id, + formatted_query_string=merged_query_text, + upstreams=resolved_lineage_info.upstreams, + column_lineage=resolved_lineage_info.column_lineage, + confidence_score=resolved_lineage_info.confidence_score, + ) + + return resolved_query + + def _gen_usage_statistics_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: + if not self._usage_aggregator: + return + + for wu in self._usage_aggregator.generate_workunits( + resource_urn_builder=lambda urn: urn, + user_urn_builder=lambda urn: urn, + ): + # TODO: We should change the usage aggregator to return MCPWs directly. + yield cast(MetadataChangeProposalWrapper, wu.metadata) + + def _gen_operation_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: + if not self.generate_operations: + return + + for downstream_urn, query_ids in self._lineage_map.items(): + for query_id in query_ids: + yield from self._gen_operation_for_downstream(downstream_urn, query_id) + + def _gen_operation_for_downstream( + self, downstream_urn: UrnStr, query_id: QueryId + ) -> Iterable[MetadataChangeProposalWrapper]: + query = self._query_map[query_id] + if query.latest_timestamp is None: + return + + operation_type = query.query_type.to_operation_type() + if operation_type is None: + # We don't generate operations for SELECTs. + return + + aspect = models.OperationClass( + timestampMillis=make_ts_millis(datetime.now(tz=timezone.utc)), + operationType=operation_type, + lastUpdatedTimestamp=make_ts_millis(query.latest_timestamp), + actor=query.actor.urn() if query.actor else None, + customProperties={ + "query_urn": self._query_urn(query_id), + }, + ) + yield MetadataChangeProposalWrapper(entityUrn=downstream_urn, aspect=aspect) diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_common.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_common.py new file mode 100644 index 0000000000000..a1b850b56677a --- /dev/null +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_common.py @@ -0,0 +1,71 @@ +import enum +from typing import Optional + +from typing_extensions import TypedDict + +from datahub.metadata.schema_classes import OperationTypeClass + +PLATFORMS_WITH_CASE_SENSITIVE_TABLES = { + "bigquery", +} + +DIALECTS_WITH_CASE_INSENSITIVE_COLS = { + # Column identifiers are case-insensitive in BigQuery, so we need to + # do a normalization step beforehand to make sure it's resolved correctly. + "bigquery", + # Our snowflake source lowercases column identifiers, so we are forced + # to do fuzzy (case-insensitive) resolution instead of exact resolution. + "snowflake", + # Teradata column names are case-insensitive. + # A name, even when enclosed in double quotation marks, is not case sensitive. For example, CUSTOMER and Customer are the same. + # See more below: + # https://documentation.sas.com/doc/en/pgmsascdc/9.4_3.5/acreldb/n0ejgx4895bofnn14rlguktfx5r3.htm + "teradata", +} +DIALECTS_WITH_DEFAULT_UPPERCASE_COLS = { + # In some dialects, column identifiers are effectively case insensitive + # because they are automatically converted to uppercase. Most other systems + # automatically lowercase unquoted identifiers. + "snowflake", +} + + +class QueryType(enum.Enum): + UNKNOWN = "UNKNOWN" + + CREATE_DDL = "CREATE_DDL" + CREATE_VIEW = "CREATE_VIEW" + CREATE_TABLE_AS_SELECT = "CREATE_TABLE_AS_SELECT" + CREATE_OTHER = "CREATE_OTHER" + + SELECT = "SELECT" + INSERT = "INSERT" + UPDATE = "UPDATE" + DELETE = "DELETE" + MERGE = "MERGE" + + def is_create(self) -> bool: + return self in { + QueryType.CREATE_DDL, + QueryType.CREATE_VIEW, + QueryType.CREATE_TABLE_AS_SELECT, + QueryType.CREATE_OTHER, + } + + def to_operation_type(self) -> Optional[str]: + if self.is_create(): + return OperationTypeClass.CREATE + + query_to_operation_mapping = { + QueryType.SELECT: None, + QueryType.INSERT: OperationTypeClass.INSERT, + QueryType.UPDATE: OperationTypeClass.UPDATE, + QueryType.DELETE: OperationTypeClass.DELETE, + QueryType.MERGE: OperationTypeClass.UPDATE, + } + return query_to_operation_mapping.get(self, OperationTypeClass.UNKNOWN) + + +class QueryTypeProps(TypedDict, total=False): + kind: str # used for create statements + temporary: bool diff --git a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_lineage.py similarity index 68% rename from metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py rename to metadata-ingestion/src/datahub/sql_parsing/sqlglot_lineage.py index 5b063451df9cf..a68a98ecb5cbe 100644 --- a/metadata-ingestion/src/datahub/utilities/sqlglot_lineage.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_lineage.py @@ -1,11 +1,8 @@ -import contextlib -import enum import functools import itertools import logging -import pathlib from collections import defaultdict -from typing import Any, Dict, Iterable, List, Optional, Set, Tuple, Union +from typing import Any, Dict, List, Optional, Set, Tuple, Union import pydantic.dataclasses import sqlglot @@ -14,38 +11,43 @@ import sqlglot.optimizer.annotate_types import sqlglot.optimizer.optimizer import sqlglot.optimizer.qualify -from pydantic import BaseModel -from typing_extensions import TypedDict -from datahub.configuration.pydantic_migration_helpers import PYDANTIC_VERSION_2 -from datahub.emitter.mce_builder import ( - DEFAULT_ENV, - make_dataset_urn_with_platform_instance, -) -from datahub.ingestion.api.closeable import Closeable from datahub.ingestion.graph.client import DataHubGraph -from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier from datahub.metadata.schema_classes import ( ArrayTypeClass, BooleanTypeClass, DateTypeClass, + NullTypeClass, NumberTypeClass, - OperationTypeClass, + SchemaFieldClass, SchemaFieldDataTypeClass, - SchemaMetadataClass, StringTypeClass, TimeTypeClass, ) -from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedDict -from datahub.utilities.urns.field_paths import get_simple_field_path_from_v2_field_path +from datahub.sql_parsing._models import _FrozenModel, _ParserBaseModel, _TableName +from datahub.sql_parsing.schema_resolver import ( + SchemaInfo, + SchemaResolver, + SchemaResolverInterface, +) +from datahub.sql_parsing.sql_parsing_common import ( + DIALECTS_WITH_CASE_INSENSITIVE_COLS, + DIALECTS_WITH_DEFAULT_UPPERCASE_COLS, + QueryType, + QueryTypeProps, +) +from datahub.sql_parsing.sqlglot_utils import ( + DialectOrStr, + get_dialect, + get_query_fingerprint, + is_dialect_instance, + parse_statement, +) logger = logging.getLogger(__name__) Urn = str -# A lightweight table schema: column -> type mapping. -SchemaInfo = Dict[str, str] - SQL_PARSE_RESULT_CACHE_SIZE = 1000 @@ -69,46 +71,51 @@ assert 0 < len(RULES_BEFORE_TYPE_ANNOTATION) < len(sqlglot.optimizer.optimizer.RULES) -class GraphQLSchemaField(TypedDict): - fieldPath: str - nativeDataType: str +def _is_temp_table(table: sqlglot.exp.Table, dialect: sqlglot.Dialect) -> bool: + identifier: sqlglot.exp.Identifier = table.this + return identifier.args.get("temporary") or ( + is_dialect_instance(dialect, "redshift") and identifier.name.startswith("#") + ) -class GraphQLSchemaMetadata(TypedDict): - fields: List[GraphQLSchemaField] +def _get_create_type_from_kind(kind: Optional[str]) -> QueryType: + if kind and "TABLE" in kind: + return QueryType.CREATE_TABLE_AS_SELECT + elif kind and "VIEW" in kind: + return QueryType.CREATE_VIEW + else: + return QueryType.CREATE_OTHER -class QueryType(enum.Enum): - CREATE = "CREATE" - SELECT = "SELECT" - INSERT = "INSERT" - UPDATE = "UPDATE" - DELETE = "DELETE" - MERGE = "MERGE" - UNKNOWN = "UNKNOWN" +def get_query_type_of_sql( + expression: sqlglot.exp.Expression, dialect: DialectOrStr +) -> Tuple[QueryType, QueryTypeProps]: + dialect = get_dialect(dialect) + query_type_props: QueryTypeProps = {} - def to_operation_type(self) -> Optional[str]: - if self == QueryType.CREATE: - return OperationTypeClass.CREATE - elif self == QueryType.INSERT: - return OperationTypeClass.INSERT - elif self == QueryType.UPDATE: - return OperationTypeClass.UPDATE - elif self == QueryType.DELETE: - return OperationTypeClass.DELETE - elif self == QueryType.MERGE: - return OperationTypeClass.UPDATE - elif self == QueryType.SELECT: - return None - else: - return OperationTypeClass.UNKNOWN + # For creates, we need to look at the inner expression. + if isinstance(expression, sqlglot.exp.Create): + if _is_create_table_ddl(expression): + return QueryType.CREATE_DDL, query_type_props + + kind = expression.args.get("kind") + if kind: + kind = kind.upper() + query_type_props["kind"] = kind + target = expression.this + if any( + isinstance(prop, sqlglot.exp.TemporaryProperty) + for prop in (expression.args.get("properties") or []) + ) or _is_temp_table(target, dialect=dialect): + query_type_props["temporary"] = True + + query_type = _get_create_type_from_kind(kind) + return query_type, query_type_props -def get_query_type_of_sql(expression: sqlglot.exp.Expression) -> QueryType: # UPGRADE: Once we use Python 3.10, replace this with a match expression. mapping = { - sqlglot.exp.Create: QueryType.CREATE, sqlglot.exp.Select: QueryType.SELECT, sqlglot.exp.Insert: QueryType.INSERT, sqlglot.exp.Update: QueryType.UPDATE, @@ -119,78 +126,8 @@ def get_query_type_of_sql(expression: sqlglot.exp.Expression) -> QueryType: for cls, query_type in mapping.items(): if isinstance(expression, cls): - return query_type - return QueryType.UNKNOWN - - -class _ParserBaseModel( - BaseModel, - arbitrary_types_allowed=True, - json_encoders={ - SchemaFieldDataTypeClass: lambda v: v.to_obj(), - }, -): - def json(self, *args: Any, **kwargs: Any) -> str: - if PYDANTIC_VERSION_2: - return super().model_dump_json(*args, **kwargs) # type: ignore - else: - return super().json(*args, **kwargs) - - -@functools.total_ordering -class _FrozenModel(_ParserBaseModel, frozen=True): - def __lt__(self, other: "_FrozenModel") -> bool: - # TODO: The __fields__ attribute is deprecated in Pydantic v2. - for field in self.__fields__: - self_v = getattr(self, field) - other_v = getattr(other, field) - if self_v != other_v: - return self_v < other_v - - return False - - -class _TableName(_FrozenModel): - database: Optional[str] = None - db_schema: Optional[str] = None - table: str - - def as_sqlglot_table(self) -> sqlglot.exp.Table: - return sqlglot.exp.Table( - catalog=sqlglot.exp.Identifier(this=self.database) - if self.database - else None, - db=sqlglot.exp.Identifier(this=self.db_schema) if self.db_schema else None, - this=sqlglot.exp.Identifier(this=self.table), - ) - - def qualified( - self, - dialect: sqlglot.Dialect, - default_db: Optional[str] = None, - default_schema: Optional[str] = None, - ) -> "_TableName": - database = self.database or default_db - db_schema = self.db_schema or default_schema - - return _TableName( - database=database, - db_schema=db_schema, - table=self.table, - ) - - @classmethod - def from_sqlglot_table( - cls, - table: sqlglot.exp.Table, - default_db: Optional[str] = None, - default_schema: Optional[str] = None, - ) -> "_TableName": - return cls( - database=table.catalog or default_db, - db_schema=table.db or default_schema, - table=table.this.name, - ) + return query_type, query_type_props + return QueryType.UNKNOWN, {} class _ColumnRef(_FrozenModel): @@ -257,6 +194,8 @@ def error(self) -> Optional[Exception]: class SqlParsingResult(_ParserBaseModel): query_type: QueryType = QueryType.UNKNOWN + query_type_props: QueryTypeProps = {} + query_fingerprint: Optional[str] = None in_tables: List[Urn] out_tables: List[Urn] @@ -282,15 +221,6 @@ def make_from_error(cls, error: Exception) -> "SqlParsingResult": ) -def _parse_statement( - sql: sqlglot.exp.ExpOrStr, dialect: sqlglot.Dialect -) -> sqlglot.Expression: - statement: sqlglot.Expression = sqlglot.maybe_parse( - sql, dialect=dialect, error_level=sqlglot.ErrorLevel.RAISE - ) - return statement - - def _table_level_lineage( statement: sqlglot.Expression, dialect: sqlglot.Dialect ) -> Tuple[Set[_TableName], Set[_TableName]]: @@ -308,10 +238,14 @@ def _table_level_lineage( # 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. + # For statements that include a column list, like + # CREATE DDL statements and `INSERT INTO table (col1, col2) SELECT ...` + # 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) + for expr in statement.find_all( + sqlglot.exp.Create, + sqlglot.exp.Insert, + ) if isinstance(expr.this, sqlglot.exp.Schema) and isinstance(expr.this.this, sqlglot.exp.Table) } @@ -338,158 +272,6 @@ def _table_level_lineage( return tables, modified -TABLE_CASE_SENSITIVE_PLATFORMS = {"bigquery"} - - -class SchemaResolver(Closeable): - def __init__( - self, - *, - platform: str, - platform_instance: Optional[str] = None, - env: str = DEFAULT_ENV, - graph: Optional[DataHubGraph] = None, - _cache_filename: Optional[pathlib.Path] = None, - ): - # TODO handle platforms when prefixed with urn:li:dataPlatform: - self.platform = platform - self.platform_instance = platform_instance - self.env = env - - self.graph = graph - - # Init cache, potentially restoring from a previous run. - shared_conn = None - if _cache_filename: - shared_conn = ConnectionWrapper(filename=_cache_filename) - self._schema_cache: FileBackedDict[Optional[SchemaInfo]] = FileBackedDict( - shared_connection=shared_conn, - ) - - def get_urns(self) -> Set[str]: - return set(self._schema_cache.keys()) - - def get_urn_for_table(self, table: _TableName, lower: bool = False) -> str: - # TODO: Validate that this is the correct 2/3 layer hierarchy for the platform. - - table_name = ".".join( - filter(None, [table.database, table.db_schema, table.table]) - ) - - platform_instance = self.platform_instance - - if lower: - table_name = table_name.lower() - platform_instance = platform_instance.lower() if platform_instance else None - - if self.platform == "bigquery": - # Normalize shard numbers and other BigQuery weirdness. - with contextlib.suppress(IndexError): - table_name = BigqueryTableIdentifier.from_string_name( - table_name - ).get_table_name() - - urn = make_dataset_urn_with_platform_instance( - platform=self.platform, - platform_instance=platform_instance, - env=self.env, - name=table_name, - ) - return urn - - def resolve_table(self, table: _TableName) -> Tuple[str, Optional[SchemaInfo]]: - urn = self.get_urn_for_table(table) - - schema_info = self._resolve_schema_info(urn) - if schema_info: - return urn, schema_info - - urn_lower = self.get_urn_for_table(table, lower=True) - if urn_lower != urn: - schema_info = self._resolve_schema_info(urn_lower) - if schema_info: - return urn_lower, schema_info - - if self.platform in TABLE_CASE_SENSITIVE_PLATFORMS: - return urn, None - else: - return urn_lower, None - - def _resolve_schema_info(self, urn: str) -> Optional[SchemaInfo]: - if urn in self._schema_cache: - return self._schema_cache[urn] - - # TODO: For bigquery partitioned tables, add the pseudo-column _PARTITIONTIME - # or _PARTITIONDATE where appropriate. - - if self.graph: - schema_info = self._fetch_schema_info(self.graph, urn) - if schema_info: - self._save_to_cache(urn, schema_info) - return schema_info - - self._save_to_cache(urn, None) - return None - - def add_schema_metadata( - self, urn: str, schema_metadata: SchemaMetadataClass - ) -> None: - schema_info = self._convert_schema_aspect_to_info(schema_metadata) - self._save_to_cache(urn, schema_info) - - def add_raw_schema_info(self, urn: str, schema_info: SchemaInfo) -> None: - self._save_to_cache(urn, schema_info) - - def add_graphql_schema_metadata( - self, urn: str, schema_metadata: GraphQLSchemaMetadata - ) -> None: - schema_info = self.convert_graphql_schema_metadata_to_info(schema_metadata) - self._save_to_cache(urn, schema_info) - - def _save_to_cache(self, urn: str, schema_info: Optional[SchemaInfo]) -> None: - self._schema_cache[urn] = schema_info - - def _fetch_schema_info(self, graph: DataHubGraph, urn: str) -> Optional[SchemaInfo]: - aspect = graph.get_aspect(urn, SchemaMetadataClass) - if not aspect: - return None - - return self._convert_schema_aspect_to_info(aspect) - - @classmethod - def _convert_schema_aspect_to_info( - cls, schema_metadata: SchemaMetadataClass - ) -> SchemaInfo: - return { - get_simple_field_path_from_v2_field_path(col.fieldPath): ( - # The actual types are more of a "nice to have". - col.nativeDataType - or "str" - ) - for col in schema_metadata.fields - # TODO: We can't generate lineage to columns nested within structs yet. - if "." not in get_simple_field_path_from_v2_field_path(col.fieldPath) - } - - @classmethod - def convert_graphql_schema_metadata_to_info( - cls, schema: GraphQLSchemaMetadata - ) -> SchemaInfo: - return { - get_simple_field_path_from_v2_field_path(field["fieldPath"]): ( - # The actual types are more of a "nice to have". - field["nativeDataType"] - or "str" - ) - for field in schema["fields"] - # TODO: We can't generate lineage to columns nested within structs yet. - if "." not in get_simple_field_path_from_v2_field_path(field["fieldPath"]) - } - - def close(self) -> None: - self._schema_cache.close() - - # TODO: Once PEP 604 is supported (Python 3.10), we can unify these into a # single type. See https://peps.python.org/pep-0604/#isinstance-and-issubclass. _SupportedColumnLineageTypes = Union[ @@ -500,26 +282,6 @@ def close(self) -> None: ] _SupportedColumnLineageTypesTuple = (sqlglot.exp.Subqueryable, sqlglot.exp.DerivedTable) -DIALECTS_WITH_CASE_INSENSITIVE_COLS = { - # Column identifiers are case-insensitive in BigQuery, so we need to - # do a normalization step beforehand to make sure it's resolved correctly. - "bigquery", - # Our snowflake source lowercases column identifiers, so we are forced - # to do fuzzy (case-insensitive) resolution instead of exact resolution. - "snowflake", - # Teradata column names are case-insensitive. - # A name, even when enclosed in double quotation marks, is not case sensitive. For example, CUSTOMER and Customer are the same. - # See more below: - # https://documentation.sas.com/doc/en/pgmsascdc/9.4_3.5/acreldb/n0ejgx4895bofnn14rlguktfx5r3.htm - "teradata", -} -DIALECTS_WITH_DEFAULT_UPPERCASE_COLS = { - # In some dialects, column identifiers are effectively case insensitive - # because they are automatically converted to uppercase. Most other systems - # automatically lowercase unquoted identifiers. - "snowflake", -} - class UnsupportedStatementTypeError(TypeError): pass @@ -553,7 +315,7 @@ def _column_level_lineage( # noqa: C901 column_lineage: List[_ColumnLineageInfo] = [] - use_case_insensitive_cols = _is_dialect_instance( + use_case_insensitive_cols = is_dialect_instance( dialect, DIALECTS_WITH_CASE_INSENSITIVE_COLS ) @@ -572,7 +334,7 @@ def _column_level_lineage( # noqa: C901 col_normalized = ( # This is required to match Sqlglot's behavior. col.upper() - if _is_dialect_instance( + if is_dialect_instance( dialect, DIALECTS_WITH_DEFAULT_UPPERCASE_COLS ) else col.lower() @@ -707,7 +469,7 @@ def _schema_aware_fuzzy_column_resolve( # Otherwise, we can't process it. continue - if _is_dialect_instance(dialect, "bigquery") and output_col.lower() in { + if is_dialect_instance(dialect, "bigquery") and output_col.lower() in { "_partitiontime", "_partitiondate", }: @@ -962,18 +724,18 @@ def _translate_internal_column_lineage( downstream=DownstreamColumnRef( table=downstream_urn, column=raw_column_lineage.downstream.column, - column_type=_translate_sqlglot_type( - raw_column_lineage.downstream.column_type.this - ) - if raw_column_lineage.downstream.column_type - else None, - native_column_type=raw_column_lineage.downstream.column_type.sql( - dialect=dialect - ) - if raw_column_lineage.downstream.column_type - and raw_column_lineage.downstream.column_type.this - != sqlglot.exp.DataType.Type.UNKNOWN - else None, + column_type=( + _translate_sqlglot_type(raw_column_lineage.downstream.column_type.this) + if raw_column_lineage.downstream.column_type + else None + ), + native_column_type=( + raw_column_lineage.downstream.column_type.sql(dialect=dialect) + if raw_column_lineage.downstream.column_type + and raw_column_lineage.downstream.column_type.this + != sqlglot.exp.DataType.Type.UNKNOWN + else None + ), ), upstreams=[ ColumnRef( @@ -986,58 +748,20 @@ def _translate_internal_column_lineage( ) -def _get_dialect_str(platform: str) -> str: - # TODO: convert datahub platform names to sqlglot dialect - if platform == "presto-on-hive": - return "hive" - elif platform == "mssql": - return "tsql" - elif platform == "athena": - return "trino" - elif platform == "mysql": - # In sqlglot v20+, MySQL is now case-sensitive by default, which is the - # default behavior on Linux. However, MySQL's default case sensitivity - # actually depends on the underlying OS. - # For us, it's simpler to just assume that it's case-insensitive, and - # let the fuzzy resolution logic handle it. - return "mysql, normalization_strategy = lowercase" - else: - return platform - - -def _get_dialect(platform: str) -> sqlglot.Dialect: - return sqlglot.Dialect.get_or_raise(_get_dialect_str(platform)) - - -def _is_dialect_instance( - dialect: sqlglot.Dialect, platforms: Union[str, Iterable[str]] -) -> bool: - if isinstance(platforms, str): - platforms = [platforms] - else: - platforms = list(platforms) - - dialects = [sqlglot.Dialect.get_or_raise(platform) for platform in platforms] - - if any(isinstance(dialect, dialect_class.__class__) for dialect_class in dialects): - return True - return False - - def _sqlglot_lineage_inner( sql: sqlglot.exp.ExpOrStr, - schema_resolver: SchemaResolver, + schema_resolver: SchemaResolverInterface, default_db: Optional[str] = None, default_schema: Optional[str] = None, ) -> SqlParsingResult: - dialect = _get_dialect(schema_resolver.platform) - if _is_dialect_instance(dialect, "snowflake"): + dialect = get_dialect(schema_resolver.platform) + if is_dialect_instance(dialect, "snowflake"): # in snowflake, table identifiers must be uppercased to match sqlglot's behavior. if default_db: default_db = default_db.upper() if default_schema: default_schema = default_schema.upper() - if _is_dialect_instance(dialect, "redshift") and not default_schema: + if is_dialect_instance(dialect, "redshift") and not default_schema: # On Redshift, there's no "USE SCHEMA " command. The default schema # is public, and "current schema" is the one at the front of the search path. # See https://docs.aws.amazon.com/redshift/latest/dg/r_search_path.html @@ -1047,7 +771,7 @@ def _sqlglot_lineage_inner( pass logger.debug("Parsing lineage from sql statement: %s", sql) - statement = _parse_statement(sql, dialect=dialect) + statement = parse_statement(sql, dialect=dialect) original_statement = statement.copy() # logger.debug( @@ -1102,10 +826,13 @@ def _sqlglot_lineage_inner( total_tables_discovered = len(tables | modified) total_schemas_resolved = len(table_name_schema_mapping) debug_info = SqlParsingDebugInfo( - confidence=0.9 if total_tables_discovered == total_schemas_resolved - # If we're missing any schema info, our confidence will be in the 0.2-0.5 range depending - # on how many tables we were able to resolve. - else 0.2 + 0.3 * total_schemas_resolved / total_tables_discovered, + confidence=( + 0.9 + if total_tables_discovered == total_schemas_resolved + # If we're missing any schema info, our confidence will be in the 0.2-0.5 range depending + # on how many tables we were able to resolve. + else 0.2 + 0.3 * total_schemas_resolved / total_tables_discovered + ), tables_discovered=total_tables_discovered, table_schemas_resolved=total_schemas_resolved, ) @@ -1157,8 +884,14 @@ def _sqlglot_lineage_inner( for internal_col_lineage in column_lineage ] + query_type, query_type_props = get_query_type_of_sql( + original_statement, dialect=dialect + ) + query_fingerprint = get_query_fingerprint(original_statement, dialect=dialect) return SqlParsingResult( - query_type=get_query_type_of_sql(original_statement), + query_type=query_type, + query_type_props=query_type_props, + query_fingerprint=query_fingerprint, in_tables=in_urns, out_tables=out_urns, column_lineage=column_lineage_urns, @@ -1169,7 +902,7 @@ def _sqlglot_lineage_inner( @functools.lru_cache(maxsize=SQL_PARSE_RESULT_CACHE_SIZE) def sqlglot_lineage( sql: str, - schema_resolver: SchemaResolver, + schema_resolver: SchemaResolverInterface, default_db: Optional[str] = None, default_schema: Optional[str] = None, ) -> SqlParsingResult: @@ -1233,59 +966,6 @@ def sqlglot_lineage( return SqlParsingResult.make_from_error(e) -def detach_ctes( - sql: sqlglot.exp.ExpOrStr, platform: str, cte_mapping: Dict[str, str] -) -> sqlglot.exp.Expression: - """Replace CTE references with table references. - - For example, with cte_mapping = {"__cte_0": "_my_cte_table"}, the following SQL - - WITH __cte_0 AS (SELECT * FROM table1) SELECT * FROM table2 JOIN __cte_0 ON table2.id = __cte_0.id - - is transformed into - - WITH __cte_0 AS (SELECT * FROM table1) SELECT * FROM table2 JOIN _my_cte_table ON table2.id = _my_cte_table.id - - Note that the original __cte_0 definition remains in the query, but is simply not referenced. - The query optimizer should be able to remove it. - - This method makes a major assumption: that no other table/column has the same name as a - key in the cte_mapping. - """ - - dialect = _get_dialect(platform) - statement = _parse_statement(sql, dialect=dialect) - - def replace_cte_refs(node: sqlglot.exp.Expression) -> sqlglot.exp.Expression: - if ( - isinstance(node, sqlglot.exp.Identifier) - and node.parent - and not isinstance(node.parent.parent, sqlglot.exp.CTE) - and node.name in cte_mapping - ): - full_new_name = cte_mapping[node.name] - table_expr = sqlglot.maybe_parse( - full_new_name, dialect=dialect, into=sqlglot.exp.Table - ) - - parent = node.parent - - # We expect node.parent to be a Table or Column, both of which support catalog/db/name. - # However, we check the parent's arg_types to be safe. - if "catalog" in parent.arg_types and table_expr.catalog: - parent.set("catalog", table_expr.catalog) - if "db" in parent.arg_types and table_expr.db: - parent.set("db", table_expr.db) - - new_node = sqlglot.exp.Identifier(this=table_expr.name) - - return new_node - else: - return node - - return statement.transform(replace_cte_refs, copy=False) - - def create_lineage_sql_parsed_result( query: str, default_db: Optional[str], @@ -1325,6 +1005,25 @@ def create_lineage_sql_parsed_result( schema_resolver.close() +def infer_output_schema(result: SqlParsingResult) -> Optional[List[SchemaFieldClass]]: + if result.column_lineage is None: + return None + + output_schema = [] + for column_info in result.column_lineage: + output_schema.append( + SchemaFieldClass( + fieldPath=column_info.downstream.column, + type=( + column_info.downstream.column_type + or SchemaFieldDataTypeClass(type=NullTypeClass()) + ), + nativeDataType=column_info.downstream.native_column_type or "", + ) + ) + return output_schema + + def view_definition_lineage_helper( result: SqlParsingResult, view_urn: str ) -> SqlParsingResult: diff --git a/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py new file mode 100644 index 0000000000000..587394cc14646 --- /dev/null +++ b/metadata-ingestion/src/datahub/sql_parsing/sqlglot_utils.py @@ -0,0 +1,199 @@ +import hashlib +from typing import Dict, Iterable, Optional, Union + +import sqlglot + +DialectOrStr = Union[sqlglot.Dialect, str] + + +def _get_dialect_str(platform: str) -> str: + if platform == "presto-on-hive": + return "hive" + elif platform == "mssql": + return "tsql" + elif platform == "athena": + return "trino" + elif platform == "mysql": + # In sqlglot v20+, MySQL is now case-sensitive by default, which is the + # default behavior on Linux. However, MySQL's default case sensitivity + # actually depends on the underlying OS. + # For us, it's simpler to just assume that it's case-insensitive, and + # let the fuzzy resolution logic handle it. + return "mysql, normalization_strategy = lowercase" + else: + return platform + + +def get_dialect(platform: DialectOrStr) -> sqlglot.Dialect: + if isinstance(platform, sqlglot.Dialect): + return platform + return sqlglot.Dialect.get_or_raise(_get_dialect_str(platform)) + + +def is_dialect_instance( + dialect: sqlglot.Dialect, platforms: Union[str, Iterable[str]] +) -> bool: + if isinstance(platforms, str): + platforms = [platforms] + else: + platforms = list(platforms) + + dialects = [sqlglot.Dialect.get_or_raise(platform) for platform in platforms] + + if any(isinstance(dialect, dialect_class.__class__) for dialect_class in dialects): + return True + return False + + +def parse_statement( + sql: sqlglot.exp.ExpOrStr, dialect: sqlglot.Dialect +) -> sqlglot.Expression: + statement: sqlglot.Expression = sqlglot.maybe_parse( + sql, dialect=dialect, error_level=sqlglot.ErrorLevel.RAISE + ) + return statement + + +def generalize_query(expression: sqlglot.exp.ExpOrStr, dialect: DialectOrStr) -> str: + """ + Generalize/normalize a SQL query. + + The generalized query will strip comments and normalize things like + whitespace and keyword casing. It will also replace things like date + literals with placeholders so that the generalized query can be used + for query fingerprinting. + + Args: + expression: The SQL query to generalize. + dialect: The SQL dialect to use. + + Returns: + The generalized SQL query. + """ + + # Similar to sql-metadata's query normalization. + # Implementation: https://github.com/macbre/sql-metadata/blob/master/sql_metadata/generalizator.py + # Tests: https://github.com/macbre/sql-metadata/blob/master/test/test_normalization.py + # + # Note that this is somewhat different from sqlglot's normalization + # https://tobikodata.com/are_these_sql_queries_the_same.html + # which is used to determine if queries are functionally equivalent. + + dialect = get_dialect(dialect) + expression = sqlglot.maybe_parse(expression, dialect=dialect) + + def _simplify_node_expressions(node: sqlglot.exp.Expression) -> None: + # Replace all literals in the expressions with a single placeholder. + is_last_literal = True + for i, expression in reversed(list(enumerate(node.expressions))): + if isinstance(expression, sqlglot.exp.Literal): + if is_last_literal: + node.expressions[i] = sqlglot.exp.Placeholder() + is_last_literal = False + else: + node.expressions.pop(i) + + elif isinstance(expression, sqlglot.exp.Tuple): + _simplify_node_expressions(expression) + + def _strip_expression( + node: sqlglot.exp.Expression, + ) -> Optional[sqlglot.exp.Expression]: + node.comments = None + + if isinstance(node, (sqlglot.exp.In, sqlglot.exp.Values)): + _simplify_node_expressions(node) + elif isinstance(node, sqlglot.exp.Literal): + return sqlglot.exp.Placeholder() + + return node + + return expression.transform(_strip_expression, copy=True).sql(dialect=dialect) + + +def generate_hash(text: str) -> str: + # Once we move to Python 3.9+, we can set `usedforsecurity=False`. + return hashlib.sha256(text.encode("utf-8")).hexdigest() + + +def get_query_fingerprint( + expression: sqlglot.exp.ExpOrStr, dialect: DialectOrStr +) -> str: + """Get a fingerprint for a SQL query. + + The fingerprint is a SHA-256 hash of the generalized query. + + If two queries have the same fingerprint, they should have the same table + and column lineage. The only exception is if the query uses an indirection + function like Snowflake's `IDENTIFIER`. + + Queries that are functionally equivalent equivalent may not have the same + same fingerprint. For example, `SELECT 1+1` and `SELECT 2` have different + fingerprints. + + Args: + expression: The SQL query to fingerprint. + dialect: The SQL dialect to use. + + Returns: + The fingerprint for the SQL query. + """ + + dialect = get_dialect(dialect) + expression_sql = generalize_query(expression, dialect=dialect) + fingerprint = generate_hash(expression_sql) + + return fingerprint + + +def detach_ctes( + sql: sqlglot.exp.ExpOrStr, platform: str, cte_mapping: Dict[str, str] +) -> sqlglot.exp.Expression: + """Replace CTE references with table references. + + For example, with cte_mapping = {"__cte_0": "_my_cte_table"}, the following SQL + + WITH __cte_0 AS (SELECT * FROM table1) SELECT * FROM table2 JOIN __cte_0 ON table2.id = __cte_0.id + + is transformed into + + WITH __cte_0 AS (SELECT * FROM table1) SELECT * FROM table2 JOIN _my_cte_table ON table2.id = _my_cte_table.id + + Note that the original __cte_0 definition remains in the query, but is simply not referenced. + The query optimizer should be able to remove it. + + This method makes a major assumption: that no other table/column has the same name as a + key in the cte_mapping. + """ + + dialect = get_dialect(platform) + statement = parse_statement(sql, dialect=dialect) + + def replace_cte_refs(node: sqlglot.exp.Expression) -> sqlglot.exp.Expression: + if ( + isinstance(node, sqlglot.exp.Identifier) + and node.parent + and not isinstance(node.parent.parent, sqlglot.exp.CTE) + and node.name in cte_mapping + ): + full_new_name = cte_mapping[node.name] + table_expr = sqlglot.maybe_parse( + full_new_name, dialect=dialect, into=sqlglot.exp.Table + ) + + parent = node.parent + + # We expect node.parent to be a Table or Column, both of which support catalog/db/name. + # However, we check the parent's arg_types to be safe. + if "catalog" in parent.arg_types and table_expr.catalog: + parent.set("catalog", table_expr.catalog) + if "db" in parent.arg_types and table_expr.db: + parent.set("db", table_expr.db) + + new_node = sqlglot.exp.Identifier(this=table_expr.name) + + return new_node + else: + return node + + return statement.transform(replace_cte_refs, copy=False) diff --git a/metadata-ingestion/src/datahub/testing/check_sql_parser_result.py b/metadata-ingestion/src/datahub/testing/check_sql_parser_result.py index 2b610947e9043..39c0dddd31400 100644 --- a/metadata-ingestion/src/datahub/testing/check_sql_parser_result.py +++ b/metadata-ingestion/src/datahub/testing/check_sql_parser_result.py @@ -6,9 +6,9 @@ import deepdiff from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier -from datahub.utilities.sqlglot_lineage import ( +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import ( SchemaInfo, - SchemaResolver, SqlParsingResult, sqlglot_lineage, ) diff --git a/metadata-ingestion/src/datahub/utilities/file_backed_collections.py b/metadata-ingestion/src/datahub/utilities/file_backed_collections.py index 18493edded4b7..05c2041519858 100644 --- a/metadata-ingestion/src/datahub/utilities/file_backed_collections.py +++ b/metadata-ingestion/src/datahub/utilities/file_backed_collections.py @@ -8,6 +8,7 @@ import tempfile from dataclasses import dataclass, field from datetime import datetime +from enum import Enum from types import TracebackType from typing import ( Any, @@ -26,6 +27,8 @@ Union, ) +from typing_extensions import Final + from datahub.ingestion.api.closeable import Closeable logger: logging.Logger = logging.getLogger(__name__) @@ -42,6 +45,16 @@ _VT = TypeVar("_VT") +class Unset(Enum): + token = 0 + + +# It's pretty annoying to create a true sentinel that works with typing. +# https://peps.python.org/pep-0484/#support-for-singleton-types-in-unions +# Can't wait for https://peps.python.org/pep-0661/ +_unset: Final = Unset.token + + class ConnectionWrapper: """ Wraps a SQlite connection, allowing connection reuse across multiple FileBacked* objects. @@ -58,9 +71,11 @@ class ConnectionWrapper: filename: pathlib.Path _temp_directory: Optional[str] + _dependent_objects: List[Union["FileBackedList", "FileBackedDict"]] def __init__(self, filename: Optional[pathlib.Path] = None): self._temp_directory = None + self._dependent_objects = [] # Warning: If filename is provided, the file will not be automatically cleaned up. if not filename: @@ -100,6 +115,8 @@ def executemany( return self.conn.executemany(sql, parameters) def close(self) -> None: + for obj in self._dependent_objects: + obj.close() self.conn.close() if self._temp_directory: shutil.rmtree(self._temp_directory) @@ -187,6 +204,7 @@ def __post_init__(self) -> None: if self.shared_connection: self._conn = self.shared_connection + self.shared_connection._dependent_objects.append(self) else: self._conn = ConnectionWrapper() @@ -227,10 +245,14 @@ def create_indexes(self) -> None: def _add_to_cache(self, key: str, value: _VT, dirty: bool) -> None: self._active_object_cache[key] = value, dirty - if len(self._active_object_cache) > self.cache_max_size: + if self.cache_max_size == 0: + self._prune_cache(len(self._active_object_cache)) + elif len(self._active_object_cache) > self.cache_max_size: # Try to prune in batches rather than one at a time. + # However, we don't want to prune the thing we just added, + # in case there's a mark_dirty() call immediately after. num_items_to_prune = min( - len(self._active_object_cache), self.cache_eviction_batch_size + len(self._active_object_cache) - 1, self.cache_eviction_batch_size ) self._prune_cache(num_items_to_prune) @@ -277,6 +299,27 @@ def __getitem__(self, key: str) -> _VT: def __setitem__(self, key: str, value: _VT) -> None: self._add_to_cache(key, value, True) + def for_mutation( + self, + /, + key: str, + default: Union[_VT, Unset] = _unset, + ) -> _VT: + # If key is in the dictionary, this is similar to __getitem__ + mark_dirty. + # If key is not in the dictionary, this is similar to __setitem__. + assert self.cache_max_size > 0, "Cache must be enabled to use getsetdefault" + + try: + value = self[key] + self.mark_dirty(key) + return value + except KeyError: + if default is _unset: + raise + + self[key] = default + return default + def __delitem__(self, key: str) -> None: in_cache = False if key in self._active_object_cache: @@ -290,7 +333,13 @@ def __delitem__(self, key: str) -> None: raise KeyError(key) def mark_dirty(self, key: str) -> None: - if key in self._active_object_cache and not self._active_object_cache[key][1]: + if key not in self._active_object_cache: + raise ValueError( + f"key {key} not in active object cache, which means any dirty value " + "is already persisted or lost" + ) + + if not self._active_object_cache[key][1]: self._active_object_cache[key] = self._active_object_cache[key][0], True def __iter__(self) -> Iterator[str]: @@ -395,7 +444,7 @@ class FileBackedList(Generic[_VT]): def __init__( self, - connection: Optional[ConnectionWrapper] = None, + shared_connection: Optional[ConnectionWrapper] = None, tablename: str = _DEFAULT_TABLE_NAME, serializer: Callable[[_VT], SqliteValue] = _default_serializer, deserializer: Callable[[Any], _VT] = _default_deserializer, @@ -405,10 +454,10 @@ def __init__( ) -> None: self._len = 0 self._dict = FileBackedDict[_VT]( - shared_connection=connection, + shared_connection=shared_connection, + tablename=tablename, serializer=serializer, deserializer=deserializer, - tablename=tablename, extra_columns=extra_columns or {}, cache_max_size=cache_max_size or _DEFAULT_MEMORY_CACHE_MAX_SIZE, cache_eviction_batch_size=cache_eviction_batch_size diff --git a/metadata-ingestion/src/datahub/utilities/ordered_set.py b/metadata-ingestion/src/datahub/utilities/ordered_set.py new file mode 100644 index 0000000000000..249e6e1c89f4a --- /dev/null +++ b/metadata-ingestion/src/datahub/utilities/ordered_set.py @@ -0,0 +1,41 @@ +from typing import Dict, Generic, Iterable, Iterator, MutableSet, Optional, TypeVar + +T = TypeVar("T") + + +class OrderedSet(MutableSet[T], Generic[T]): + """Ordered set implementation. + + This is a fairly naive implementation - it uses a dict to store the items, and ignores the dict values. + """ + + def __init__(self, iterable: Optional[Iterable[T]] = None) -> None: + self._data: Dict[T, None] = {} + if iterable: + for item in iterable: + self.add(item) + + def add(self, item: T) -> None: + self._data[item] = None + + def discard(self, item: T) -> None: + self._data.pop(item, None) + + def update(self, items: Iterable[T]) -> None: + for item in items: + self.add(item) + + def __contains__(self, item: object) -> bool: + return item in self._data + + def __len__(self) -> int: + return len(self._data) + + def __repr__(self) -> str: + return f"OrderedSet({list(self._data)})" + + def __iter__(self) -> Iterator[T]: + return iter(self._data) + + def __reversed__(self) -> Iterator[T]: + return reversed(list(self._data)) diff --git a/metadata-ingestion/tests/integration/git/test_git_clone.py b/metadata-ingestion/tests/integration/git/test_git_clone.py index 2428a6dfb1c9e..cf1f649825e0c 100644 --- a/metadata-ingestion/tests/integration/git/test_git_clone.py +++ b/metadata-ingestion/tests/integration/git/test_git_clone.py @@ -85,10 +85,12 @@ def test_github_branch(): def test_sanitize_repo_url(): import datahub.ingestion.source.git.git_import - assert doctest.testmod(datahub.ingestion.source.git.git_import) == ( - 0, - 3, - ) # 0 failures, 3 tests + assert ( + doctest.testmod( + datahub.ingestion.source.git.git_import, raise_on_error=True + ).attempted + == 3 + ) def test_git_clone_public(tmp_path): diff --git a/metadata-ingestion/tests/integration/tableau/test_tableau_ingest.py b/metadata-ingestion/tests/integration/tableau/test_tableau_ingest.py index 474228e9c9fc4..6deb4e84751e1 100644 --- a/metadata-ingestion/tests/integration/tableau/test_tableau_ingest.py +++ b/metadata-ingestion/tests/integration/tableau/test_tableau_ingest.py @@ -27,7 +27,7 @@ UpstreamLineage, ) from datahub.metadata.schema_classes import MetadataChangeProposalClass, UpstreamClass -from datahub.utilities.sqlglot_lineage import SqlParsingResult +from datahub.sql_parsing.sqlglot_lineage import SqlParsingResult from tests.test_helpers import mce_helpers, test_connection_helpers from tests.test_helpers.state_helpers import ( get_current_checkpoint_from_pipeline, diff --git a/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py b/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py index 9bbe9c45887a8..124db588fe08a 100644 --- a/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py +++ b/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py @@ -13,8 +13,8 @@ ) from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor +from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.utilities.perf_timer import PerfTimer -from datahub.utilities.sqlglot_lineage import SchemaResolver from tests.performance.bigquery.bigquery_events import generate_events, ref_from_table from tests.performance.data_generation import ( NormalDistribution, diff --git a/metadata-ingestion/tests/test_helpers/mce_helpers.py b/metadata-ingestion/tests/test_helpers/mce_helpers.py index 46bf5cbea6edf..1445f402f78b9 100644 --- a/metadata-ingestion/tests/test_helpers/mce_helpers.py +++ b/metadata-ingestion/tests/test_helpers/mce_helpers.py @@ -1,7 +1,9 @@ import json import logging import os +import pathlib import re +import tempfile from typing import ( Any, Callable, @@ -16,6 +18,7 @@ ) from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.ingestion.sink.file import write_metadata_file from datahub.metadata.schema_classes import MetadataChangeEventClass from datahub.testing.compare_metadata_json import ( assert_metadata_files_equal, @@ -90,6 +93,23 @@ def check_golden_file( ) +def check_goldens_stream( + pytestconfig: PytestConfig, + outputs: List, + golden_path: Union[str, os.PathLike], + ignore_paths: Sequence[str] = (), +) -> None: + with tempfile.NamedTemporaryFile() as f: + write_metadata_file(pathlib.Path(f.name), outputs) + + check_golden_file( + pytestconfig=pytestconfig, + output_path=f.name, + golden_path=golden_path, + ignore_paths=ignore_paths, + ) + + def _get_field_for_entity_type_in_mce(entity_type: str) -> str: """Returns the field to look for depending on the type of entity in the MCE""" if entity_type == EntityType.DATASET: diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_aggregate_operations.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_aggregate_operations.json new file mode 100644 index 0000000000000..551760b42394c --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_aggregate_operations.json @@ -0,0 +1,44 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1707182625000, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "actor": "urn:li:corpuser:user2", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2" + }, + "lastUpdatedTimestamp": 25000 + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1707182625000, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "actor": "urn:li:corpuser:user3", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:7fd78ed5f3d60f7f91206f5e0fea6851a2afe940944455fd292267613b7ee1e6" + }, + "lastUpdatedTimestamp": 26000 + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_lineage.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_lineage.json new file mode 100644 index 0000000000000..5eaeb4e983925 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_lineage.json @@ -0,0 +1,94 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),a)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),b)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),b)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create table foo as select a, b from bar", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cbdb3e148ea7fdae81815da4dd64f57873fb9c3d7d4bfad4e83b3d1ebd3c45c2", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)" + } + ] + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_overlapping_inserts.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_overlapping_inserts.json new file mode 100644 index 0000000000000..27bd757c267b7 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_overlapping_inserts.json @@ -0,0 +1,172 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 20000, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream2,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e" + }, + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 20000, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream1,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:377a73bbf094c8b176b15157c24242cdfc7a0f407d78e52e63ded08c913468f1" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream1,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD),a)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:377a73bbf094c8b176b15157c24242cdfc7a0f407d78e52e63ded08c913468f1" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream2,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD),a)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream2,PROD),c)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD),c)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream1,PROD),b)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD),b)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:377a73bbf094c8b176b15157c24242cdfc7a0f407d78e52e63ded08c913468f1" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:377a73bbf094c8b176b15157c24242cdfc7a0f407d78e52e63ded08c913468f1", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "insert into downstream (a, b) select a, b from upstream1", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:377a73bbf094c8b176b15157c24242cdfc7a0f407d78e52e63ded08c913468f1", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream1,PROD)" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "insert into downstream (a, c) select a, c from upstream2", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.downstream,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.upstream2,PROD)" + } + ] + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_temp_table.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_temp_table.json new file mode 100644 index 0000000000000..31a37d6237e7b --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_temp_table.json @@ -0,0 +1,254 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session3,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:3e85e6f353c7fa33d6514cb090482852064d23df6491c9a8ae28be0d990a3c71" + } + ], + "fineGrainedLineages": [] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:3e85e6f353c7fa33d6514cb090482852064d23df6491c9a8ae28be0d990a3c71", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create table foo_session3 as select * from foo", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:3e85e6f353c7fa33d6514cb090482852064d23df6491c9a8ae28be0d990a3c71", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session3,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)" + } + ] + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session2,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:composite_66ddf44283e4543440529f1d13b82221b5d60635b6a8c39751718049ce4f47ec" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session2,PROD),a)" + ], + "confidenceScore": 0.35, + "query": "urn:li:query:composite_66ddf44283e4543440529f1d13b82221b5d60635b6a8c39751718049ce4f47ec" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),b)", + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),c)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session2,PROD),c)" + ], + "confidenceScore": 0.35, + "query": "urn:li:query:composite_66ddf44283e4543440529f1d13b82221b5d60635b6a8c39751718049ce4f47ec" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_66ddf44283e4543440529f1d13b82221b5d60635b6a8c39751718049ce4f47ec", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create temp table foo as select a, b+c as c from bar;\n\ncreate table foo_session2 as select * from foo", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 1707251710392, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:8b3a079997d562bdb1b14eb500e6123c4b00bb0263565dcaa0b66170e72602a1" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),a)" + ], + "confidenceScore": 0.35, + "query": "urn:li:query:8b3a079997d562bdb1b14eb500e6123c4b00bb0263565dcaa0b66170e72602a1" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),b)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),b)" + ], + "confidenceScore": 0.35, + "query": "urn:li:query:8b3a079997d562bdb1b14eb500e6123c4b00bb0263565dcaa0b66170e72602a1" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_66ddf44283e4543440529f1d13b82221b5d60635b6a8c39751718049ce4f47ec", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo_session2,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8b3a079997d562bdb1b14eb500e6123c4b00bb0263565dcaa0b66170e72602a1", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create table foo as select a, 2*b as b from bar", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8b3a079997d562bdb1b14eb500e6123c4b00bb0263565dcaa0b66170e72602a1", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)" + } + ] + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_view_lineage.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_view_lineage.json new file mode 100644 index 0000000000000..3f8fa7e5a1e28 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_view_lineage.json @@ -0,0 +1,94 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)", + "type": "VIEW", + "query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aredshift%2Cdev.public.foo%2CPROD%29" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),a)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),a)" + ], + "confidenceScore": 0.9, + "query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aredshift%2Cdev.public.foo%2CPROD%29" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD),b)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD),b)" + ], + "confidenceScore": 0.9, + "query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aredshift%2Cdev.public.foo%2CPROD%29" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aredshift%2Cdev.public.foo%2CPROD%29", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create view foo as select a, b from bar", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aredshift%2Cdev.public.foo%2CPROD%29", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)" + } + ] + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_create_view_with_cte.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_create_view_with_cte.json index d610b0a83f229..c237e3bff2a9b 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_create_view_with_cte.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_create_view_with_cte.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_VIEW", + "query_type_props": { + "kind": "VIEW" + }, + "query_fingerprint": "87b3f194c310daa80b6ce300a9bfa6f147460f43589e614ff05827952449a517", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,my-proj-1.dataset.table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:bigquery,my-proj-1.dataset.table2,PROD)", diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_from_sharded_table_wildcard.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_from_sharded_table_wildcard.json index 2d3d188d28316..1f4b3563f0114 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_from_sharded_table_wildcard.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_from_sharded_table_wildcard.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "3316d40c409d45e97615e8dece5ea9ba11020aca4bb8d903100ee8c81372e73d", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,bq-proj.dataset.table_yyyymmdd,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_nested_subqueries.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_nested_subqueries.json index 41ae0885941b0..63df823b31ccb 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_nested_subqueries.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_nested_subqueries.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "4a9acda021e0e969af81bf41690fcb383e0e1009e1716945087cb92d90a137b8", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,bq-proj.dataset.table1,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_sharded_table_normalization.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_sharded_table_normalization.json index 2d3d188d28316..edc70335244c6 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_sharded_table_normalization.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_sharded_table_normalization.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "68b038ff09626bbe2c4bc79be39ce51b50937457008e08461cdd6ed3b6ae3f2e", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,bq-proj.dataset.table_yyyymmdd,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_star_with_replace.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_star_with_replace.json index 26f8f8f59a3ff..56b592112df88 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_star_with_replace.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_star_with_replace.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_VIEW", + "query_type_props": { + "kind": "VIEW" + }, + "query_fingerprint": "4b2d3a58d47ddc4c1beeaddf5d296ff460a85ad5142009950aa072bb97fe771d", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,my-project2.my-dataset2.test_physical_table,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_view_from_union.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_view_from_union.json index 83365c09f69c2..7153546632d64 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_view_from_union.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_bigquery_view_from_union.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_VIEW", + "query_type_props": { + "kind": "VIEW" + }, + "query_fingerprint": "e947f7884942bae354b9f90cdb649c01efd85ab9fddf830b0bec194c22652639", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,my_project_2.my_dataset_2.sometable,PROD)", "urn:li:dataset:(urn:li:dataPlatform:bigquery,my_project_2.my_dataset_2.sometable2,PROD)" 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 index cf31b71cb50f6..a0ae968bfefb7 100644 --- 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 @@ -1,5 +1,7 @@ { - "query_type": "CREATE", + "query_type": "CREATE_DDL", + "query_type_props": {}, + "query_fingerprint": "22a615c14258f9fc2e1da76c26882cc4bc175eab53caf3c2591e1978df49d22e", "in_tables": [], "out_tables": [ "urn:li:dataset:(urn:li:dataPlatform:sqlite,costs,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_view_as_select.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_view_as_select.json index 8a6b60d0f1bde..24c85e7d0871b 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_view_as_select.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_create_view_as_select.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_VIEW", + "query_type_props": { + "kind": "VIEW" + }, + "query_fingerprint": "00400d6c706790ec92f8f1a9fbe41e05573168b071f32f9dac714180c3ef3b86", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:oracle,scott.emp,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_expand_select_star_basic.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_expand_select_star_basic.json index eecb2265eaec5..d72471884e97d 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_expand_select_star_basic.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_expand_select_star_basic.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "58d404ff045cd7d48c59383145619c04dbbc8ead2ffdf2b7236b7546e81e27fc", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_as_select.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_as_select.json index 326db47e7ab33..dbc844490ab33 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_as_select.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_as_select.json @@ -1,5 +1,7 @@ { "query_type": "INSERT", + "query_type_props": {}, + "query_fingerprint": "2ef01d0ff1cb2f129dbe746a3dc1a1c4403f1203fb9bd1bfaf8aa8cafdd2ac32", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:hive,catalog_returns,PROD)", "urn:li:dataset:(urn:li:dataPlatform:hive,catalog_sales,PROD)", diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_with_column_list.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_with_column_list.json new file mode 100644 index 0000000000000..8dd3e352bc535 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_insert_with_column_list.json @@ -0,0 +1,41 @@ +{ + "query_type": "INSERT", + "query_type_props": {}, + "query_fingerprint": "c4b3a21ef8c262ebbe99a5bdb6c29cb0be646392bb4af10b6f4a758af881470e", + "in_tables": [ + "urn:li:dataset:(urn:li:dataPlatform:redshift,upstream2,PROD)" + ], + "out_tables": [ + "urn:li:dataset:(urn:li:dataPlatform:redshift,downstream,PROD)" + ], + "column_lineage": [ + { + "downstream": { + "table": "urn:li:dataset:(urn:li:dataPlatform:redshift,downstream,PROD)", + "column": "a", + "column_type": null, + "native_column_type": null + }, + "upstreams": [ + { + "table": "urn:li:dataset:(urn:li:dataPlatform:redshift,upstream2,PROD)", + "column": "a" + } + ] + }, + { + "downstream": { + "table": "urn:li:dataset:(urn:li:dataPlatform:redshift,downstream,PROD)", + "column": "c", + "column_type": null, + "native_column_type": null + }, + "upstreams": [ + { + "table": "urn:li:dataset:(urn:li:dataPlatform:redshift,upstream2,PROD)", + "column": "c" + } + ] + } + ] +} \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_merge_from_union.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_merge_from_union.json index ec8599353fdca..ddbb559e8f1b1 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_merge_from_union.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_merge_from_union.json @@ -1,5 +1,7 @@ { "query_type": "MERGE", + "query_type_props": {}, + "query_fingerprint": "d6631edda99334cff011305d30c423f01d996185372c2aec6bb3e82e185fcad0", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,demo-pipelines-stg.referrer.prep_from_ios,PROD)", "urn:li:dataset:(urn:li:dataPlatform:bigquery,demo-pipelines-stg.referrer.prep_from_web,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_postgres_select_subquery.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_postgres_select_subquery.json index 0c40ce120c934..660da7d53a105 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_postgres_select_subquery.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_postgres_select_subquery.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "3f92d4d52c670825ca41786036f54a8658763c584b55c54ba3aeb85b09215346", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:postgres,my_db.my_schema.table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:postgres,my_db.my_schema.table2,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_materialized_view_auto_refresh.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_materialized_view_auto_refresh.json index fce65056a32f7..de55394173329 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_materialized_view_auto_refresh.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_materialized_view_auto_refresh.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_VIEW", + "query_type_props": { + "kind": "VIEW" + }, + "query_fingerprint": "95c53f18203e740eaff371573f18506801fe194fdf38b7e06527e45eb67a34a0", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:redshift,customer,PROD)", "urn:li:dataset:(urn:li:dataPlatform:redshift,orders,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_temp_table_shortcut.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_temp_table_shortcut.json index 974eddb961d64..9a4069d0304e1 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_temp_table_shortcut.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_redshift_temp_table_shortcut.json @@ -1,5 +1,10 @@ { - "query_type": "CREATE", + "query_type": "CREATE_TABLE_AS_SELECT", + "query_type_props": { + "kind": "TABLE", + "temporary": true + }, + "query_fingerprint": "252f5a0232a14a4533919960412ad2681c14b14b8045c046b23ac3d2411c4c5e", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:redshift,my_db.other_schema.table1,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_ambiguous_column_no_schema.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_ambiguous_column_no_schema.json index b5fd5eebeb1b1..4763a736b08c3 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_ambiguous_column_no_schema.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_ambiguous_column_no_schema.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "ac169845f106faeb32c4c1598b239b689947eaa906d72d65d5aed241c05ade2b", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:hive,t1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:hive,t2,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_count.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_count.json index a67c944822138..bdff51521b2a6 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_count.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_count.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "b701c1952e9d41bcfc1b163e2d9e6c518e7e6f812d5ed60473a4de7242e8e99e", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:mysql,something_prd.fact_complaint_snapshot,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_struct_subfields.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_struct_subfields.json index 2424fcda34752..438accca34649 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_struct_subfields.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_struct_subfields.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "eb98657e75b827086b921f27b2b468629b88a972e208ded05ce7478b494c6eb0", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,my-bq-proj.data_reporting.abcde_transformed,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_union.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_union.json index 5d1d421f49a2a..c6a24638a86c8 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_union.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_from_union.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "8829a82b034a08ebbac7ce5019e2537bc220cc3904bb81159085e7430a39408f", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf10.orders,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf100.orders,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max.json index 6ea88f45847ce..46edda4f0909b 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "ca9df3c737a7275934a0d3950bc76169f36a5931ccc05967b394849ea1d1863f", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:mysql,mytable,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max_with_schema.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max_with_schema.json new file mode 100644 index 0000000000000..a914447bdd4c0 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_max_with_schema.json @@ -0,0 +1,29 @@ +{ + "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "727911301a80a595a80671e784793aa2ea1be2e2924e10eba3e0c622e5082b0d", + "in_tables": [ + "urn:li:dataset:(urn:li:dataPlatform:mysql,mytable,PROD)" + ], + "out_tables": [], + "column_lineage": [ + { + "downstream": { + "table": null, + "column": "max_col", + "column_type": null, + "native_column_type": null + }, + "upstreams": [ + { + "table": "urn:li:dataset:(urn:li:dataPlatform:mysql,mytable,PROD)", + "column": "col1" + }, + { + "table": "urn:li:dataset:(urn:li:dataPlatform:mysql,mytable,PROD)", + "column": "col2" + } + ] + } + ] +} \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_ctes.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_ctes.json index 67e9fd2d21a0e..baa5ade05bb3c 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_ctes.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_ctes.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "a82cc26433c370a26b8aeecc2d8a1b672eab44349e00ec6f3da837c633e39b1c", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:oracle,table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:oracle,table2,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_full_col_name.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_full_col_name.json index 8dd2633eff612..a118fa7acb711 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_full_col_name.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_select_with_full_col_name.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "48a706402e955c2f64112c8c71a96dfd81a2b39101bf592670f14a5a9069c8c1", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:bigquery,my-bq-ProjectName.data_reporting.abcde_transformed,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_case_statement.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_case_statement.json index a876824127ec1..25fb499cfd084 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_case_statement.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_case_statement.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "ce34f7c5875166da53d4ab9d32b1314564634797cabe477a74eeebc7a164c3cd", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_cast.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_cast.json index 7545e2b3269dc..0d3fda383870e 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_cast.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_cast.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "3aefbf517edfe6067a2215de3431f133e5e181b2119c8e1c41ca71757f152fa4", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_normalization.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_normalization.json index 84e6b053000f1..3f4a8ece99a8c 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_normalization.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_column_normalization.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "312006fbb9ac30c190e1d15a23c84de1c3ba1f4eb4b8fac208be0f72b1a3b239", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_ctas_column_normalization.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_ctas_column_normalization.json index 39c94cf83c561..5ccf44a333db0 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_ctas_column_normalization.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_ctas_column_normalization.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_TABLE_AS_SELECT", + "query_type_props": { + "kind": "TABLE" + }, + "query_fingerprint": "6aaac237788fd64545ccb5409ad7a175e82b6986a880cc73f3e42a0d00a69497", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_cte_name_collision.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_cte_name_collision.json index 44f1075c058ad..ca0fdac55d555 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_cte_name_collision.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_cte_name_collision.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "f5250fa91f43bbe0d7d5a6fc0d2806a4f989952e2114725cf9a274ec09857171", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,my_db.my_schema.table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,my_db.my_schema.table2,PROD)", diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_default_normalization.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_default_normalization.json index dbf5b1b9a4453..53fe500950507 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_default_normalization.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_default_normalization.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_TABLE_AS_SELECT", + "query_type_props": { + "kind": "TABLE" + }, + "query_fingerprint": "7ec7291bc95ab8e4978f86f3854336100aadca5143b79cfd92edbd5f245af916", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,long_tail_companions.analytics.customer_last_purchase_date,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,long_tail_companions.ecommerce.purchases,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_full_table_name_col_reference.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_full_table_name_col_reference.json index f8301f1e8189e..1e88e638f0fae 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_full_table_name_col_reference.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_full_table_name_col_reference.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "2b685ace8590f4bf11fe501a1b9fa342ed095904e3d3097f6b41276f590d99cd", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,my_db.my_schema.my_table,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_unused_cte.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_unused_cte.json index 3916c6dc7c5ef..d629d16848cdb 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_unused_cte.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_unused_cte.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "2b4cf29392c9aad44822b36fff4e4d833f747a3eb8b7b1b7143af55bc88dc1e0", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,table2,PROD)", diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_from_table.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_from_table.json index d51001f969799..a4b2e636a6867 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_from_table.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_from_table.json @@ -1,5 +1,7 @@ { "query_type": "UPDATE", + "query_type_props": {}, + "query_fingerprint": "a18081be3e7d4ab35c3357efb3c9e7acb66a1d4f000367e1db48dbdf00c5b3c3", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,my_db.my_schema.my_table,PROD)", "urn:li:dataset:(urn:li:dataPlatform:snowflake,my_db.my_schema.table1,PROD)", diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_hardcoded.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_hardcoded.json index f421b28530c64..a6ac74f36006b 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_hardcoded.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_hardcoded.json @@ -1,5 +1,7 @@ { "query_type": "UPDATE", + "query_type_props": {}, + "query_fingerprint": "bd1279a3a8d360045a0522a2c3b98d8f937edba4ca14c3851acba1fd3554849d", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_self.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_self.json index c8cc32164a3eb..6ffc9cfc2c714 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_self.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_snowflake_update_self.json @@ -1,5 +1,7 @@ { "query_type": "UPDATE", + "query_type_props": {}, + "query_fingerprint": "07c0bfecc799fac2e1e9bdbd29cc33688c797b4baaa54724555eacfce2e2bf57", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:snowflake,snowflake_sample_data.tpch_sf1.orders,PROD)" ], diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_default_normalization.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_default_normalization.json index ee80285d87f60..40cf06865ac21 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_default_normalization.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_default_normalization.json @@ -1,5 +1,9 @@ { - "query_type": "CREATE", + "query_type": "CREATE_TABLE_AS_SELECT", + "query_type_props": { + "kind": "TABLE" + }, + "query_fingerprint": "4950101f5207463ec1e8637ea686b65823d23f4c4d0682bc0ae560b9abbf82fe", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:teradata,myteradata.demo_user.pima_patient_diagnoses,PROD)", "urn:li:dataset:(urn:li:dataPlatform:teradata,myteradata.demo_user.pima_patient_features,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_strange_operators.json b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_strange_operators.json index 4b21a2512ccd1..d928502487984 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_strange_operators.json +++ b/metadata-ingestion/tests/unit/sql_parsing/goldens/test_teradata_strange_operators.json @@ -1,5 +1,7 @@ { "query_type": "SELECT", + "query_type_props": {}, + "query_fingerprint": "f32a0697f19afcbf7261385518fc404ac477496c82115f63cfddda8790160731", "in_tables": [ "urn:li:dataset:(urn:li:dataPlatform:teradata,dbc.table1,PROD)", "urn:li:dataset:(urn:li:dataPlatform:teradata,dbc.table2,PROD)" diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_schemaresolver.py b/metadata-ingestion/tests/unit/sql_parsing/test_schemaresolver.py index 3fd5d72b4d41a..5786c135a8d8c 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_schemaresolver.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_schemaresolver.py @@ -1,4 +1,4 @@ -from datahub.utilities.sqlglot_lineage import SchemaResolver, _TableName +from datahub.sql_parsing.schema_resolver import SchemaResolver, _TableName def test_get_urn_for_table_lowercase(): @@ -31,3 +31,4 @@ def test_get_urn_for_table_not_lower_should_keep_capital_letters(): schema_resolver.get_urn_for_table(table=table, lower=False) == "urn:li:dataset:(urn:li:dataPlatform:mssql,Uppercased-Instance.Database.DataSet.Table,PROD)" ) + assert schema_resolver.schema_count() == 0 diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py new file mode 100644 index 0000000000000..8b0318664ea05 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py @@ -0,0 +1,217 @@ +import pathlib +from datetime import datetime, timezone + +import pytest +from freezegun import freeze_time + +import datahub.emitter.mce_builder as builder +from datahub.metadata.urns import CorpUserUrn, DatasetUrn +from datahub.sql_parsing.sql_parsing_aggregator_v2 import ( + QueryLogSetting, + SqlParsingAggregator, +) +from tests.test_helpers import mce_helpers + +RESOURCE_DIR = pathlib.Path(__file__).parent / "aggregator_goldens" +FROZEN_TIME = "2024-02-06 01:23:45" + + +def _ts(ts: int) -> datetime: + return datetime.fromtimestamp(ts, tz=timezone.utc) + + +@freeze_time(FROZEN_TIME) +def test_basic_lineage(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + platform_instance=None, + env=builder.DEFAULT_ENV, + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=False, + ) + + aggregator.add_observed_query( + query="create table foo as select a, b from bar", + default_db="dev", + default_schema="public", + ) + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_basic_lineage.json", + ) + + +@freeze_time(FROZEN_TIME) +def test_overlapping_inserts(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + platform_instance=None, + env=builder.DEFAULT_ENV, + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=False, + ) + + aggregator.add_observed_query( + query="insert into downstream (a, b) select a, b from upstream1", + default_db="dev", + default_schema="public", + query_timestamp=_ts(20), + ) + aggregator.add_observed_query( + query="insert into downstream (a, c) select a, c from upstream2", + default_db="dev", + default_schema="public", + query_timestamp=_ts(25), + ) + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_overlapping_inserts.json", + ) + + +@freeze_time(FROZEN_TIME) +def test_temp_table(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + platform_instance=None, + env=builder.DEFAULT_ENV, + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=False, + ) + + aggregator._schema_resolver.add_raw_schema_info( + DatasetUrn("redshift", "dev.public.bar").urn(), + {"a": "int", "b": "int", "c": "int"}, + ) + + aggregator.add_observed_query( + query="create table foo as select a, 2*b as b from bar", + default_db="dev", + default_schema="public", + session_id="session1", + ) + aggregator.add_observed_query( + query="create temp table foo as select a, b+c as c from bar", + default_db="dev", + default_schema="public", + session_id="session2", + ) + aggregator.add_observed_query( + query="create table foo_session2 as select * from foo", + default_db="dev", + default_schema="public", + session_id="session2", + ) + aggregator.add_observed_query( + query="create table foo_session3 as select * from foo", + default_db="dev", + default_schema="public", + session_id="session3", + ) + + # foo_session2 should come from bar (via temp table foo), have columns a and c, and depend on bar.{a,b,c} + # foo_session3 should come from foo, have columns a and b, and depend on bar.b + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_temp_table.json", + ) + + +@freeze_time(FROZEN_TIME) +def test_aggregate_operations(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + platform_instance=None, + env=builder.DEFAULT_ENV, + generate_lineage=False, + generate_queries=False, + generate_usage_statistics=False, + generate_operations=True, + ) + + aggregator.add_observed_query( + query="create table foo as select a, b from bar", + default_db="dev", + default_schema="public", + query_timestamp=_ts(20), + user=CorpUserUrn("user1"), + ) + aggregator.add_observed_query( + query="create table foo as select a, b from bar", + default_db="dev", + default_schema="public", + query_timestamp=_ts(25), + user=CorpUserUrn("user2"), + ) + aggregator.add_observed_query( + query="create table foo as select a, b+1 as b from bar", + default_db="dev", + default_schema="public", + query_timestamp=_ts(26), + user=CorpUserUrn("user3"), + ) + + # The first query will basically be ignored, as it's a duplicate of the second one. + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_aggregate_operations.json", + ) + + +@freeze_time(FROZEN_TIME) +def test_view_lineage(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + platform_instance=None, + env=builder.DEFAULT_ENV, + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=False, + query_log=QueryLogSetting.STORE_ALL, + ) + + aggregator.add_view_definition( + view_urn=DatasetUrn("redshift", "dev.public.foo"), + view_definition="create view foo as select a, b from bar", + default_db="dev", + default_schema="public", + ) + + aggregator._schema_resolver.add_raw_schema_info( + urn=DatasetUrn("redshift", "dev.public.foo").urn(), + schema_info={"a": "int", "b": "int"}, + ) + aggregator._schema_resolver.add_raw_schema_info( + urn=DatasetUrn("redshift", "dev.public.bar").urn(), + schema_info={"a": "int", "b": "int"}, + ) + + # Because we have schema information, despite it being registered after the view definition, + # the confidence score should be high. + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_view_lineage.json", + ) diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_detach.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_detach.py index c99b05c35e0f5..b45e2d0eff693 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sql_detach.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_detach.py @@ -1,4 +1,4 @@ -from datahub.utilities.sqlglot_lineage import detach_ctes +from datahub.sql_parsing.sqlglot_utils import detach_ctes def test_detach_ctes_simple(): 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 42863ab005f07..e3396e74fccdd 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_lineage.py @@ -33,8 +33,7 @@ def test_select_max_with_schema(): "col2": "NUMBER", }, }, - # Shared with the test above. - expected_file=RESOURCE_DIR / "test_select_max.json", + expected_file=RESOURCE_DIR / "test_select_max_with_schema.json", ) @@ -134,6 +133,16 @@ def test_insert_as_select(): ) +def test_insert_with_column_list(): + assert_sql_result( + """\ +insert into downstream (a, c) select a, c from upstream2 +""", + dialect="redshift", + expected_file=RESOURCE_DIR / "test_insert_with_column_list.json", + ) + + def test_select_with_full_col_name(): # In this case, `widget` is a struct column. # This also tests the `default_db` functionality. diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_utils.py b/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_utils.py index b01c512c383cb..7c4d431520a7e 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_utils.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sqlglot_utils.py @@ -1,7 +1,17 @@ -from datahub.utilities.sqlglot_lineage import ( +import textwrap + +import sqlglot + +from datahub.sql_parsing.sql_parsing_common import QueryType +from datahub.sql_parsing.sqlglot_lineage import ( _UPDATE_ARGS_NOT_SUPPORTED_BY_SELECT, - _get_dialect, - _is_dialect_instance, + get_query_type_of_sql, +) +from datahub.sql_parsing.sqlglot_utils import ( + generalize_query, + get_dialect, + get_query_fingerprint, + is_dialect_instance, ) @@ -10,11 +20,97 @@ def test_update_from_select(): def test_is_dialect_instance(): - snowflake = _get_dialect("snowflake") + snowflake = get_dialect("snowflake") + + assert is_dialect_instance(snowflake, "snowflake") + assert not is_dialect_instance(snowflake, "bigquery") + + redshift = get_dialect("redshift") + assert is_dialect_instance(redshift, ["redshift", "snowflake"]) + assert is_dialect_instance(redshift, ["postgres", "snowflake"]) + + +def test_query_types(): + assert get_query_type_of_sql( + sqlglot.parse_one( + "create temp table foo as select * from bar", dialect="redshift" + ), + dialect="redshift", + ) == (QueryType.CREATE_TABLE_AS_SELECT, {"kind": "TABLE", "temporary": True}) + + assert get_query_type_of_sql( + sqlglot.parse_one("create table #foo as select * from bar", dialect="redshift"), + dialect="redshift", + ) == (QueryType.CREATE_TABLE_AS_SELECT, {"kind": "TABLE", "temporary": True}) + + assert get_query_type_of_sql( + sqlglot.parse_one("create view foo as select * from bar", dialect="redshift"), + dialect="redshift", + ) == (QueryType.CREATE_VIEW, {"kind": "VIEW"}) + + +def test_query_generalization(): + # Basic keyword normalization. + assert ( + generalize_query("select * from foo", dialect="redshift") == "SELECT * FROM foo" + ) + + # Comment removal and whitespace normalization. + assert ( + generalize_query( + "/* query system = foo, id = asdf */\nselect /* inline comment */ *\nfrom foo", + dialect="redshift", + ) + == "SELECT * FROM foo" + ) + + # Parameter normalization. + assert ( + generalize_query( + "UPDATE \"books\" SET page_count = page_count + 1, author_count = author_count + 1 WHERE book_title = 'My New Book'", + dialect="redshift", + ) + == 'UPDATE "books" SET page_count = page_count + ?, author_count = author_count + ? WHERE book_title = ?' + ) + assert ( + generalize_query( + "select * from foo where date = '2021-01-01'", dialect="redshift" + ) + == "SELECT * FROM foo WHERE date = ?" + ) + assert ( + generalize_query( + "select * from books where category in ('fiction', 'biography', 'fantasy')", + dialect="redshift", + ) + == "SELECT * FROM books WHERE category IN (?)" + ) + assert ( + generalize_query( + textwrap.dedent( + """\ + /* Copied from https://stackoverflow.com/a/452934/5004662 */ + INSERT INTO MyTable + ( Column1, Column2, Column3 ) + VALUES + /* multiple value rows */ + ('John', 123, 'Lloyds Office'), + ('Jane', 124, 'Lloyds Office'), + ('Billy', 125, 'London Office'), + ('Miranda', 126, 'Bristol Office'); + """ + ), + dialect="mssql", + ) + == "INSERT INTO MyTable (Column1, Column2, Column3) VALUES (?), (?), (?), (?)" + ) + - assert _is_dialect_instance(snowflake, "snowflake") - assert not _is_dialect_instance(snowflake, "bigquery") +def test_query_fingerprint(): + assert get_query_fingerprint( + "select * /* everything */ from foo where ts = 34", dialect="redshift" + ) == get_query_fingerprint("SELECT * FROM foo where ts = 38", dialect="redshift") - redshift = _get_dialect("redshift") - assert _is_dialect_instance(redshift, ["redshift", "snowflake"]) - assert _is_dialect_instance(redshift, ["postgres", "snowflake"]) + assert get_query_fingerprint( + "select 1 + 1", dialect="postgres" + ) != get_query_fingerprint("select 2", dialect="postgres") diff --git a/metadata-ingestion/tests/unit/test_bigquery_lineage.py b/metadata-ingestion/tests/unit/test_bigquery_lineage.py index e23494963e475..1edac3fde0a6c 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_lineage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_lineage.py @@ -14,7 +14,7 @@ BigqueryLineageExtractor, LineageEdge, ) -from datahub.utilities.sqlglot_lineage import SchemaResolver +from datahub.sql_parsing.schema_resolver import SchemaResolver @pytest.fixture diff --git a/metadata-ingestion/tests/unit/test_bigquery_sql_lineage.py b/metadata-ingestion/tests/unit/test_bigquery_sql_lineage.py index 755e9081dda39..b685f12fae774 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_sql_lineage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_sql_lineage.py @@ -1,7 +1,8 @@ from typing import List from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigQueryTableRef -from datahub.utilities.sqlglot_lineage import SchemaResolver, sqlglot_lineage +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sqlglot_lineage import sqlglot_lineage class BigQuerySQLParser: diff --git a/metadata-ingestion/tests/unit/test_bigquery_usage.py b/metadata-ingestion/tests/unit/test_bigquery_usage.py index 664d3112810ff..30734414460f0 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_usage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_usage.py @@ -34,8 +34,8 @@ OperationClass, TimeWindowSizeClass, ) +from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.testing.compare_metadata_json import diff_metadata_json -from datahub.utilities.sqlglot_lineage import SchemaResolver from tests.performance.bigquery.bigquery_events import generate_events, ref_from_table from tests.performance.data_generation import generate_data, generate_queries from tests.performance.data_model import Container, FieldAccess, Query, Table, View @@ -922,12 +922,16 @@ def test_operational_stats( timestampMillis=int(FROZEN_TIME.timestamp() * 1000), lastUpdatedTimestamp=int(query.timestamp.timestamp() * 1000), actor=f"urn:li:corpuser:{query.actor.split('@')[0]}", - operationType=query.type - if query.type in OPERATION_STATEMENT_TYPES.values() - else "CUSTOM", - customOperationType=None - if query.type in OPERATION_STATEMENT_TYPES.values() - else query.type, + operationType=( + query.type + if query.type in OPERATION_STATEMENT_TYPES.values() + else "CUSTOM" + ), + customOperationType=( + None + if query.type in OPERATION_STATEMENT_TYPES.values() + else query.type + ), affectedDatasets=list( dict.fromkeys( # Preserve order BigQueryTableRef.from_string_name( diff --git a/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py b/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py index 25e849a509293..80f9ab927f887 100644 --- a/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py +++ b/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py @@ -10,7 +10,7 @@ from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryV2Config from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor -from datahub.utilities.sqlglot_lineage import SchemaResolver +from datahub.sql_parsing.schema_resolver import SchemaResolver FROZEN_TIME = "2021-07-20 00:00:00" diff --git a/metadata-ingestion/tests/unit/test_ordered_set.py b/metadata-ingestion/tests/unit/test_ordered_set.py new file mode 100644 index 0000000000000..d50c6798d283b --- /dev/null +++ b/metadata-ingestion/tests/unit/test_ordered_set.py @@ -0,0 +1,44 @@ +from datahub.utilities.ordered_set import OrderedSet + + +def test_ordered_set(): + # Test initialization + ordered_set: OrderedSet[int] = OrderedSet() + assert len(ordered_set) == 0 + + # Test adding items + ordered_set.add(1) + ordered_set.add(2) + ordered_set.add(3) + assert len(ordered_set) == 3 + + # Test adding duplicate item + ordered_set.add(1) + assert len(ordered_set) == 3 + + # Test discarding item + ordered_set.discard(2) + assert len(ordered_set) == 2 + assert 2 not in ordered_set + + # Test updating with iterable + ordered_set.update([4, 5, 6]) + assert len(ordered_set) == 5 + assert 4 in ordered_set + assert 5 in ordered_set + assert 6 in ordered_set + + # Test containment check + assert 3 in ordered_set + assert 7 not in ordered_set + + # Test iteration + items = list(ordered_set) + assert items == [1, 3, 4, 5, 6] + + # Test reverse iteration + items = list(reversed(ordered_set)) + assert items == [6, 5, 4, 3, 1] + + # Test string representation + assert repr(ordered_set) == "OrderedSet([1, 3, 4, 5, 6])" diff --git a/metadata-ingestion/tests/unit/test_redshift_lineage.py b/metadata-ingestion/tests/unit/test_redshift_lineage.py index 6a3e6e47bd96a..366a6009ee46a 100644 --- a/metadata-ingestion/tests/unit/test_redshift_lineage.py +++ b/metadata-ingestion/tests/unit/test_redshift_lineage.py @@ -3,7 +3,7 @@ from typing import List from unittest.mock import MagicMock -import datahub.utilities.sqlglot_lineage as sqlglot_l +import datahub.sql_parsing.sqlglot_lineage as sqlglot_l from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.graph.client import DataHubGraph from datahub.ingestion.source.redshift.config import RedshiftConfig @@ -17,11 +17,12 @@ ) from datahub.ingestion.source.redshift.redshift_schema import TempTableRow from datahub.ingestion.source.redshift.report import RedshiftReport -from datahub.metadata._schema_classes import NumberTypeClass, SchemaFieldDataTypeClass -from datahub.utilities.sqlglot_lineage import ( +from datahub.metadata.schema_classes import NumberTypeClass, SchemaFieldDataTypeClass +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sql_parsing_common import QueryType +from datahub.sql_parsing.sqlglot_lineage import ( ColumnLineageInfo, DownstreamColumnRef, - QueryType, SqlParsingDebugInfo, SqlParsingResult, ) @@ -223,7 +224,7 @@ def mock_graph() -> DataHubGraph: graph = MagicMock() - graph._make_schema_resolver.return_value = sqlglot_l.SchemaResolver( + graph._make_schema_resolver.return_value = SchemaResolver( platform="redshift", env="PROD", platform_instance=None, @@ -294,7 +295,7 @@ def test_collapse_temp_recursive_cll_lineage(): session_id="abc", create_command="CREATE TABLE #player_price", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.#player_activity_temp,PROD)" ], @@ -346,7 +347,7 @@ def test_collapse_temp_recursive_cll_lineage(): session_id="abc", create_command="CREATE TABLE #player_activity_temp", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.player_activity,PROD)" ], @@ -458,7 +459,7 @@ def test_collapse_temp_recursive_with_compex_column_cll_lineage(): session_id="abc", create_command="CREATE TABLE #player_price", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.#player_activity_temp,PROD)" ], @@ -514,7 +515,7 @@ def test_collapse_temp_recursive_with_compex_column_cll_lineage(): session_id="abc", create_command="CREATE TABLE #player_activity_temp", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.player_activity,PROD)" ], @@ -657,7 +658,7 @@ def test_collapse_temp_recursive_cll_lineage_with_circular_reference(): session_id="abc", create_command="CREATE TABLE #player_price", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.#player_activity_temp,PROD)" ], @@ -709,7 +710,7 @@ def test_collapse_temp_recursive_cll_lineage_with_circular_reference(): session_id="abc", create_command="CREATE TABLE #player_activity_temp", parsed_result=SqlParsingResult( - query_type=QueryType.CREATE, + query_type=QueryType.CREATE_TABLE_AS_SELECT, in_tables=[ "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.player_activity,PROD)" ], diff --git a/metadata-ingestion/tests/unit/utilities/test_file_backed_collections.py b/metadata-ingestion/tests/unit/utilities/test_file_backed_collections.py index 2d7556eb88341..06d1cfc7d154d 100644 --- a/metadata-ingestion/tests/unit/utilities/test_file_backed_collections.py +++ b/metadata-ingestion/tests/unit/utilities/test_file_backed_collections.py @@ -1,6 +1,7 @@ import dataclasses import json import pathlib +import random import sqlite3 from dataclasses import dataclass from typing import Counter, Dict @@ -54,6 +55,12 @@ def test_file_dict() -> None: assert cache["key-3"] == 99 cache["key-3"] = 3 + # Test in operator + assert "key-3" in cache + assert "key-99" in cache + assert "missing" not in cache + assert "missing" not in cache + # Test deleting keys, in and out of cache del cache["key-0"] del cache["key-99"] @@ -155,16 +162,23 @@ def test_file_dict_stores_counter() -> None: ) n = 5 + + # initialize in_memory_counters: Dict[int, Counter[str]] = {} for i in range(n): cache[str(i)] = Counter[str]() in_memory_counters[i] = Counter[str]() - for j in range(n): - if i == j: - cache[str(i)][str(j)] += 100 - in_memory_counters[i][str(j)] += 100 - cache[str(i)][str(j)] += j + + # increment the counters + increments = [(i, j) for i in range(n) for j in range(n)] + random.shuffle(increments) + for i, j in increments: + if i == j: + cache[str(i)][str(j)] += 100 + j cache.mark_dirty(str(i)) + in_memory_counters[i][str(j)] += 100 + j + else: + cache.for_mutation(str(i))[str(j)] += j in_memory_counters[i][str(j)] += j for i in range(n):