diff --git a/metadata-ingestion/docs/sources/trino/trino_recipe.yml b/metadata-ingestion/docs/sources/trino/trino_recipe.yml index 70e2afa81b972..06158669143a9 100644 --- a/metadata-ingestion/docs/sources/trino/trino_recipe.yml +++ b/metadata-ingestion/docs/sources/trino/trino_recipe.yml @@ -13,6 +13,15 @@ source: # options: # connect_args: # http_scheme: http + + # Optional -- A mapping of trino catalog to its connector details like connector database, env and platform instance. + # This configuration is used to ingest lineage of datasets to connectors. Use catalog name as key. + # catalog_to_connector_details: + # catalog_name: + # connector_database: db_name + # connector_platform: connector_platform_name + # platform_instance: cloud_instance + # env: DEV sink: # sink configs diff --git a/metadata-ingestion/src/datahub/ingestion/source/sql/trino.py b/metadata-ingestion/src/datahub/ingestion/source/sql/trino.py index cb2e05765bfff..cf199237e3041 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/sql/trino.py +++ b/metadata-ingestion/src/datahub/ingestion/source/sql/trino.py @@ -1,7 +1,9 @@ +import functools import json +import logging import uuid from textwrap import dedent -from typing import Any, Dict, List, Optional +from typing import Any, Dict, Iterable, List, Optional, Union import sqlalchemy import trino @@ -9,13 +11,19 @@ from pydantic.fields import Field from sqlalchemy import exc, sql from sqlalchemy.engine import reflection +from sqlalchemy.engine.base import Engine from sqlalchemy.engine.reflection import Inspector from sqlalchemy.sql import sqltypes from sqlalchemy.types import TypeEngine -from trino.exceptions import TrinoQueryError from trino.sqlalchemy import datatype from trino.sqlalchemy.dialect import TrinoDialect +from datahub.configuration.source_common import ( + EnvConfigMixin, + PlatformInstanceConfigMixin, +) +from datahub.emitter.mce_builder import make_dataset_urn_with_platform_instance +from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.api.decorators import ( SourceCapability, @@ -25,12 +33,23 @@ platform_name, support_status, ) +from datahub.ingestion.api.workunit import MetadataWorkUnit from datahub.ingestion.extractor import schema_util from datahub.ingestion.source.sql.sql_common import ( SQLAlchemySource, + SqlWorkUnit, register_custom_type, ) -from datahub.ingestion.source.sql.sql_config import BasicSQLAlchemyConfig +from datahub.ingestion.source.sql.sql_config import ( + BasicSQLAlchemyConfig, + SQLCommonConfig, +) +from datahub.metadata.com.linkedin.pegasus2avro.common import Siblings +from datahub.metadata.com.linkedin.pegasus2avro.dataset import ( + DatasetLineageType, + Upstream, + UpstreamLineage, +) from datahub.metadata.com.linkedin.pegasus2avro.schema import ( MapTypeClass, NumberTypeClass, @@ -42,11 +61,46 @@ register_custom_type(datatype.MAP, MapTypeClass) register_custom_type(datatype.DOUBLE, NumberTypeClass) + +KNOWN_CONNECTOR_PLATFORM_MAPPING = { + "clickhouse": "clickhouse", + "hive": "hive", + "glue": "glue", + "iceberg": "iceberg", + "mysql": "mysql", + "postgresql": "postgres", + "redshift": "redshift", + "bigquery": "bigquery", + "snowflake_distributed": "snowflake", + "snowflake_parallel": "snowflake", + "snowflake_jdbc": "snowflake", +} + +TWO_TIER_CONNECTORS = ["clickhouse", "hive", "glue", "mysql", "iceberg"] + +PROPERTIES_TABLE_SUPPORTED_CONNECTORS = ["hive", "iceberg"] + # Type JSON was introduced in trino sqlalchemy dialect in version 0.317.0 if version.parse(trino.__version__) >= version.parse("0.317.0"): register_custom_type(datatype.JSON, RecordTypeClass) +@functools.lru_cache() +def gen_catalog_connector_dict(engine: Engine) -> Dict[str, str]: + query = dedent( + """ + SELECT * + FROM "system"."metadata"."catalogs" + """ + ).strip() + res = engine.execute(sql.text(query)) + return {row.catalog_name: row.connector_name for row in res} + + +def get_catalog_connector_name(engine: Engine, catalog_name: str) -> Optional[str]: + return gen_catalog_connector_dict(engine).get(catalog_name) + + # Read only table names and skip view names, as view names will also be returned # from get_view_names @reflection.cache # type: ignore @@ -69,26 +123,27 @@ def get_table_names(self, connection, schema: str = None, **kw): # type: ignore @reflection.cache # type: ignore def get_table_comment(self, connection, table_name: str, schema: str = None, **kw): # type: ignore try: - properties_table = self._get_full_table(f"{table_name}$properties", schema) - query = f"SELECT * FROM {properties_table}" - row = connection.execute(sql.text(query)).fetchone() - - # Generate properties dictionary. - properties = {} - if row: - for col_name, col_value in row.items(): - if col_value is not None: - properties[col_name] = col_value - - return {"text": properties.get("comment", None), "properties": properties} - # Fallback to default trino-sqlalchemy behaviour if `$properties` table doesn't exist - except TrinoQueryError: - return self.get_table_comment_default(connection, table_name, schema) - # Exception raised when using Starburst Delta Connector that falls back to a Hive Catalog - except exc.ProgrammingError as e: - if isinstance(e.orig, TrinoQueryError): + catalog_name = self._get_default_catalog_name(connection) + if catalog_name is None: + raise exc.NoSuchTableError("catalog is required in connection") + connector_name = get_catalog_connector_name(connection.engine, catalog_name) + if connector_name is None: + return {} + if connector_name in PROPERTIES_TABLE_SUPPORTED_CONNECTORS: + properties_table = self._get_full_table(f"{table_name}$properties", schema) + query = f"SELECT * FROM {properties_table}" + row = connection.execute(sql.text(query)).fetchone() + + # Generate properties dictionary. + properties = {} + if row: + for col_name, col_value in row.items(): + if col_value is not None: + properties[col_name] = col_value + + return {"text": properties.get("comment", None), "properties": properties} + else: return self.get_table_comment_default(connection, table_name, schema) - raise except Exception: return {} @@ -131,19 +186,38 @@ def _get_columns(self, connection, table_name, schema: str = None, **kw): # typ TrinoDialect._get_columns = _get_columns +class ConnectorDetail(PlatformInstanceConfigMixin, EnvConfigMixin): + connector_database: Optional[str] = Field(default=None, description="") + connector_platform: Optional[str] = Field( + default=None, + description="A connector's actual platform name. If not provided, will take from metadata tables" + "Eg: hive catalog can have a connector platform as 'hive' or 'glue' or some other metastore.", + ) + + class TrinoConfig(BasicSQLAlchemyConfig): # defaults scheme: str = Field(default="trino", description="", hidden_from_docs=True) + database: str = Field(description="database (catalog)") + + catalog_to_connector_details: Dict[str, ConnectorDetail] = Field( + default={}, + description="A mapping of trino catalog to its connector details like connector database, env and platform instance." + "This configuration is used to build lineage to the underlying connector. Use catalog name as key.", + ) + + ingest_lineage_to_connectors: bool = Field( + default=True, + description="Whether lineage of datasets to connectors should be ingested", + ) + + trino_as_primary: bool = Field( + default=True, + description="Experimental feature. Whether trino dataset should be primary entity of the set of siblings", + ) def get_identifier(self: BasicSQLAlchemyConfig, schema: str, table: str) -> str: - identifier = f"{schema}.{table}" - if self.database: # TODO: this should be required field - identifier = f"{self.database}.{identifier}" - return ( - f"{self.platform_instance}.{identifier}" - if self.platform_instance - else identifier - ) + return f"{self.database}.{schema}.{table}" @platform_name("Trino", doc_order=1) @@ -175,6 +249,133 @@ def get_db_name(self, inspector: Inspector) -> str: else: return super().get_db_name(inspector) + def _get_source_dataset_urn( + self, + dataset_name: str, + inspector: Inspector, + schema: str, + table: str, + ) -> Optional[str]: + catalog_name = dataset_name.split(".")[0] + connector_name = get_catalog_connector_name(inspector.engine, catalog_name) + if not connector_name: + return None + connector_details = self.config.catalog_to_connector_details.get( + catalog_name, ConnectorDetail() + ) + connector_platform_name = KNOWN_CONNECTOR_PLATFORM_MAPPING.get( + connector_details.connector_platform or connector_name + ) + if not connector_platform_name: + logging.debug(f"Platform '{connector_platform_name}' is not yet supported.") + return None + + if connector_platform_name in TWO_TIER_CONNECTORS: # connector is two tier + return make_dataset_urn_with_platform_instance( + platform=connector_platform_name, + name=f"{schema}.{table}", + platform_instance=connector_details.platform_instance, + env=connector_details.env, + ) + elif connector_details.connector_database: # else connector is three tier + return make_dataset_urn_with_platform_instance( + platform=connector_platform_name, + name=f"{connector_details.connector_database}.{schema}.{table}", + platform_instance=connector_details.platform_instance, + env=connector_details.env, + ) + else: + logging.warning(f"Connector database missing for Catalog '{catalog_name}'.") + return None + + def gen_siblings_workunit( + self, + dataset_urn: str, + source_dataset_urn: str, + ) -> Iterable[MetadataWorkUnit]: + """ + Generate sibling workunit for both trino dataset and its connector source dataset + """ + yield MetadataChangeProposalWrapper( + entityUrn=dataset_urn, + aspect=Siblings( + primary=self.config.trino_as_primary, siblings=[source_dataset_urn] + ), + ).as_workunit() + + yield MetadataChangeProposalWrapper( + entityUrn=source_dataset_urn, + aspect=Siblings( + primary=not self.config.trino_as_primary, siblings=[dataset_urn] + ), + ).as_workunit() + + def gen_lineage_workunit( + self, + dataset_urn: str, + source_dataset_urn: str, + ) -> Iterable[MetadataWorkUnit]: + """ + Generate dataset to source connector lineage workunit + """ + yield MetadataChangeProposalWrapper( + entityUrn=dataset_urn, + aspect=UpstreamLineage( + upstreams=[ + Upstream(dataset=source_dataset_urn, type=DatasetLineageType.VIEW) + ] + ), + ).as_workunit() + + def _process_table( + self, + dataset_name: str, + inspector: Inspector, + schema: str, + table: str, + sql_config: SQLCommonConfig, + ) -> Iterable[Union[SqlWorkUnit, MetadataWorkUnit]]: + yield from super()._process_table( + dataset_name, inspector, schema, table, sql_config + ) + if self.config.ingest_lineage_to_connectors: + dataset_urn = make_dataset_urn_with_platform_instance( + self.platform, + dataset_name, + self.config.platform_instance, + self.config.env, + ) + source_dataset_urn = self._get_source_dataset_urn( + dataset_name, inspector, schema, table + ) + if source_dataset_urn: + yield from self.gen_siblings_workunit(dataset_urn, source_dataset_urn) + yield from self.gen_lineage_workunit(dataset_urn, source_dataset_urn) + + def _process_view( + self, + dataset_name: str, + inspector: Inspector, + schema: str, + view: str, + sql_config: SQLCommonConfig, + ) -> Iterable[Union[SqlWorkUnit, MetadataWorkUnit]]: + yield from super()._process_view( + dataset_name, inspector, schema, view, sql_config + ) + if self.config.ingest_lineage_to_connectors: + dataset_urn = make_dataset_urn_with_platform_instance( + self.platform, + dataset_name, + self.config.platform_instance, + self.config.env, + ) + source_dataset_urn = self._get_source_dataset_urn( + dataset_name, inspector, schema, view + ) + if source_dataset_urn: + yield from self.gen_siblings_workunit(dataset_urn, source_dataset_urn) + @classmethod def create(cls, config_dict, ctx): config = TrinoConfig.parse_obj(config_dict) diff --git a/metadata-ingestion/tests/integration/trino/test_trino.py b/metadata-ingestion/tests/integration/trino/test_trino.py index 8ab3ed8056e90..4e24b5c887194 100644 --- a/metadata-ingestion/tests/integration/trino/test_trino.py +++ b/metadata-ingestion/tests/integration/trino/test_trino.py @@ -1,4 +1,3 @@ -import re import subprocess import pytest @@ -9,7 +8,7 @@ from datahub.ingestion.run.pipeline import Pipeline from datahub.ingestion.sink.file import FileSinkConfig from datahub.ingestion.source.ge_profiling_config import GEProfilingConfig -from datahub.ingestion.source.sql.trino import TrinoConfig +from datahub.ingestion.source.sql.trino import ConnectorDetail, TrinoConfig from tests.test_helpers import fs_helpers, mce_helpers from tests.test_helpers.docker_helpers import wait_for_port @@ -53,7 +52,6 @@ def loaded_trino(trino_runner): @freeze_time(FROZEN_TIME) -@pytest.mark.xfail def test_trino_ingest( loaded_trino, test_resources_dir, pytestconfig, tmp_path, mock_time ): @@ -73,7 +71,7 @@ def test_trino_ingest( username="foo", schema_pattern=AllowDenyPattern(allow=["^librarydb"]), profile_pattern=AllowDenyPattern( - allow=["library_catalog.librarydb.*"] + allow=["postgresqldb.librarydb.*"] ), profiling=GEProfilingConfig( enabled=True, @@ -89,6 +87,12 @@ def test_trino_ingest( include_field_histogram=True, include_field_sample_values=True, ), + catalog_to_connector_details={ + "postgresqldb": ConnectorDetail( + connector_database="postgres", + platform_instance="local_server", + ) + }, ).dict(), }, "sink": { @@ -169,8 +173,6 @@ def test_trino_hive_ingest( def test_trino_instance_ingest( loaded_trino, test_resources_dir, pytestconfig, tmp_path, mock_time ): - instance = "production_warehouse" - platform = "trino" mce_out_file = "trino_instance_mces.json" events_file = tmp_path / mce_out_file pipeline_config = { @@ -183,6 +185,12 @@ def test_trino_instance_ingest( username="foo", platform_instance="production_warehouse", schema_pattern=AllowDenyPattern(allow=["^db1"]), + catalog_to_connector_details={ + "hivedb": ConnectorDetail( + connector_platform="glue", + platform_instance="local_server", + ) + }, ).dict(), }, "sink": { @@ -197,40 +205,12 @@ def test_trino_instance_ingest( pipeline.pretty_print_summary() pipeline.raise_from_status(raise_warnings=True) - # Assert that all events generated have instance specific urns - urn_pattern = "^" + re.escape( - f"urn:li:dataset:(urn:li:dataPlatform:{platform},{instance}." - ) - assert ( - mce_helpers.assert_mce_entity_urn( - "ALL", - entity_type="dataset", - regex_pattern=urn_pattern, - file=events_file, - ) - >= 0 - ), "There should be at least one match" - - assert ( - mce_helpers.assert_mcp_entity_urn( - "ALL", - entity_type="dataset", - regex_pattern=urn_pattern, - file=events_file, - ) - >= 0 - ), "There should be at least one MCP" - - # all dataset entities emitted must have a dataPlatformInstance aspect emitted - # there must be at least one entity emitted - assert ( - mce_helpers.assert_for_each_entity( - entity_type="dataset", - aspect_name="dataPlatformInstance", - aspect_field_matcher={ - "instance": f"urn:li:dataPlatformInstance:(urn:li:dataPlatform:{platform},{instance})" - }, - file=events_file, - ) - >= 1 + # Verify the output. + mce_helpers.check_golden_file( + pytestconfig, + output_path=events_file, + golden_path=test_resources_dir / "trino_hive_instance_mces_golden.json", + ignore_paths=[ + r"root\[\d+\]\['proposedSnapshot'\]\['com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot'\]\['aspects'\]\[\d+\]\['com.linkedin.pegasus2avro.dataset.DatasetProperties'\]\['customProperties'\]\['transient_lastddltime'\]", + ], ) diff --git a/metadata-ingestion/tests/integration/trino/trino_hive_instance_mces_golden.json b/metadata-ingestion/tests/integration/trino/trino_hive_instance_mces_golden.json new file mode 100644 index 0000000000000..d63995506cb9c --- /dev/null +++ b/metadata-ingestion/tests/integration/trino/trino_hive_instance_mces_golden.json @@ -0,0 +1,2990 @@ +[ +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "json": { + "customProperties": { + "platform": "trino", + "instance": "production_warehouse", + "env": "PROD", + "database": "hivedb" + }, + "name": "hivedb" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Database" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "containerProperties", + "aspect": { + "json": { + "customProperties": { + "platform": "trino", + "instance": "production_warehouse", + "env": "PROD", + "database": "hivedb", + "schema": "db1" + }, + "name": "db1" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Schema" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "another.comment": "This table has no partitions", + "comment": "This table has array of structs", + "numfiles": "1", + "numrows": "1", + "rawdatasize": "32", + "totalsize": "33", + "transient_lastddltime": "1708925463" + }, + "name": "array_struct_test", + "description": "This table has array of structs", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.array_struct_test", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "property_id", + "nullable": true, + "description": "id of property", + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service", + "nullable": true, + "description": "service types and providers", + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "record" + ] + } + } + }, + "nativeDataType": "ARRAY(ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))]))", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))]))\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=string].type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=array].[type=int].provider", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "int" + ] + } + } + }, + "nativeDataType": "ARRAY(INTEGER())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(INTEGER())\"}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "numfiles": "0", + "numrows": "0", + "rawdatasize": "0", + "totalsize": "0", + "transient_lastddltime": "1708925466" + }, + "name": "map_test", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.map_test", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "keyvalue", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=map].[type=string].recordid", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.MapType": { + "keyType": "string", + "valueType": "string" + } + } + }, + "nativeDataType": "MAP(INTEGER(), VARCHAR())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"MAP(INTEGER(), VARCHAR())\", \"key_type\": {\"type\": \"int\", \"native_data_type\": \"INTEGER()\", \"_nullable\": true}, \"key_native_data_type\": \"INTEGER()\"}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.map_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.map_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "numfiles": "0", + "numrows": "0", + "rawdatasize": "0", + "totalsize": "0", + "transient_lastddltime": "1708925466" + }, + "name": "nested_struct_test", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.nested_struct_test", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "property_id", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('type', VARCHAR()), ('provider', ROW([('name', VARCHAR(length=50)), ('id', SMALLINT())]))])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('type', VARCHAR()), ('provider', ROW([('name', VARCHAR(length=50)), ('id', SMALLINT())]))])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=string].type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=struct].provider", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('name', VARCHAR(length=50)), ('id', SMALLINT())])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('name', VARCHAR(length=50)), ('id', SMALLINT())])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=struct].provider.[type=string].name", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR(length=50)", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR(length=50)\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=struct].provider.[type=int].id", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "SMALLINT()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"SMALLINT()\", \"_nullable\": true}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.nested_struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.nested_struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "transient_lastddltime": "1708925457" + }, + "name": "pokes", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.pokes", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "foo", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "bar", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "baz", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.pokes,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.pokes,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "numfiles": "0", + "numrows": "0", + "rawdatasize": "0", + "totalsize": "0", + "transient_lastddltime": "1708925459" + }, + "name": "struct_test", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.struct_test", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "property_id", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=string].type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=array].[type=int].provider", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "int" + ] + } + } + }, + "nativeDataType": "ARRAY(INTEGER())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(INTEGER())\"}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "numfiles": "0", + "totalsize": "0", + "transient_lastddltime": "1708925466" + }, + "name": "struct_test_view_materialized", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.struct_test_view_materialized", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "property_id", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=string].type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].service.[type=array].[type=int].provider", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "int" + ] + } + } + }, + "nativeDataType": "ARRAY(INTEGER())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(INTEGER())\"}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test_view_materialized,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test_view_materialized,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "numfiles": "0", + "numrows": "0", + "rawdatasize": "0", + "totalsize": "0", + "transient_lastddltime": "1708925459" + }, + "name": "_test_table_underscore", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1._test_table_underscore", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "foo", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "bar", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1._test_table_underscore,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1._test_table_underscore,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "column_stats_accurate": "{\"BASIC_STATS\":\"true\"}", + "numfiles": "0", + "numrows": "0", + "rawdatasize": "0", + "totalsize": "0", + "transient_lastddltime": "1708925466" + }, + "name": "union_test", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.union_test", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('tag', SMALLINT()), ('field0', INTEGER()), ('field1', DOUBLE()), ('field2', ARRAY(VARCHAR())), ('field3', ROW([('a', INTEGER()), ('b', VARCHAR())]))])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('tag', SMALLINT()), ('field0', INTEGER()), ('field1', DOUBLE()), ('field2', ARRAY(VARCHAR())), ('field3', ROW([('a', INTEGER()), ('b', VARCHAR())]))])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=int].tag", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "SMALLINT()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"SMALLINT()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=int].field0", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"INTEGER()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=double].field1", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "DOUBLE()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"DOUBLE()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=array].[type=string].field2", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "string" + ] + } + } + }, + "nativeDataType": "ARRAY(VARCHAR())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(VARCHAR())\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=struct].field3", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.RecordType": {} + } + }, + "nativeDataType": "ROW([('a', INTEGER()), ('b', VARCHAR())])", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ROW([('a', INTEGER()), ('b', VARCHAR())])\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=struct].field3.[type=int].a", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"INTEGER()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=struct].foo.[type=struct].field3.[type=string].b", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "Table" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.union_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.union_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "transient_lastddltime": "1708925466", + "view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\"", + "is_view": "True" + }, + "name": "array_struct_test_view", + "tags": [] + } + }, + { + "com.linkedin.pegasus2avro.schema.SchemaMetadata": { + "schemaName": "hivedb.db1.array_struct_test_view", + "platform": "urn:li:dataPlatform:trino", + "version": 0, + "created": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "lastModified": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "hash": "", + "platformSchema": { + "com.linkedin.pegasus2avro.schema.MySqlDDL": { + "tableSchema": "" + } + }, + "fields": [ + { + "fieldPath": "property_id", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.NumberType": {} + } + }, + "nativeDataType": "INTEGER()", + "recursive": false, + "isPartOfKey": false + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "record" + ] + } + } + }, + "nativeDataType": "ARRAY(ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))]))", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(ROW([('type', VARCHAR()), ('provider', ARRAY(INTEGER()))]))\"}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=string].type", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.StringType": {} + } + }, + "nativeDataType": "VARCHAR()", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"VARCHAR()\", \"_nullable\": true}" + }, + { + "fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=array].[type=int].provider", + "nullable": true, + "type": { + "type": { + "com.linkedin.pegasus2avro.schema.ArrayType": { + "nestedType": [ + "int" + ] + } + } + }, + "nativeDataType": "ARRAY(INTEGER())", + "recursive": false, + "isPartOfKey": false, + "jsonProps": "{\"native_data_type\": \"ARRAY(INTEGER())\"}" + } + ] + } + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:trino", + "instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "subTypes", + "aspect": { + "json": { + "typeNames": [ + "View" + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "viewProperties", + "aspect": { + "json": { + "materialized": false, + "viewLogic": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\"", + "viewLanguage": "SQL" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test_view,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "type": "VIEW" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD),property_id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD),property_id)" + ], + "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD),service)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD),service)" + ], + "confidenceScore": 1.0 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:glue,local_server.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)", + "urn": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:trino,production_warehouse)" + }, + { + "id": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b", + "urn": "urn:li:container:f311add3fdc7c16e8a50a63fe1dcce8b" + }, + { + "id": "urn:li:container:46baa6eebd802861e5ee3d043456e171", + "urn": "urn:li:container:46baa6eebd802861e5ee3d043456e171" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-instance-test", + "lastRunId": "no-run-id-provided" + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/trino/trino_hive_mces_golden.json b/metadata-ingestion/tests/integration/trino/trino_hive_mces_golden.json index c43223c68a6b6..3e79c8721486e 100644 --- a/metadata-ingestion/tests/integration/trino/trino_hive_mces_golden.json +++ b/metadata-ingestion/tests/integration/trino/trino_hive_mces_golden.json @@ -231,7 +231,7 @@ "numrows": "1", "rawdatasize": "32", "totalsize": "33", - "transient_lastddltime": "1698223433" + "transient_lastddltime": "1708925463" }, "name": "array_struct_test", "description": "This table has array of structs", @@ -349,6 +349,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", @@ -374,6 +393,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)", @@ -408,7 +471,7 @@ "numrows": "0", "rawdatasize": "0", "totalsize": "0", - "transient_lastddltime": "1698223435" + "transient_lastddltime": "1708925466" }, "name": "map_test", "tags": [] @@ -492,6 +555,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)", @@ -517,6 +599,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)", @@ -551,7 +677,7 @@ "numrows": "0", "rawdatasize": "0", "totalsize": "0", - "transient_lastddltime": "1698223435" + "transient_lastddltime": "1708925466" }, "name": "nested_struct_test", "tags": [] @@ -684,6 +810,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.nested_struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)", @@ -709,6 +854,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.nested_struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)", @@ -738,7 +927,7 @@ { "com.linkedin.pegasus2avro.dataset.DatasetProperties": { "customProperties": { - "transient_lastddltime": "1698223429" + "transient_lastddltime": "1708925457" }, "name": "pokes", "tags": [] @@ -830,6 +1019,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.pokes,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)", @@ -855,6 +1063,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.pokes,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)", @@ -889,7 +1141,7 @@ "numrows": "0", "rawdatasize": "0", "totalsize": "0", - "transient_lastddltime": "1698223431" + "transient_lastddltime": "1708925459" }, "name": "struct_test", "tags": [] @@ -1000,6 +1252,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)", @@ -1027,12 +1298,15 @@ }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test,PROD)", "changeType": "UPSERT", - "aspectName": "container", + "aspectName": "siblings", "aspect": { "json": { - "container": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)" + ], + "primary": false } }, "systemMetadata": { @@ -1042,26 +1316,67 @@ } }, { - "proposedSnapshot": { - "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", - "aspects": [ - { - "com.linkedin.pegasus2avro.common.Status": { - "removed": false - } - }, + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ { - "com.linkedin.pegasus2avro.dataset.DatasetProperties": { - "customProperties": { - "numfiles": "0", - "totalsize": "0", - "transient_lastddltime": "1698223435" - }, - "name": "struct_test_view_materialized", - "tags": [] - } - }, + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "container", + "aspect": { + "json": { + "container": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "proposedSnapshot": { + "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "aspects": [ + { + "com.linkedin.pegasus2avro.common.Status": { + "removed": false + } + }, + { + "com.linkedin.pegasus2avro.dataset.DatasetProperties": { + "customProperties": { + "numfiles": "0", + "totalsize": "0", + "transient_lastddltime": "1708925466" + }, + "name": "struct_test_view_materialized", + "tags": [] + } + }, { "com.linkedin.pegasus2avro.schema.SchemaMetadata": { "schemaName": "hivedb.db1.struct_test_view_materialized", @@ -1167,6 +1482,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test_view_materialized,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", @@ -1192,6 +1526,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test_view_materialized,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)", @@ -1226,7 +1604,7 @@ "numrows": "0", "rawdatasize": "0", "totalsize": "0", - "transient_lastddltime": "1698223431" + "transient_lastddltime": "1708925459" }, "name": "_test_table_underscore", "tags": [] @@ -1306,6 +1684,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1._test_table_underscore,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)", @@ -1331,6 +1728,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1._test_table_underscore,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)", @@ -1365,7 +1806,7 @@ "numrows": "0", "rawdatasize": "0", "totalsize": "0", - "transient_lastddltime": "1698223435" + "transient_lastddltime": "1708925466" }, "name": "union_test", "tags": [] @@ -1529,6 +1970,25 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.union_test,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)", @@ -1554,6 +2014,50 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.union_test,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)", @@ -1583,7 +2087,7 @@ { "com.linkedin.pegasus2avro.dataset.DatasetProperties": { "customProperties": { - "transient_lastddltime": "1698223435", + "transient_lastddltime": "1708925466", "view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\"", "is_view": "True" }, @@ -1722,41 +2226,485 @@ "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)", "changeType": "UPSERT", - "aspectName": "upstreamLineage", + "aspectName": "siblings", "aspect": { "json": { - "upstreams": [ - { - "auditStamp": { - "time": 0, - "actor": "urn:li:corpuser:unknown" - }, - "dataset": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", - "type": "VIEW" - } + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)" ], - "fineGrainedLineages": [ + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),property_id)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),property_id)" - ], - "confidenceScore": 1.0 + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" }, { - "upstreamType": "FIELD_SET", - "upstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),service)" - ], - "downstreamType": "FIELD", - "downstreams": [ - "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),service)" - ], + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", + "type": "VIEW" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),property_id)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),property_id)" + ], "confidenceScore": 1.0 + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),service)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),service)" + ], + "confidenceScore": 1.0 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.map_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.nested_struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.pokes,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.struct_test_view_materialized,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1._test_table_underscore,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-hive-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.union_test,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7", + "urn": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7" + }, + { + "id": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84", + "urn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84" } ] } diff --git a/metadata-ingestion/tests/integration/trino/trino_mces_golden.json b/metadata-ingestion/tests/integration/trino/trino_mces_golden.json index dce10ce377be3..1f03f02fa9408 100644 --- a/metadata-ingestion/tests/integration/trino/trino_mces_golden.json +++ b/metadata-ingestion/tests/integration/trino/trino_mces_golden.json @@ -1,7 +1,7 @@ [ { "entityType": "container", - "entityUrn": "urn:li:container:57aba13b10c1691508600999cd411c25", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", "changeType": "UPSERT", "aspectName": "containerProperties", "aspect": { @@ -9,19 +9,20 @@ "customProperties": { "platform": "trino", "env": "PROD", - "database": "library_catalog" + "database": "postgresqldb" }, - "name": "library_catalog" + "name": "postgresqldb" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:57aba13b10c1691508600999cd411c25", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -31,12 +32,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:57aba13b10c1691508600999cd411c25", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", "changeType": "UPSERT", "aspectName": "dataPlatformInstance", "aspect": { @@ -46,12 +48,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:57aba13b10c1691508600999cd411c25", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -63,12 +66,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:57aba13b10c1691508600999cd411c25", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { @@ -78,12 +82,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "containerProperties", "aspect": { @@ -91,7 +96,7 @@ "customProperties": { "platform": "trino", "env": "PROD", - "database": "library_catalog", + "database": "postgresqldb", "schema": "librarydb" }, "name": "librarydb" @@ -99,12 +104,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "status", "aspect": { @@ -114,12 +120,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "dataPlatformInstance", "aspect": { @@ -129,12 +136,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -146,63 +154,67 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "container", "aspect": { "json": { - "container": "urn:li:container:57aba13b10c1691508600999cd411c25" + "container": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "container", - "entityUrn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { "json": { "path": [ { - "id": "urn:li:container:57aba13b10c1691508600999cd411c25", - "urn": "urn:li:container:57aba13b10c1691508600999cd411c25" + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" } ] } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", "changeType": "UPSERT", "aspectName": "container", "aspect": { "json": { - "container": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "container": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "proposedSnapshot": { "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book,PROD)", + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", "aspects": [ { "com.linkedin.pegasus2avro.common.Status": { @@ -218,7 +230,7 @@ }, { "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "library_catalog.librarydb.book", + "schemaName": "postgresqldb.librarydb.book", "platform": "urn:li:dataPlatform:trino", "version": 0, "created": { @@ -321,12 +333,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -338,52 +351,118 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { "json": { "path": [ { - "id": "urn:li:container:57aba13b10c1691508600999cd411c25", - "urn": "urn:li:container:57aba13b10c1691508600999cd411c25" + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" }, { - "id": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", - "urn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book,PROD)", + "type": "VIEW" } ] } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.issue_history,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", "changeType": "UPSERT", "aspectName": "container", "aspect": { "json": { - "container": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "container": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "proposedSnapshot": { "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.issue_history,PROD)", + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", "aspects": [ { "com.linkedin.pegasus2avro.common.Status": { @@ -399,7 +478,7 @@ }, { "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "library_catalog.librarydb.issue_history", + "schemaName": "postgresqldb.librarydb.issue_history", "platform": "urn:li:dataPlatform:trino", "version": 0, "created": { @@ -473,12 +552,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.issue_history,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -490,52 +570,118 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.issue_history,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.issue_history,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { "json": { "path": [ { - "id": "urn:li:container:57aba13b10c1691508600999cd411c25", - "urn": "urn:li:container:57aba13b10c1691508600999cd411c25" + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" }, { - "id": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", - "urn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } ] } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.member,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.issue_history,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.issue_history,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", "changeType": "UPSERT", "aspectName": "container", "aspect": { "json": { - "container": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "container": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "proposedSnapshot": { "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.member,PROD)", + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", "aspects": [ { "com.linkedin.pegasus2avro.common.Status": { @@ -551,7 +697,7 @@ }, { "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "library_catalog.librarydb.member", + "schemaName": "postgresqldb.librarydb.member", "platform": "urn:li:dataPlatform:trino", "version": 0, "created": { @@ -601,12 +747,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.member,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -618,52 +765,118 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.member,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.member,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { "json": { "path": [ { - "id": "urn:li:container:57aba13b10c1691508600999cd411c25", - "urn": "urn:li:container:57aba13b10c1691508600999cd411c25" + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" }, { - "id": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", - "urn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } ] } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.member,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book_in_circulation,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.member,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", "changeType": "UPSERT", "aspectName": "container", "aspect": { "json": { - "container": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "container": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "proposedSnapshot": { "com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": { - "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book_in_circulation,PROD)", + "urn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", "aspects": [ { "com.linkedin.pegasus2avro.common.Status": { @@ -679,7 +892,7 @@ }, { "com.linkedin.pegasus2avro.schema.SchemaMetadata": { - "schemaName": "library_catalog.librarydb.book_in_circulation", + "schemaName": "postgresqldb.librarydb.book_in_circulation", "platform": "urn:li:dataPlatform:trino", "version": 0, "created": { @@ -777,12 +990,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book_in_circulation,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", "changeType": "UPSERT", "aspectName": "subTypes", "aspect": { @@ -794,36 +1008,101 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book_in_circulation,PROD)" + ], + "primary": true + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book_in_circulation,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", "changeType": "UPSERT", "aspectName": "browsePathsV2", "aspect": { "json": { "path": [ { - "id": "urn:li:container:57aba13b10c1691508600999cd411c25", - "urn": "urn:li:container:57aba13b10c1691508600999cd411c25" + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" }, { - "id": "urn:li:container:8568c0971ce28c183fde0dff7f88e617", - "urn": "urn:li:container:8568c0971ce28c183fde0dff7f88e617" + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" } ] } }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book_in_circulation,PROD)", + "changeType": "UPSERT", + "aspectName": "siblings", + "aspect": { + "json": { + "siblings": [ + "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)" + ], + "primary": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 0, + "actor": "urn:li:corpuser:unknown" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book_in_circulation,PROD)", + "type": "VIEW" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", "changeType": "UPSERT", "aspectName": "datasetProfile", "aspect": { @@ -902,12 +1181,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.issue_history,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", "changeType": "UPSERT", "aspectName": "datasetProfile", "aspect": { @@ -988,12 +1268,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.member,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", "changeType": "UPSERT", "aspectName": "datasetProfile", "aspect": { @@ -1038,12 +1319,13 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } }, { "entityType": "dataset", - "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,library_catalog.librarydb.book_in_circulation,PROD)", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", "changeType": "UPSERT", "aspectName": "datasetProfile", "aspect": { @@ -1130,7 +1412,209 @@ }, "systemMetadata": { "lastObserved": 1632398400000, - "runId": "trino-test" + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.book_in_circulation,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.issue_history,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:postgres,local_server.postgres.librarydb.member,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "container", + "entityUrn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" + }, + { + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.issue_history,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" + }, + { + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.member,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" + }, + { + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:trino,postgresqldb.librarydb.book_in_circulation,PROD)", + "changeType": "UPSERT", + "aspectName": "browsePathsV2", + "aspect": { + "json": { + "path": [ + { + "id": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761", + "urn": "urn:li:container:ad9f7c5e0d4bf83d6278f62271c28761" + }, + { + "id": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c", + "urn": "urn:li:container:2d206e03e435f48a5b8bacf444bf565c" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1632398400000, + "runId": "trino-test", + "lastRunId": "no-run-id-provided" } } ] \ No newline at end of file