mirror of
https://github.com/datahub-project/datahub.git
synced 2025-10-04 21:46:30 +00:00
fix(ingest/sql-common): sql_common to use SqlParsingAggregator (#12220)
This commit is contained in:
parent
6b6d820eea
commit
b79857fd94
@ -838,3 +838,18 @@ class HiveSource(TwoTierSQLAlchemySource):
|
||||
entityUrn=dataset_urn,
|
||||
aspect=view_properties_aspect,
|
||||
).as_workunit()
|
||||
|
||||
if view_definition and self.config.include_view_lineage:
|
||||
default_db = None
|
||||
default_schema = None
|
||||
try:
|
||||
default_db, default_schema = self.get_db_schema(dataset_name)
|
||||
except ValueError:
|
||||
logger.warning(f"Invalid view identifier: {dataset_name}")
|
||||
|
||||
self.aggregator.add_view_definition(
|
||||
view_urn=dataset_urn,
|
||||
view_definition=view_definition,
|
||||
default_db=default_db,
|
||||
default_schema=default_schema,
|
||||
)
|
||||
|
@ -123,6 +123,10 @@ class HiveMetastore(BasicSQLAlchemyConfig):
|
||||
description="Dataset Subtype name to be 'Table' or 'View' Valid options: ['True', 'False']",
|
||||
)
|
||||
|
||||
include_view_lineage: bool = Field(
|
||||
default=False, description="", hidden_from_docs=True
|
||||
)
|
||||
|
||||
include_catalog_name_in_ids: bool = Field(
|
||||
default=False,
|
||||
description="Add the Presto catalog name (e.g. hive) to the generated dataset urns. `urn:li:dataset:(urn:li:dataPlatform:hive,hive.user.logging_events,PROD)` versus `urn:li:dataset:(urn:li:dataPlatform:hive,user.logging_events,PROD)`",
|
||||
@ -160,6 +164,9 @@ class HiveMetastore(BasicSQLAlchemyConfig):
|
||||
@capability(SourceCapability.DELETION_DETECTION, "Enabled via stateful ingestion")
|
||||
@capability(SourceCapability.DATA_PROFILING, "Not Supported", False)
|
||||
@capability(SourceCapability.CLASSIFICATION, "Not Supported", False)
|
||||
@capability(
|
||||
SourceCapability.LINEAGE_COARSE, "View lineage is not supported", supported=False
|
||||
)
|
||||
class HiveMetastoreSource(SQLAlchemySource):
|
||||
"""
|
||||
This plugin extracts the following:
|
||||
|
@ -724,7 +724,7 @@ class SQLServerSource(SQLAlchemySource):
|
||||
):
|
||||
yield from auto_workunit(
|
||||
generate_procedure_lineage(
|
||||
schema_resolver=self.schema_resolver,
|
||||
schema_resolver=self.get_schema_resolver(),
|
||||
procedure=procedure,
|
||||
procedure_job_urn=MSSQLDataJob(entity=procedure).urn,
|
||||
is_temp_table=self.is_temp_table,
|
||||
|
@ -11,7 +11,6 @@ from typing import (
|
||||
Dict,
|
||||
Iterable,
|
||||
List,
|
||||
MutableMapping,
|
||||
Optional,
|
||||
Set,
|
||||
Tuple,
|
||||
@ -36,7 +35,6 @@ from datahub.emitter.mce_builder import (
|
||||
make_tag_urn,
|
||||
)
|
||||
from datahub.emitter.mcp import MetadataChangeProposalWrapper
|
||||
from datahub.emitter.sql_parsing_builder import SqlParsingBuilder
|
||||
from datahub.ingestion.api.common import PipelineContext
|
||||
from datahub.ingestion.api.decorators import capability
|
||||
from datahub.ingestion.api.incremental_lineage_helper import auto_incremental_lineage
|
||||
@ -79,7 +77,6 @@ from datahub.ingestion.source.state.stateful_ingestion_base import (
|
||||
StatefulIngestionSourceBase,
|
||||
)
|
||||
from datahub.metadata.com.linkedin.pegasus2avro.common import StatusClass
|
||||
from datahub.metadata.com.linkedin.pegasus2avro.dataset import UpstreamLineage
|
||||
from datahub.metadata.com.linkedin.pegasus2avro.metadata.snapshot import DatasetSnapshot
|
||||
from datahub.metadata.com.linkedin.pegasus2avro.mxe import MetadataChangeEvent
|
||||
from datahub.metadata.com.linkedin.pegasus2avro.schema import (
|
||||
@ -106,17 +103,11 @@ from datahub.metadata.schema_classes import (
|
||||
GlobalTagsClass,
|
||||
SubTypesClass,
|
||||
TagAssociationClass,
|
||||
UpstreamClass,
|
||||
ViewPropertiesClass,
|
||||
)
|
||||
from datahub.sql_parsing.schema_resolver import SchemaResolver
|
||||
from datahub.sql_parsing.sqlglot_lineage import (
|
||||
SqlParsingResult,
|
||||
sqlglot_lineage,
|
||||
view_definition_lineage_helper,
|
||||
)
|
||||
from datahub.sql_parsing.sql_parsing_aggregator import SqlParsingAggregator
|
||||
from datahub.telemetry import telemetry
|
||||
from datahub.utilities.file_backed_collections import FileBackedDict
|
||||
from datahub.utilities.registries.domain_registry import DomainRegistry
|
||||
from datahub.utilities.sqlalchemy_type_converter import (
|
||||
get_native_data_type_for_sqlalchemy_type,
|
||||
@ -347,17 +338,19 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
)
|
||||
|
||||
self.views_failed_parsing: Set[str] = set()
|
||||
self.schema_resolver: SchemaResolver = SchemaResolver(
|
||||
|
||||
self.discovered_datasets: Set[str] = set()
|
||||
self.aggregator = SqlParsingAggregator(
|
||||
platform=self.platform,
|
||||
platform_instance=self.config.platform_instance,
|
||||
env=self.config.env,
|
||||
graph=self.ctx.graph,
|
||||
generate_lineage=self.include_lineage,
|
||||
generate_usage_statistics=False,
|
||||
generate_operations=False,
|
||||
eager_graph_load=False,
|
||||
)
|
||||
self.discovered_datasets: Set[str] = set()
|
||||
self._view_definition_cache: MutableMapping[str, str]
|
||||
if self.config.use_file_backed_cache:
|
||||
self._view_definition_cache = FileBackedDict[str]()
|
||||
else:
|
||||
self._view_definition_cache = {}
|
||||
self.report.sql_aggregator = self.aggregator.report
|
||||
|
||||
@classmethod
|
||||
def test_connection(cls, config_dict: dict) -> TestConnectionReport:
|
||||
@ -572,36 +565,9 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
profile_requests, profiler, platform=self.platform
|
||||
)
|
||||
|
||||
if self.config.include_view_lineage:
|
||||
yield from self.get_view_lineage()
|
||||
|
||||
def get_view_lineage(self) -> Iterable[MetadataWorkUnit]:
|
||||
builder = SqlParsingBuilder(
|
||||
generate_lineage=True,
|
||||
generate_usage_statistics=False,
|
||||
generate_operations=False,
|
||||
)
|
||||
for dataset_name in self._view_definition_cache.keys():
|
||||
# TODO: Ensure that the lineage generated from the view definition
|
||||
# matches the dataset_name.
|
||||
view_definition = self._view_definition_cache[dataset_name]
|
||||
result = self._run_sql_parser(
|
||||
dataset_name,
|
||||
view_definition,
|
||||
self.schema_resolver,
|
||||
)
|
||||
if result and result.out_tables:
|
||||
# This does not yield any workunits but we use
|
||||
# yield here to execute this method
|
||||
yield from builder.process_sql_parsing_result(
|
||||
result=result,
|
||||
query=view_definition,
|
||||
is_view_ddl=True,
|
||||
include_column_lineage=self.config.include_view_column_lineage,
|
||||
)
|
||||
else:
|
||||
self.views_failed_parsing.add(dataset_name)
|
||||
yield from builder.gen_workunits()
|
||||
# Generate workunit for aggregated SQL parsing results
|
||||
for mcp in self.aggregator.gen_metadata():
|
||||
yield mcp.as_workunit()
|
||||
|
||||
def get_identifier(
|
||||
self, *, schema: str, entity: str, inspector: Inspector, **kwargs: Any
|
||||
@ -760,16 +726,6 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
)
|
||||
dataset_snapshot.aspects.append(dataset_properties)
|
||||
|
||||
if self.config.include_table_location_lineage and location_urn:
|
||||
external_upstream_table = UpstreamClass(
|
||||
dataset=location_urn,
|
||||
type=DatasetLineageTypeClass.COPY,
|
||||
)
|
||||
yield MetadataChangeProposalWrapper(
|
||||
entityUrn=dataset_snapshot.urn,
|
||||
aspect=UpstreamLineage(upstreams=[external_upstream_table]),
|
||||
).as_workunit()
|
||||
|
||||
extra_tags = self.get_extra_tags(inspector, schema, table)
|
||||
pk_constraints: dict = inspector.get_pk_constraint(table, schema)
|
||||
partitions: Optional[List[str]] = self.get_partitions(inspector, schema, table)
|
||||
@ -795,7 +751,7 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
|
||||
dataset_snapshot.aspects.append(schema_metadata)
|
||||
if self._save_schema_to_resolver():
|
||||
self.schema_resolver.add_schema_metadata(dataset_urn, schema_metadata)
|
||||
self.aggregator.register_schema(dataset_urn, schema_metadata)
|
||||
self.discovered_datasets.add(dataset_name)
|
||||
db_name = self.get_db_name(inspector)
|
||||
|
||||
@ -815,6 +771,13 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
),
|
||||
)
|
||||
|
||||
if self.config.include_table_location_lineage and location_urn:
|
||||
self.aggregator.add_known_lineage_mapping(
|
||||
upstream_urn=location_urn,
|
||||
downstream_urn=dataset_snapshot.urn,
|
||||
lineage_type=DatasetLineageTypeClass.COPY,
|
||||
)
|
||||
|
||||
if self.config.domain:
|
||||
assert self.domain_registry
|
||||
yield from get_domain_wu(
|
||||
@ -1089,6 +1052,7 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
self.config.platform_instance,
|
||||
self.config.env,
|
||||
)
|
||||
|
||||
try:
|
||||
columns = inspector.get_columns(view, schema)
|
||||
except KeyError:
|
||||
@ -1108,7 +1072,7 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
canonical_schema=schema_fields,
|
||||
)
|
||||
if self._save_schema_to_resolver():
|
||||
self.schema_resolver.add_schema_metadata(dataset_urn, schema_metadata)
|
||||
self.aggregator.register_schema(dataset_urn, schema_metadata)
|
||||
self.discovered_datasets.add(dataset_name)
|
||||
|
||||
description, properties, _ = self.get_table_properties(inspector, schema, view)
|
||||
@ -1117,7 +1081,18 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
view_definition = self._get_view_definition(inspector, schema, view)
|
||||
properties["view_definition"] = view_definition
|
||||
if view_definition and self.config.include_view_lineage:
|
||||
self._view_definition_cache[dataset_name] = view_definition
|
||||
default_db = None
|
||||
default_schema = None
|
||||
try:
|
||||
default_db, default_schema = self.get_db_schema(dataset_name)
|
||||
except ValueError:
|
||||
logger.warning(f"Invalid view identifier: {dataset_name}")
|
||||
self.aggregator.add_view_definition(
|
||||
view_urn=dataset_urn,
|
||||
view_definition=view_definition,
|
||||
default_db=default_db,
|
||||
default_schema=default_schema,
|
||||
)
|
||||
|
||||
dataset_snapshot = DatasetSnapshot(
|
||||
urn=dataset_urn,
|
||||
@ -1169,48 +1144,9 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
hasattr(self.config, "include_lineage") and self.config.include_lineage
|
||||
)
|
||||
|
||||
def _run_sql_parser(
|
||||
self, view_identifier: str, query: str, schema_resolver: SchemaResolver
|
||||
) -> Optional[SqlParsingResult]:
|
||||
try:
|
||||
database, schema = self.get_db_schema(view_identifier)
|
||||
except ValueError:
|
||||
logger.warning(f"Invalid view identifier: {view_identifier}")
|
||||
return None
|
||||
raw_lineage = sqlglot_lineage(
|
||||
query,
|
||||
schema_resolver=schema_resolver,
|
||||
default_db=database,
|
||||
default_schema=schema,
|
||||
)
|
||||
view_urn = make_dataset_urn_with_platform_instance(
|
||||
self.platform,
|
||||
view_identifier,
|
||||
self.config.platform_instance,
|
||||
self.config.env,
|
||||
)
|
||||
|
||||
if raw_lineage.debug_info.table_error:
|
||||
logger.debug(
|
||||
f"Failed to parse lineage for view {view_identifier}: "
|
||||
f"{raw_lineage.debug_info.table_error}"
|
||||
)
|
||||
self.report.num_view_definitions_failed_parsing += 1
|
||||
self.report.view_definitions_parsing_failures.append(
|
||||
f"Table-level sql parsing error for view {view_identifier}: {raw_lineage.debug_info.table_error}"
|
||||
)
|
||||
return None
|
||||
|
||||
elif raw_lineage.debug_info.column_error:
|
||||
self.report.num_view_definitions_failed_column_parsing += 1
|
||||
self.report.view_definitions_parsing_failures.append(
|
||||
f"Column-level sql parsing error for view {view_identifier}: {raw_lineage.debug_info.column_error}"
|
||||
)
|
||||
else:
|
||||
self.report.num_view_definitions_parsed += 1
|
||||
if raw_lineage.out_tables != [view_urn]:
|
||||
self.report.num_view_definitions_view_urn_mismatch += 1
|
||||
return view_definition_lineage_helper(raw_lineage, view_urn)
|
||||
@property
|
||||
def include_lineage(self):
|
||||
return self.config.include_view_lineage
|
||||
|
||||
def get_db_schema(self, dataset_identifier: str) -> Tuple[Optional[str], str]:
|
||||
database, schema, _view = dataset_identifier.split(".", 2)
|
||||
@ -1411,5 +1347,8 @@ class SQLAlchemySource(StatefulIngestionSourceBase, TestableSource):
|
||||
schema=schema, table=table, partition=partition, custom_sql=custom_sql
|
||||
)
|
||||
|
||||
def get_schema_resolver(self) -> SchemaResolver:
|
||||
return self.aggregator._schema_resolver
|
||||
|
||||
def get_report(self):
|
||||
return self.report
|
||||
|
@ -5,6 +5,7 @@ from datahub.ingestion.glossary.classification_mixin import ClassificationReport
|
||||
from datahub.ingestion.source.state.stale_entity_removal_handler import (
|
||||
StaleEntityRemovalSourceReport,
|
||||
)
|
||||
from datahub.sql_parsing.sql_parsing_aggregator import SqlAggregatorReport
|
||||
from datahub.utilities.lossy_collections import LossyList
|
||||
from datahub.utilities.sqlalchemy_query_combiner import SQLAlchemyQueryCombinerReport
|
||||
from datahub.utilities.stats_collections import TopKDict, int_top_k_dict
|
||||
@ -52,6 +53,7 @@ class SQLSourceReport(
|
||||
num_view_definitions_failed_parsing: int = 0
|
||||
num_view_definitions_failed_column_parsing: int = 0
|
||||
view_definitions_parsing_failures: LossyList[str] = field(default_factory=LossyList)
|
||||
sql_aggregator: Optional[SqlAggregatorReport] = None
|
||||
|
||||
def report_entity_scanned(self, name: str, ent_type: str = "table") -> None:
|
||||
"""
|
||||
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -238,7 +238,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -268,7 +268,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -428,7 +428,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -463,7 +463,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -672,10 +672,15 @@
|
||||
"path": "/name",
|
||||
"value": "nested_struct_test"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258695"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -697,11 +702,6 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/table_type",
|
||||
@ -715,7 +715,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -926,11 +926,6 @@
|
||||
"path": "/customProperties/another.comment",
|
||||
"value": "This table has no partitions"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numRows",
|
||||
@ -943,13 +938,18 @@
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258689"
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298448"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -974,7 +974,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1164,6 +1164,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298442"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1174,11 +1179,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1202,7 +1202,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1386,6 +1386,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298441"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1396,11 +1401,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1424,7 +1424,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1576,7 +1576,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258672"
|
||||
"value": "1735298433"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1591,7 +1591,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1636,6 +1636,19 @@
|
||||
}
|
||||
},
|
||||
"fields": [
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=int].foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=string].baz",
|
||||
"nullable": true,
|
||||
@ -1650,19 +1663,6 @@
|
||||
"isPartitioningKey": true,
|
||||
"jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=int].foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=string].bar",
|
||||
"nullable": true,
|
||||
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -238,7 +238,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -268,7 +268,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -428,7 +428,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -463,7 +463,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -672,10 +672,15 @@
|
||||
"path": "/name",
|
||||
"value": "nested_struct_test"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258695"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -697,11 +702,6 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/table_type",
|
||||
@ -715,7 +715,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -926,11 +926,6 @@
|
||||
"path": "/customProperties/another.comment",
|
||||
"value": "This table has no partitions"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numRows",
|
||||
@ -943,13 +938,18 @@
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258689"
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298448"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -974,7 +974,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1164,6 +1164,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298442"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1174,11 +1179,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1202,7 +1202,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1386,6 +1386,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298441"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1396,11 +1401,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1424,7 +1424,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1576,7 +1576,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258672"
|
||||
"value": "1735298433"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1591,7 +1591,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1636,6 +1636,19 @@
|
||||
}
|
||||
},
|
||||
"fields": [
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=int].foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=string].baz",
|
||||
"nullable": true,
|
||||
@ -1650,19 +1663,6 @@
|
||||
"isPartitioningKey": true,
|
||||
"jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=int].foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=string].bar",
|
||||
"nullable": true,
|
||||
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:1cfce89b5a05e1da5092d88ad9eb4589"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "hive-metastore-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:9ba2e350c97c893a91bcaee4838cdcae",
|
||||
@ -238,7 +238,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -268,7 +268,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -428,7 +428,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258696"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -463,7 +463,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -672,10 +672,15 @@
|
||||
"path": "/name",
|
||||
"value": "nested_struct_test"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258695"
|
||||
"value": "1735298453"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -697,11 +702,6 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/table_type",
|
||||
@ -715,7 +715,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -926,11 +926,6 @@
|
||||
"path": "/customProperties/another.comment",
|
||||
"value": "This table has no partitions"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numRows",
|
||||
@ -943,13 +938,18 @@
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258689"
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/totalSize",
|
||||
"value": "33"
|
||||
"path": "/customProperties/numFiles",
|
||||
"value": "1"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298448"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -974,7 +974,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1164,6 +1164,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298442"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1174,11 +1179,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1202,7 +1202,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1386,6 +1386,11 @@
|
||||
"path": "/customProperties/numRows",
|
||||
"value": "0"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1735298441"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/numFiles",
|
||||
@ -1396,11 +1401,6 @@
|
||||
"path": "/customProperties/COLUMN_STATS_ACCURATE",
|
||||
"value": "{\"BASIC_STATS\":\"true\"}"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258680"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/rawDataSize",
|
||||
@ -1424,7 +1424,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
}
|
||||
]
|
||||
},
|
||||
@ -1576,7 +1576,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/transient_lastDdlTime",
|
||||
"value": "1715258672"
|
||||
"value": "1735298433"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1591,7 +1591,7 @@
|
||||
{
|
||||
"op": "add",
|
||||
"path": "/customProperties/create_date",
|
||||
"value": "2024-05-09"
|
||||
"value": "2024-12-27"
|
||||
},
|
||||
{
|
||||
"op": "add",
|
||||
@ -1636,6 +1636,19 @@
|
||||
}
|
||||
},
|
||||
"fields": [
|
||||
{
|
||||
"fieldPath": "foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "baz",
|
||||
"nullable": true,
|
||||
@ -1650,19 +1663,6 @@
|
||||
"isPartitioningKey": true,
|
||||
"jsonProps": "{\"native_data_type\": \"string\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "foo",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.NumberType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "int",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"int\", \"_nullable\": true}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "bar",
|
||||
"nullable": true,
|
||||
|
@ -118,7 +118,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/_test_table_underscore",
|
||||
@ -128,7 +128,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166683",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218716",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -268,7 +268,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/array_struct_test",
|
||||
@ -280,7 +280,7 @@
|
||||
"Table Parameters: numRows": "1",
|
||||
"Table Parameters: rawDataSize": "32",
|
||||
"Table Parameters: totalSize": "33",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166687",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218720",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -458,11 +458,11 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Table Type:": "VIRTUAL_VIEW",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "null",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -608,6 +608,187 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:ded36d15fcfbbb939830549697122661"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": {
|
||||
"urn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"aspects": [
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Status": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Table Type:": "VIRTUAL_VIEW",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "null",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
"Compressed:": "No",
|
||||
"Num Buckets:": "-1",
|
||||
"Bucket Columns:": "[]",
|
||||
"Sort Columns:": "[]",
|
||||
"View Original Text:": "select * from db1.array_struct_test_view",
|
||||
"View Expanded Text:": "select `array_struct_test_view`.`property_id`, `array_struct_test_view`.`service` from `db1`.`array_struct_test_view`",
|
||||
"View Rewrite Enabled:": "No"
|
||||
},
|
||||
"name": "array_struct_test_view_2",
|
||||
"tags": []
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.schema.SchemaMetadata": {
|
||||
"schemaName": "db1.array_struct_test_view_2",
|
||||
"platform": "urn:li:dataPlatform:hive",
|
||||
"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": "int",
|
||||
"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<struct<type:string,provider:array<int>>>",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"array<struct<type:string,provider:array<int>>>\"}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=string].type",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.StringType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "string",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"string\", \"_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<int>",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"array<int>\"}"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "subTypes",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"typeNames": [
|
||||
"Table"
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "browsePathsV2",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"path": [
|
||||
{
|
||||
"id": "urn:li:container:ded36d15fcfbbb939830549697122661",
|
||||
"urn": "urn:li:container:ded36d15fcfbbb939830549697122661"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)",
|
||||
@ -639,7 +820,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:31 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:04 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/map_test",
|
||||
@ -649,7 +830,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166691",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218724",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -793,7 +974,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/nested_struct_test",
|
||||
@ -803,7 +984,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -996,7 +1177,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:20 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:53 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/pokes",
|
||||
@ -1006,7 +1187,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "5812",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166680",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218713",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -1158,7 +1339,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/struct_test",
|
||||
@ -1168,7 +1349,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166683",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218716",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -1339,14 +1520,14 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:02 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/struct_test_view_materialized",
|
||||
"Table Type:": "MATERIALIZED_VIEW",
|
||||
"Table Parameters: numFiles": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218722",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.ql.io.orc.OrcSerde",
|
||||
"InputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
|
||||
@ -1519,7 +1700,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/union_test",
|
||||
@ -1529,7 +1710,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.ql.io.orc.OrcSerde",
|
||||
"InputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
|
||||
@ -1756,6 +1937,24 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "viewProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"materialized": false,
|
||||
"viewLogic": "CREATE VIEW `db1.array_struct_test_view_2` AS select `array_struct_test_view`.`property_id`, `array_struct_test_view`.`service` from `db1`.`array_struct_test_view`",
|
||||
"viewLanguage": "SQL"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:8cc876554899e33efe67c389aaf29c4b",
|
||||
@ -1875,7 +2074,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db2",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:22 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:55 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db2.db/pokes",
|
||||
@ -1884,7 +2083,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "5812",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166683",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218716",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -2080,5 +2279,307 @@
|
||||
"runId": "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": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),property_id)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),service)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW `db1.array_struct_test_view` AS\nSELECT\n `array_struct_test`.`property_id`,\n `array_struct_test`.`service`\nFROM `db1`.`array_struct_test`",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:hive"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),property_id)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW `db1.array_struct_test_view_2` AS\nSELECT\n `array_struct_test_view`.`property_id`,\n `array_struct_test_view`.`service`\nFROM `db1`.`array_struct_test_view`",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:hive"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -118,7 +118,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/_test_table_underscore",
|
||||
@ -128,7 +128,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166683",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218716",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -268,7 +268,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/array_struct_test",
|
||||
@ -280,7 +280,7 @@
|
||||
"Table Parameters: numRows": "1",
|
||||
"Table Parameters: rawDataSize": "32",
|
||||
"Table Parameters: totalSize": "33",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166687",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218720",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -458,11 +458,11 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Table Type:": "VIRTUAL_VIEW",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "null",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -608,6 +608,187 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:ded36d15fcfbbb939830549697122661"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": {
|
||||
"urn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"aspects": [
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Status": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Table Type:": "VIRTUAL_VIEW",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "null",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
"Compressed:": "No",
|
||||
"Num Buckets:": "-1",
|
||||
"Bucket Columns:": "[]",
|
||||
"Sort Columns:": "[]",
|
||||
"View Original Text:": "select * from db1.array_struct_test_view",
|
||||
"View Expanded Text:": "select `array_struct_test_view`.`property_id`, `array_struct_test_view`.`service` from `db1`.`array_struct_test_view`",
|
||||
"View Rewrite Enabled:": "No"
|
||||
},
|
||||
"name": "array_struct_test_view_2",
|
||||
"tags": []
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.schema.SchemaMetadata": {
|
||||
"schemaName": "db1.array_struct_test_view_2",
|
||||
"platform": "urn:li:dataPlatform:hive",
|
||||
"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": "int",
|
||||
"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<struct<type:string,provider:array<int>>>",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"array<struct<type:string,provider:array<int>>>\"}"
|
||||
},
|
||||
{
|
||||
"fieldPath": "[version=2.0].[type=struct].[type=array].[type=struct].service.[type=string].type",
|
||||
"nullable": true,
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.StringType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "string",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"string\", \"_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<int>",
|
||||
"recursive": false,
|
||||
"isPartOfKey": false,
|
||||
"jsonProps": "{\"native_data_type\": \"array<int>\"}"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "subTypes",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"typeNames": [
|
||||
"Table"
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "browsePathsV2",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"path": [
|
||||
{
|
||||
"id": "urn:li:container:ded36d15fcfbbb939830549697122661",
|
||||
"urn": "urn:li:container:ded36d15fcfbbb939830549697122661"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.map_test,PROD)",
|
||||
@ -639,7 +820,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:31 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:04 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/map_test",
|
||||
@ -649,7 +830,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166691",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218724",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -793,7 +974,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/nested_struct_test",
|
||||
@ -803,7 +984,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -996,7 +1177,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:20 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:53 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/pokes",
|
||||
@ -1006,7 +1187,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "5812",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166680",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218713",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -1158,7 +1339,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:23 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:11:56 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/struct_test",
|
||||
@ -1168,7 +1349,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166683",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218716",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe",
|
||||
"InputFormat:": "org.apache.hadoop.mapred.TextInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat",
|
||||
@ -1339,14 +1520,14 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:02 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/struct_test_view_materialized",
|
||||
"Table Type:": "MATERIALIZED_VIEW",
|
||||
"Table Parameters: numFiles": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218722",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.ql.io.orc.OrcSerde",
|
||||
"InputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
|
||||
@ -1519,7 +1700,7 @@
|
||||
"customProperties": {
|
||||
"Database:": "db1",
|
||||
"Owner:": "root",
|
||||
"CreateTime:": "Tue Aug 20 15:11:30 UTC 2024",
|
||||
"CreateTime:": "Thu Dec 26 13:12:03 UTC 2024",
|
||||
"LastAccessTime:": "UNKNOWN",
|
||||
"Retention:": "0",
|
||||
"Location:": "hdfs://namenode:8020/user/hive/warehouse/db1.db/union_test",
|
||||
@ -1529,7 +1710,7 @@
|
||||
"Table Parameters: numRows": "0",
|
||||
"Table Parameters: rawDataSize": "0",
|
||||
"Table Parameters: totalSize": "0",
|
||||
"Table Parameters: transient_lastDdlTime": "1724166690",
|
||||
"Table Parameters: transient_lastDdlTime": "1735218723",
|
||||
"SerDe Library:": "org.apache.hadoop.hive.ql.io.orc.OrcSerde",
|
||||
"InputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat",
|
||||
"OutputFormat:": "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat",
|
||||
@ -1755,5 +1936,325 @@
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "viewProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"materialized": false,
|
||||
"viewLogic": "CREATE VIEW `db1.array_struct_test_view_2` AS select `array_struct_test_view`.`property_id`, `array_struct_test_view`.`service` from `db1`.`array_struct_test_view`",
|
||||
"viewLanguage": "SQL"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "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": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),property_id)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),service)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW `db1.array_struct_test_view` AS\nSELECT\n `array_struct_test`.`property_id`,\n `array_struct_test`.`service`\nFROM `db1`.`array_struct_test`",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:hive"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),property_id)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 0.35,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW `db1.array_struct_test_view_2` AS\nSELECT\n `array_struct_test_view`.`property_id`,\n `array_struct_test_view`.`service`\nFROM `db1`.`array_struct_test_view`",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:hive,db1.array_struct_test_view_2,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:hive"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Ahive%2Cdb1.array_struct_test_view_2%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -42,6 +42,8 @@ select * from test_data;
|
||||
CREATE MATERIALIZED VIEW db1.struct_test_view_materialized as select * from db1.struct_test;
|
||||
CREATE VIEW db1.array_struct_test_view as select * from db1.array_struct_test;
|
||||
|
||||
CREATE VIEW db1.array_struct_test_view_2 as select * from db1.array_struct_test_view;
|
||||
|
||||
CREATE TABLE IF NOT EXISTS db1.nested_struct_test
|
||||
(
|
||||
property_id INT,
|
||||
|
@ -1550,8 +1550,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `metadata_index_view` AS select `metadata_index`.`id` AS `id`,`metadata_index`.`urn` AS `urn`,`metadata_index`.`path` AS `path`,`metadata_index`.`doubleVal` AS `doubleVal` from `metadata_index`",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`localhost` SQL SECURITY DEFINER VIEW `metadata_index_view` AS select `metadata_index`.`id` AS `id`,`metadata_index`.`urn` AS `urn`,`metadata_index`.`path` AS `path`,`metadata_index`.`doubleVal` AS `doubleVal` from `metadata_index`"
|
||||
},
|
||||
"name": "metadata_index_view",
|
||||
"tags": []
|
||||
@ -2701,25 +2701,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),doubleVal)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),doubleVal)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -2729,18 +2723,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),id)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),path)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),path)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -2751,7 +2735,32 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),urn)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),path)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),path)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),doubleVal)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),doubleVal)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -2762,6 +2771,113 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE ALGORITHM=UNDEFINED\nDEFINER=\"root\"@\"localhost\"\nSQL SECURITY DEFINER VIEW `metadata_index_view` AS\nSELECT\n `metadata_index`.`id` AS `id`,\n `metadata_index`.`urn` AS `urn`,\n `metadata_index`.`path` AS `path`,\n `metadata_index`.`doubleVal` AS `doubleVal`\nFROM `metadata_index`",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "mysql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),doubleVal)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),path)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),path)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mysql,metagalaxy.metadata_index_view,PROD),doubleVal)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "mysql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mysql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "mysql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amysql%2Cmetagalaxy.metadata_index_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "mysql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "glossaryTerm",
|
||||
"entityUrn": "urn:li:glossaryTerm:Email_Address",
|
||||
|
@ -17,7 +17,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -33,7 +33,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -49,7 +49,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -67,7 +67,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -83,7 +83,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -99,7 +99,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -122,7 +122,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -138,7 +138,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -154,7 +154,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -172,7 +172,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -193,7 +193,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -209,7 +209,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -272,7 +272,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -290,7 +290,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -315,7 +315,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -331,7 +331,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -394,7 +394,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -412,7 +412,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -437,7 +437,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -453,7 +453,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -470,8 +470,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table"
|
||||
},
|
||||
"name": "view1",
|
||||
"description": "Some mock comment here ...",
|
||||
@ -519,7 +519,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -537,7 +537,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -555,7 +555,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -580,7 +580,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -596,7 +596,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -619,7 +619,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -635,7 +635,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -651,7 +651,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -669,7 +669,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -690,7 +690,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -706,7 +706,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -769,7 +769,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -787,7 +787,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -812,7 +812,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -828,7 +828,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -891,7 +891,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -909,7 +909,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -934,7 +934,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -950,7 +950,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -967,8 +967,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table"
|
||||
},
|
||||
"name": "view1",
|
||||
"description": "Some mock comment here ...",
|
||||
@ -1016,7 +1016,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1034,7 +1034,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1052,7 +1052,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1077,7 +1077,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1091,11 +1091,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema1.mock_table,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
@ -1108,7 +1113,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema1.view1,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -1119,14 +1125,94 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema1.view1,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW mock_view AS\nSELECT\n mock_column1,\n mock_column2\nFROM mock_table",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema1.mock_table,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema1.mock_table,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema1.mock_table,PROD),MOCK_COLUMN2)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema1.view1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema1.view1,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema1.view1,PROD),MOCK_COLUMN2)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:oracle"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1140,11 +1226,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema2.mock_table,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
@ -1157,7 +1248,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema2.view1,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -1168,14 +1260,126 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema2.view1,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-uzcdxn",
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW mock_view AS\nSELECT\n mock_column1,\n mock_column2\nFROM mock_table",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema2.mock_table,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema2.mock_table,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,oradoc.schema2.mock_table,PROD),MOCK_COLUMN2)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema2.view1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema2.view1,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,OraDoc.schema2.view1,PROD),MOCK_COLUMN2)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:oracle"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2COraDoc.schema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-38ppfw",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
|
@ -17,7 +17,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -33,7 +33,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -49,7 +49,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -67,7 +67,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -83,7 +83,23 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:937a38ee28b69ecae38665c5e842d0ad",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0e497517e191d344b0c403231bc708d0"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -106,7 +122,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -122,7 +138,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -138,7 +154,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -156,23 +172,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:937a38ee28b69ecae38665c5e842d0ad",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0e497517e191d344b0c403231bc708d0"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -193,7 +193,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -209,7 +209,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -272,7 +272,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -290,7 +290,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -315,7 +315,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -331,7 +331,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -394,7 +394,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -412,7 +412,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -437,7 +437,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -453,7 +453,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -470,8 +470,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table"
|
||||
},
|
||||
"name": "view1",
|
||||
"description": "Some mock comment here ...",
|
||||
@ -519,7 +519,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -537,7 +537,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -555,7 +555,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -580,7 +580,23 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:1965527855ae77f259a8ddea2b8eed2f",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0e497517e191d344b0c403231bc708d0"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -603,7 +619,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -619,7 +635,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -635,7 +651,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -653,23 +669,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:1965527855ae77f259a8ddea2b8eed2f",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0e497517e191d344b0c403231bc708d0"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -690,7 +690,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -706,7 +706,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -769,7 +769,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -787,7 +787,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -812,7 +812,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -828,7 +828,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -891,7 +891,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -909,7 +909,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -934,7 +934,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -950,7 +950,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -967,8 +967,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW mock_view AS\n SELECT\n mock_column1,\n mock_column2\n FROM mock_table"
|
||||
},
|
||||
"name": "view1",
|
||||
"description": "Some mock comment here ...",
|
||||
@ -1016,7 +1016,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1034,7 +1034,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1052,7 +1052,7 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
@ -1077,7 +1077,309 @@
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00",
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW mock_view AS\nSELECT\n mock_column1,\n mock_column2\nFROM mock_table",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN2)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema1.view1,PROD),MOCK_COLUMN2)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:oracle"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "upstreamLineage",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD)",
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD),MOCK_COLUMN1)"
|
||||
],
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD),MOCK_COLUMN2)"
|
||||
],
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW mock_view AS\nSELECT\n mock_column1,\n mock_column2\nFROM mock_table",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1643871600000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,mock_table,PROD),MOCK_COLUMN2)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD),MOCK_COLUMN1)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:oracle,schema2.view1,PROD),MOCK_COLUMN2)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:oracle"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema1.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Aoracle%2Cschema2.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1643871600000,
|
||||
"runId": "oracle-2022_02_03-07_00_00-ss8owb",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:a208486b83be39fa411922e07701d984",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0202f800c992262c01ae6bbd5ee313f7"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:a208486b83be39fa411922e07701d984",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:a208486b83be39fa411922e07701d984",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:0202f800c992262c01ae6bbd5ee313f7"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:a208486b83be39fa411922e07701d984",
|
||||
@ -285,6 +285,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:a6097853edba03be190d99ece4b307ff"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
@ -358,22 +374,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:a6097853edba03be190d99ece4b307ff"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
@ -640,8 +640,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": " SELECT metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\n FROM metadata_aspect_v2\n WHERE (metadata_aspect_v2.version = 0);",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": " SELECT metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\n FROM metadata_aspect_v2\n WHERE (metadata_aspect_v2.version = 0);"
|
||||
},
|
||||
"name": "metadata_aspect_view",
|
||||
"tags": []
|
||||
@ -856,25 +856,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1646575200000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -884,7 +878,20 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),urn)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -894,5 +901,100 @@
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "SELECT\n metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\nFROM metadata_aspect_v2\nWHERE\n (\n metadata_aspect_v2.version = 0\n )",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1646575200000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:postgres"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:a6097853edba03be190d99ece4b307ff"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:a6097853edba03be190d99ece4b307ff"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:51904fc8cd5cc729bc630decff284525",
|
||||
@ -464,8 +464,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": " SELECT metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\n FROM metadata_aspect_v2\n WHERE (metadata_aspect_v2.version = 0);",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": " SELECT metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\n FROM metadata_aspect_v2\n WHERE (metadata_aspect_v2.version = 0);"
|
||||
},
|
||||
"name": "metadata_aspect_view",
|
||||
"tags": []
|
||||
@ -622,25 +622,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1646575200000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -650,7 +644,20 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),urn)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -661,6 +668,101 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "SELECT\n metadata_aspect_v2.urn,\n metadata_aspect_v2.aspect\nFROM metadata_aspect_v2\nWHERE\n (\n metadata_aspect_v2.version = 0\n )",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1646575200000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),aspect)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_v2,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),urn)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:postgres,postgrestest.public.metadata_aspect_view,PROD),aspect)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:postgres"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Apostgres%2Cpostgrestest.public.metadata_aspect_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1646575200000,
|
||||
"runId": "postgres-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "glossaryTerm",
|
||||
"entityUrn": "urn:li:glossaryTerm:URN",
|
||||
|
@ -113,11 +113,11 @@
|
||||
"aspect": {
|
||||
"json": {
|
||||
"customProperties": {
|
||||
"job_id": "c2d77890-83ba-435f-879b-1c77fa38dd47",
|
||||
"job_id": "a06cfdca-b65e-42de-8db2-8c21c183c5dd",
|
||||
"job_name": "Weekly Demo Data Backup",
|
||||
"description": "No description available.",
|
||||
"date_created": "2024-12-05 16:44:43.910000",
|
||||
"date_modified": "2024-12-05 16:44:44.043000",
|
||||
"date_created": "2024-12-26 12:03:35.420000",
|
||||
"date_modified": "2024-12-26 12:03:35.590000",
|
||||
"step_id": "1",
|
||||
"step_name": "Set database to read only",
|
||||
"subsystem": "TSQL",
|
||||
@ -2103,8 +2103,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n"
|
||||
},
|
||||
"name": "PersonsView",
|
||||
"tags": []
|
||||
@ -2282,8 +2282,8 @@
|
||||
"code": "CREATE PROCEDURE [Foo].[Proc.With.SpecialChar] @ID INT\nAS\n SELECT @ID AS ThatDB;\n",
|
||||
"input parameters": "['@ID']",
|
||||
"parameter @ID": "{'type': 'int'}",
|
||||
"date_created": "2024-12-05 16:44:43.800000",
|
||||
"date_modified": "2024-12-05 16:44:43.800000"
|
||||
"date_created": "2024-12-26 12:03:35.230000",
|
||||
"date_modified": "2024-12-26 12:03:35.230000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.Proc.With.SpecialChar",
|
||||
@ -2310,8 +2310,8 @@
|
||||
"depending_on_procedure": "{}",
|
||||
"code": "CREATE PROCEDURE [Foo].[NewProc]\n AS\n BEGIN\n --insert into items table from salesreason table\n insert into Foo.Items (ID, ItemName)\n SELECT TempID, Name\n FROM Foo.SalesReason;\n\n\n IF OBJECT_ID('Foo.age_dist', 'U') IS NULL\n BEGIN\n -- Create and populate if table doesn't exist\n SELECT Age, COUNT(*) as Count\n INTO Foo.age_dist\n FROM Foo.Persons\n GROUP BY Age\n END\n ELSE\n BEGIN\n -- Update existing table\n TRUNCATE TABLE Foo.age_dist;\n\n INSERT INTO Foo.age_dist (Age, Count)\n SELECT Age, COUNT(*) as Count\n FROM Foo.Persons\n GROUP BY Age\n END\n\n SELECT ID, Age INTO #TEMPTABLE FROM NewData.FooNew.PersonsNew\n \n UPDATE DemoData.Foo.Persons\n SET Age = t.Age\n FROM DemoData.Foo.Persons p\n JOIN #TEMPTABLE t ON p.ID = t.ID\n\n END\n",
|
||||
"input parameters": "[]",
|
||||
"date_created": "2024-12-05 16:44:43.803000",
|
||||
"date_modified": "2024-12-05 16:44:43.803000"
|
||||
"date_created": "2024-12-26 12:03:35.237000",
|
||||
"date_modified": "2024-12-26 12:03:35.237000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.NewProc",
|
||||
@ -4427,8 +4427,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW FooNew.View1 AS\nSELECT LastName, FirstName\nFROM FooNew.PersonsNew\nWHERE Age > 18\n",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW FooNew.View1 AS\nSELECT LastName, FirstName\nFROM FooNew.PersonsNew\nWHERE Age > 18\n"
|
||||
},
|
||||
"name": "View1",
|
||||
"tags": []
|
||||
@ -4891,11 +4891,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -4906,6 +4911,73 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW Foo.PersonsView AS\nSELECT\n *\nFROM Foo.Persons",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214618898,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,DemoData.Foo.PersonsView,PROD)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD)",
|
||||
@ -4916,25 +4988,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),firstname)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD),firstname)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -4944,7 +5010,20 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD),lastname)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),firstname)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD),firstname)"
|
||||
],
|
||||
"confidenceScore": 0.2,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -4955,6 +5034,85 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW FooNew.View1 AS\nSELECT\n LastName,\n FirstName\nFROM FooNew.PersonsNew\nWHERE\n Age > 18",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214618906,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),firstname)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),lastname)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD),lastname)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,NewData.FooNew.View1,PROD),firstname)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataFlow",
|
||||
"entityUrn": "urn:li:dataFlow:(mssql,DemoData.Foo.stored_procedures,PROD)",
|
||||
@ -5034,5 +5192,37 @@
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CNewData.FooNew.View1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -113,11 +113,11 @@
|
||||
"aspect": {
|
||||
"json": {
|
||||
"customProperties": {
|
||||
"job_id": "c2d77890-83ba-435f-879b-1c77fa38dd47",
|
||||
"job_id": "a06cfdca-b65e-42de-8db2-8c21c183c5dd",
|
||||
"job_name": "Weekly Demo Data Backup",
|
||||
"description": "No description available.",
|
||||
"date_created": "2024-12-05 16:44:43.910000",
|
||||
"date_modified": "2024-12-05 16:44:44.043000",
|
||||
"date_created": "2024-12-26 12:03:35.420000",
|
||||
"date_modified": "2024-12-26 12:03:35.590000",
|
||||
"step_id": "1",
|
||||
"step_name": "Set database to read only",
|
||||
"subsystem": "TSQL",
|
||||
@ -2103,8 +2103,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n"
|
||||
},
|
||||
"name": "PersonsView",
|
||||
"tags": []
|
||||
@ -2282,8 +2282,8 @@
|
||||
"code": "CREATE PROCEDURE [Foo].[Proc.With.SpecialChar] @ID INT\nAS\n SELECT @ID AS ThatDB;\n",
|
||||
"input parameters": "['@ID']",
|
||||
"parameter @ID": "{'type': 'int'}",
|
||||
"date_created": "2024-12-05 16:44:43.800000",
|
||||
"date_modified": "2024-12-05 16:44:43.800000"
|
||||
"date_created": "2024-12-26 12:03:35.230000",
|
||||
"date_modified": "2024-12-26 12:03:35.230000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.Proc.With.SpecialChar",
|
||||
@ -2638,11 +2638,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -2653,6 +2658,73 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW Foo.PersonsView AS\nSELECT\n *\nFROM Foo.Persons",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214621644,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,DemoData.Foo.PersonsView,PROD)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataFlow",
|
||||
"entityUrn": "urn:li:dataFlow:(mssql,DemoData.Foo.stored_procedures,PROD)",
|
||||
@ -2716,5 +2788,21 @@
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2CDemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -137,11 +137,11 @@
|
||||
"aspect": {
|
||||
"json": {
|
||||
"customProperties": {
|
||||
"job_id": "b8907be7-52f5-4df4-a870-f4fe0679ec45",
|
||||
"job_id": "a06cfdca-b65e-42de-8db2-8c21c183c5dd",
|
||||
"job_name": "Weekly Demo Data Backup",
|
||||
"description": "No description available.",
|
||||
"date_created": "2024-12-19 12:34:45.843000",
|
||||
"date_modified": "2024-12-19 12:34:46.017000",
|
||||
"date_created": "2024-12-26 12:03:35.420000",
|
||||
"date_modified": "2024-12-26 12:03:35.590000",
|
||||
"step_id": "1",
|
||||
"step_name": "Set database to read only",
|
||||
"subsystem": "TSQL",
|
||||
@ -2532,8 +2532,8 @@
|
||||
"code": "CREATE PROCEDURE [Foo].[Proc.With.SpecialChar] @ID INT\nAS\n SELECT @ID AS ThatDB;\n",
|
||||
"input parameters": "['@ID']",
|
||||
"parameter @ID": "{'type': 'int'}",
|
||||
"date_created": "2024-12-19 12:34:45.660000",
|
||||
"date_modified": "2024-12-19 12:34:45.660000"
|
||||
"date_created": "2024-12-26 12:03:35.230000",
|
||||
"date_modified": "2024-12-26 12:03:35.230000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.Proc.With.SpecialChar",
|
||||
@ -2577,8 +2577,8 @@
|
||||
"depending_on_procedure": "{}",
|
||||
"code": "CREATE PROCEDURE [Foo].[NewProc]\n AS\n BEGIN\n --insert into items table from salesreason table\n insert into Foo.Items (ID, ItemName)\n SELECT TempID, Name\n FROM Foo.SalesReason;\n\n\n IF OBJECT_ID('Foo.age_dist', 'U') IS NULL\n BEGIN\n -- Create and populate if table doesn't exist\n SELECT Age, COUNT(*) as Count\n INTO Foo.age_dist\n FROM Foo.Persons\n GROUP BY Age\n END\n ELSE\n BEGIN\n -- Update existing table\n TRUNCATE TABLE Foo.age_dist;\n\n INSERT INTO Foo.age_dist (Age, Count)\n SELECT Age, COUNT(*) as Count\n FROM Foo.Persons\n GROUP BY Age\n END\n\n SELECT ID, Age INTO #TEMPTABLE FROM NewData.FooNew.PersonsNew\n \n UPDATE DemoData.Foo.Persons\n SET Age = t.Age\n FROM DemoData.Foo.Persons p\n JOIN #TEMPTABLE t ON p.ID = t.ID\n\n END\n",
|
||||
"input parameters": "[]",
|
||||
"date_created": "2024-12-19 12:34:45.667000",
|
||||
"date_modified": "2024-12-19 12:34:45.667000"
|
||||
"date_created": "2024-12-26 12:03:35.237000",
|
||||
"date_modified": "2024-12-26 12:03:35.237000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.NewProc",
|
||||
@ -2968,11 +2968,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,my-instance.demodata.foo.persons,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cmy-instance.DemoData.Foo.PersonsView%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -2983,6 +2988,73 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cmy-instance.DemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW Foo.PersonsView AS\nSELECT\n *\nFROM Foo.Persons",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214620908,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cmy-instance.DemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,my-instance.demodata.foo.persons,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,my-instance.DemoData.Foo.PersonsView,PROD)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cmy-instance.DemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataFlow",
|
||||
"entityUrn": "urn:li:dataFlow:(mssql,my-instance.DemoData.Foo.stored_procedures,PROD)",
|
||||
@ -3062,5 +3134,21 @@
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cmy-instance.DemoData.Foo.PersonsView%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -113,11 +113,11 @@
|
||||
"aspect": {
|
||||
"json": {
|
||||
"customProperties": {
|
||||
"job_id": "4130c37d-146c-43da-a671-dd9a413a44b3",
|
||||
"job_id": "a06cfdca-b65e-42de-8db2-8c21c183c5dd",
|
||||
"job_name": "Weekly Demo Data Backup",
|
||||
"description": "No description available.",
|
||||
"date_created": "2024-11-22 12:58:03.260000",
|
||||
"date_modified": "2024-11-22 12:58:03.440000",
|
||||
"date_created": "2024-12-26 12:03:35.420000",
|
||||
"date_modified": "2024-12-26 12:03:35.590000",
|
||||
"step_id": "1",
|
||||
"step_name": "Set database to read only",
|
||||
"subsystem": "TSQL",
|
||||
@ -2103,8 +2103,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW Foo.PersonsView AS SELECT * FROM Foo.Persons;\n"
|
||||
},
|
||||
"name": "PersonsView",
|
||||
"tags": []
|
||||
@ -2282,8 +2282,8 @@
|
||||
"code": "CREATE PROCEDURE [Foo].[Proc.With.SpecialChar] @ID INT\nAS\n SELECT @ID AS ThatDB;\n",
|
||||
"input parameters": "['@ID']",
|
||||
"parameter @ID": "{'type': 'int'}",
|
||||
"date_created": "2024-11-22 12:58:03.137000",
|
||||
"date_modified": "2024-11-22 12:58:03.137000"
|
||||
"date_created": "2024-12-26 12:03:35.230000",
|
||||
"date_modified": "2024-12-26 12:03:35.230000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.Proc.With.SpecialChar",
|
||||
@ -2310,8 +2310,8 @@
|
||||
"depending_on_procedure": "{}",
|
||||
"code": "CREATE PROCEDURE [Foo].[NewProc]\n AS\n BEGIN\n --insert into items table from salesreason table\n insert into Foo.Items (ID, ItemName)\n SELECT TempID, Name\n FROM Foo.SalesReason;\n\n\n IF OBJECT_ID('Foo.age_dist', 'U') IS NULL\n BEGIN\n -- Create and populate if table doesn't exist\n SELECT Age, COUNT(*) as Count\n INTO Foo.age_dist\n FROM Foo.Persons\n GROUP BY Age\n END\n ELSE\n BEGIN\n -- Update existing table\n TRUNCATE TABLE Foo.age_dist;\n\n INSERT INTO Foo.age_dist (Age, Count)\n SELECT Age, COUNT(*) as Count\n FROM Foo.Persons\n GROUP BY Age\n END\n\n SELECT ID, Age INTO #TEMPTABLE FROM NewData.FooNew.PersonsNew\n \n UPDATE DemoData.Foo.Persons\n SET Age = t.Age\n FROM DemoData.Foo.Persons p\n JOIN #TEMPTABLE t ON p.ID = t.ID\n\n END\n",
|
||||
"input parameters": "[]",
|
||||
"date_created": "2024-11-22 12:58:03.140000",
|
||||
"date_modified": "2024-11-22 12:58:03.140000"
|
||||
"date_created": "2024-12-26 12:03:35.237000",
|
||||
"date_modified": "2024-12-26 12:03:35.237000"
|
||||
},
|
||||
"externalUrl": "",
|
||||
"name": "DemoData.Foo.NewProc",
|
||||
@ -4427,8 +4427,8 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"view_definition": "CREATE VIEW FooNew.View1 AS\nSELECT LastName, FirstName\nFROM FooNew.PersonsNew\nWHERE Age > 18\n",
|
||||
"is_view": "True"
|
||||
"is_view": "True",
|
||||
"view_definition": "CREATE VIEW FooNew.View1 AS\nSELECT LastName, FirstName\nFROM FooNew.PersonsNew\nWHERE Age > 18\n"
|
||||
},
|
||||
"name": "View1",
|
||||
"tags": []
|
||||
@ -4891,36 +4891,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),Age)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),Age)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),FirstName)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),FirstName)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -4930,7 +4913,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),ID)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -4941,7 +4925,32 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),LastName)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),FirstName)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),FirstName)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),Age)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),Age)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -4952,6 +4961,97 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW Foo.PersonsView AS\nSELECT\n *\nFROM Foo.Persons",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214622805,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),Age)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),FirstName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),ID)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.persons,PROD),LastName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),ID)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),LastName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),FirstName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,demodata.foo.personsview,PROD),Age)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD)",
|
||||
@ -4962,25 +5062,19 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1615443388097,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),FirstName)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD),FirstName)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
@ -4990,7 +5084,20 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD),LastName)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
"upstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),FirstName)"
|
||||
],
|
||||
"downstreamType": "FIELD",
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD),FirstName)"
|
||||
],
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -5001,6 +5108,85 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "CREATE VIEW FooNew.View1 AS\nSELECT\n LastName,\n FirstName\nFROM FooNew.PersonsNew\nWHERE\n Age > 18",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1735214622810,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),FirstName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.personsnew,PROD),LastName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD),LastName)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:mssql,newdata.foonew.view1,PROD),FirstName)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:mssql"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataJob",
|
||||
"entityUrn": "urn:li:dataJob:(urn:li:dataFlow:(mssql,DemoData.Foo.stored_procedures,PROD),NewProc)",
|
||||
@ -5151,5 +5337,37 @@
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cdemodata.foo.personsview%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Amssql%2Cnewdata.foonew.view1%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1615443388097,
|
||||
"runId": "mssql-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -94,6 +94,22 @@
|
||||
"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",
|
||||
@ -169,22 +185,6 @@
|
||||
"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",
|
||||
@ -246,7 +246,7 @@
|
||||
"numrows": "1",
|
||||
"rawdatasize": "32",
|
||||
"totalsize": "33",
|
||||
"transient_lastddltime": "1724180599"
|
||||
"transient_lastddltime": "1735206396"
|
||||
},
|
||||
"name": "array_struct_test",
|
||||
"description": "This table has array of structs",
|
||||
@ -507,7 +507,7 @@
|
||||
"numrows": "3",
|
||||
"rawdatasize": "94",
|
||||
"totalsize": "97",
|
||||
"transient_lastddltime": "1724180605"
|
||||
"transient_lastddltime": "1735206403"
|
||||
},
|
||||
"name": "classification_test",
|
||||
"tags": []
|
||||
@ -766,7 +766,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "map_test",
|
||||
"tags": []
|
||||
@ -993,7 +993,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "nested_struct_test",
|
||||
"tags": []
|
||||
@ -1264,7 +1264,7 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"transient_lastddltime": "1724180591"
|
||||
"transient_lastddltime": "1735206384"
|
||||
},
|
||||
"name": "pokes",
|
||||
"tags": []
|
||||
@ -1499,7 +1499,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180595"
|
||||
"transient_lastddltime": "1735206390"
|
||||
},
|
||||
"name": "struct_test",
|
||||
"tags": []
|
||||
@ -1750,7 +1750,7 @@
|
||||
"customProperties": {
|
||||
"numfiles": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180601"
|
||||
"transient_lastddltime": "1735206399"
|
||||
},
|
||||
"name": "struct_test_view_materialized",
|
||||
"tags": []
|
||||
@ -2004,7 +2004,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180595"
|
||||
"transient_lastddltime": "1735206390"
|
||||
},
|
||||
"name": "_test_table_underscore",
|
||||
"tags": []
|
||||
@ -2227,7 +2227,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "union_test",
|
||||
"tags": []
|
||||
@ -2529,9 +2529,9 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"transient_lastddltime": "1724180602",
|
||||
"view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\"",
|
||||
"is_view": "True"
|
||||
"transient_lastddltime": "1735206400",
|
||||
"is_view": "True",
|
||||
"view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\""
|
||||
},
|
||||
"name": "array_struct_test_view",
|
||||
"tags": []
|
||||
@ -2758,11 +2758,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1632398400000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
@ -2775,7 +2780,8 @@
|
||||
"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
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -2786,7 +2792,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -2797,6 +2804,85 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "SELECT\n \"property_id\",\n \"service\"\nFROM \"db1\".\"array_struct_test\"",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1632398400000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-instance-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,production_warehouse.hivedb.db1.array_struct_test_view,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-instance-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:trino"
|
||||
}
|
||||
},
|
||||
"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)",
|
||||
@ -2956,5 +3042,21 @@
|
||||
"runId": "trino-hive-instance-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Cproduction_warehouse.hivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-instance-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
}
|
||||
]
|
@ -87,6 +87,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84",
|
||||
@ -160,22 +176,6 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:c7a81f6ed9a7cdd0c74436ac2dc4d1f7"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "container",
|
||||
"entityUrn": "urn:li:container:304fd7ad57dc0ab32fb2cb778cbccd84",
|
||||
@ -233,7 +233,7 @@
|
||||
"numrows": "1",
|
||||
"rawdatasize": "32",
|
||||
"totalsize": "33",
|
||||
"transient_lastddltime": "1724180599"
|
||||
"transient_lastddltime": "1735206396"
|
||||
},
|
||||
"name": "array_struct_test",
|
||||
"description": "This table has array of structs",
|
||||
@ -473,7 +473,7 @@
|
||||
"numrows": "3",
|
||||
"rawdatasize": "94",
|
||||
"totalsize": "97",
|
||||
"transient_lastddltime": "1724180605"
|
||||
"transient_lastddltime": "1735206403"
|
||||
},
|
||||
"name": "classification_test",
|
||||
"tags": []
|
||||
@ -755,7 +755,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "map_test",
|
||||
"tags": []
|
||||
@ -961,7 +961,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "nested_struct_test",
|
||||
"tags": []
|
||||
@ -1211,7 +1211,7 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"transient_lastddltime": "1724180591"
|
||||
"transient_lastddltime": "1735206384"
|
||||
},
|
||||
"name": "pokes",
|
||||
"tags": []
|
||||
@ -1425,7 +1425,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180595"
|
||||
"transient_lastddltime": "1735206390"
|
||||
},
|
||||
"name": "struct_test",
|
||||
"tags": []
|
||||
@ -1655,7 +1655,7 @@
|
||||
"customProperties": {
|
||||
"numfiles": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180601"
|
||||
"transient_lastddltime": "1735206399"
|
||||
},
|
||||
"name": "struct_test_view_materialized",
|
||||
"tags": []
|
||||
@ -1888,7 +1888,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180595"
|
||||
"transient_lastddltime": "1735206390"
|
||||
},
|
||||
"name": "_test_table_underscore",
|
||||
"tags": []
|
||||
@ -2090,7 +2090,7 @@
|
||||
"numrows": "0",
|
||||
"rawdatasize": "0",
|
||||
"totalsize": "0",
|
||||
"transient_lastddltime": "1724180602"
|
||||
"transient_lastddltime": "1735206400"
|
||||
},
|
||||
"name": "union_test",
|
||||
"tags": []
|
||||
@ -2371,9 +2371,9 @@
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"transient_lastddltime": "1724180602",
|
||||
"view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\"",
|
||||
"is_view": "True"
|
||||
"transient_lastddltime": "1735206400",
|
||||
"is_view": "True",
|
||||
"view_definition": "SELECT \"property_id\", \"service\"\nFROM \"db1\".\"array_struct_test\""
|
||||
},
|
||||
"name": "array_struct_test_view",
|
||||
"tags": []
|
||||
@ -2579,11 +2579,16 @@
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1632398400000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)",
|
||||
"type": "VIEW"
|
||||
"type": "VIEW",
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
],
|
||||
"fineGrainedLineages": [
|
||||
@ -2596,7 +2601,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),property_id)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
},
|
||||
{
|
||||
"upstreamType": "FIELD_SET",
|
||||
@ -2607,7 +2613,8 @@
|
||||
"downstreams": [
|
||||
"urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),service)"
|
||||
],
|
||||
"confidenceScore": 1.0
|
||||
"confidenceScore": 0.9,
|
||||
"query": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -2618,6 +2625,85 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "queryProperties",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"statement": {
|
||||
"value": "SELECT\n \"property_id\",\n \"service\"\nFROM \"db1\".\"array_struct_test\"",
|
||||
"language": "SQL"
|
||||
},
|
||||
"source": "SYSTEM",
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 1632398400000,
|
||||
"actor": "urn:li:corpuser:_ingestion"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "querySubjects",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"subjects": [
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test,PROD),service)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),property_id)"
|
||||
},
|
||||
{
|
||||
"entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:trino,hivedb.db1.array_struct_test_view,PROD),service)"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:trino"
|
||||
}
|
||||
},
|
||||
"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)",
|
||||
@ -2778,6 +2864,22 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "query",
|
||||
"entityUrn": "urn:li:query:view_urn%3Ali%3Adataset%3A%28urn%3Ali%3AdataPlatform%3Atrino%2Chivedb.db1.array_struct_test_view%2CPROD%29",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1632398400000,
|
||||
"runId": "trino-hive-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "glossaryTerm",
|
||||
"entityUrn": "urn:li:glossaryTerm:Age",
|
||||
|
Loading…
x
Reference in New Issue
Block a user