mirror of
https://github.com/datahub-project/datahub.git
synced 2025-12-26 09:26:22 +00:00
feat(ingest): add looker meta extractor support in sql parsing (#12062)
Co-authored-by: Mayuri N <mayuri.nehate@gslab.com> Co-authored-by: Mayuri Nehate <33225191+mayurinehate@users.noreply.github.com>
This commit is contained in:
parent
953893cf2e
commit
2e544614f1
@ -63,3 +63,16 @@ class DatasetLineageProviderConfigBase(EnvConfigMixin):
|
||||
default=None,
|
||||
description="A holder for platform -> platform_instance mappings to generate correct dataset urns",
|
||||
)
|
||||
|
||||
|
||||
class PlatformDetail(ConfigModel):
|
||||
platform_instance: Optional[str] = Field(
|
||||
default=None,
|
||||
description="DataHub platform instance name. To generate correct urn for upstream dataset, this should match "
|
||||
"with platform instance name used in ingestion "
|
||||
"recipe of other datahub sources.",
|
||||
)
|
||||
env: str = Field(
|
||||
default=DEFAULT_ENV,
|
||||
description="The environment that all assets produced by DataHub platform ingestion source belong to",
|
||||
)
|
||||
|
||||
@ -31,6 +31,10 @@ from looker_sdk.sdk.api40.models import (
|
||||
from pydantic.class_validators import validator
|
||||
|
||||
import datahub.emitter.mce_builder as builder
|
||||
from datahub.api.entities.platformresource.platform_resource import (
|
||||
PlatformResource,
|
||||
PlatformResourceKey,
|
||||
)
|
||||
from datahub.emitter.mcp import MetadataChangeProposalWrapper
|
||||
from datahub.emitter.mcp_builder import ContainerKey, create_embed_mcp
|
||||
from datahub.ingestion.api.report import Report
|
||||
@ -106,7 +110,7 @@ from datahub.utilities.lossy_collections import LossyList, LossySet
|
||||
from datahub.utilities.url_util import remove_port_from_url
|
||||
|
||||
CORPUSER_DATAHUB = "urn:li:corpuser:datahub"
|
||||
|
||||
LOOKER = "looker"
|
||||
logger = logging.getLogger(__name__)
|
||||
|
||||
|
||||
@ -1411,6 +1415,7 @@ class LookerDashboardSourceReport(StaleEntityRemovalSourceReport):
|
||||
|
||||
resolved_user_ids: int = 0
|
||||
email_ids_missing: int = 0 # resolved users with missing email addresses
|
||||
looker_user_count: int = 0
|
||||
|
||||
_looker_api: Optional[LookerAPI] = None
|
||||
query_latency: Dict[str, datetime.timedelta] = dataclasses_field(
|
||||
@ -1614,9 +1619,21 @@ class LookerDashboard:
|
||||
class LookerUserRegistry:
|
||||
looker_api_wrapper: LookerAPI
|
||||
fields: str = ",".join(["id", "email", "display_name", "first_name", "last_name"])
|
||||
_user_cache: Dict[str, LookerUser] = {}
|
||||
|
||||
def __init__(self, looker_api: LookerAPI):
|
||||
def __init__(self, looker_api: LookerAPI, report: LookerDashboardSourceReport):
|
||||
self.looker_api_wrapper = looker_api
|
||||
self.report = report
|
||||
self._initialize_user_cache()
|
||||
|
||||
def _initialize_user_cache(self) -> None:
|
||||
raw_users: Sequence[User] = self.looker_api_wrapper.all_users(
|
||||
user_fields=self.fields
|
||||
)
|
||||
|
||||
for raw_user in raw_users:
|
||||
looker_user = LookerUser.create_looker_user(raw_user)
|
||||
self._user_cache[str(looker_user.id)] = looker_user
|
||||
|
||||
def get_by_id(self, id_: str) -> Optional[LookerUser]:
|
||||
if not id_:
|
||||
@ -1624,6 +1641,9 @@ class LookerUserRegistry:
|
||||
|
||||
logger.debug(f"Will get user {id_}")
|
||||
|
||||
if str(id_) in self._user_cache:
|
||||
return self._user_cache.get(str(id_))
|
||||
|
||||
raw_user: Optional[User] = self.looker_api_wrapper.get_user(
|
||||
str(id_), user_fields=self.fields
|
||||
)
|
||||
@ -1632,3 +1652,35 @@ class LookerUserRegistry:
|
||||
|
||||
looker_user = LookerUser.create_looker_user(raw_user)
|
||||
return looker_user
|
||||
|
||||
def to_platform_resource(
|
||||
self, platform_instance: Optional[str]
|
||||
) -> Iterable[MetadataChangeProposalWrapper]:
|
||||
try:
|
||||
platform_resource_key = PlatformResourceKey(
|
||||
platform=LOOKER,
|
||||
resource_type="USER_ID_MAPPING",
|
||||
platform_instance=platform_instance,
|
||||
primary_key="",
|
||||
)
|
||||
|
||||
# Extract user email mappings
|
||||
user_email_cache = {
|
||||
user_id: user.email
|
||||
for user_id, user in self._user_cache.items()
|
||||
if user.email
|
||||
}
|
||||
|
||||
platform_resource = PlatformResource.create(
|
||||
key=platform_resource_key,
|
||||
value=user_email_cache,
|
||||
)
|
||||
|
||||
self.report.looker_user_count = len(user_email_cache)
|
||||
yield from platform_resource.to_mcps()
|
||||
|
||||
except Exception as exc:
|
||||
self.report.warning(
|
||||
message="Failed to generate platform resource for looker id mappings",
|
||||
exc=exc,
|
||||
)
|
||||
|
||||
@ -68,6 +68,7 @@ class LookerAPIStats(BaseModel):
|
||||
get_look_calls: int = 0
|
||||
search_looks_calls: int = 0
|
||||
search_dashboards_calls: int = 0
|
||||
all_user_calls: int = 0
|
||||
|
||||
|
||||
class LookerAPI:
|
||||
@ -135,7 +136,7 @@ class LookerAPI:
|
||||
|
||||
return permissions
|
||||
|
||||
@lru_cache(maxsize=1000)
|
||||
@lru_cache(maxsize=5000)
|
||||
def get_user(self, id_: str, user_fields: str) -> Optional[User]:
|
||||
self.client_stats.user_calls += 1
|
||||
try:
|
||||
@ -154,6 +155,17 @@ class LookerAPI:
|
||||
# User not found
|
||||
return None
|
||||
|
||||
def all_users(self, user_fields: str) -> Sequence[User]:
|
||||
self.client_stats.all_user_calls += 1
|
||||
try:
|
||||
return self.client.all_users(
|
||||
fields=cast(str, user_fields),
|
||||
transport_options=self.transport_options,
|
||||
)
|
||||
except SDKError as e:
|
||||
logger.warning(f"Failure was {e}")
|
||||
return []
|
||||
|
||||
def execute_query(self, write_query: WriteQuery) -> List[Dict]:
|
||||
logger.debug(f"Executing query {write_query}")
|
||||
self.client_stats.query_calls += 1
|
||||
|
||||
@ -145,7 +145,9 @@ class LookerDashboardSource(TestableSource, StatefulIngestionSourceBase):
|
||||
self.source_config: LookerDashboardSourceConfig = config
|
||||
self.reporter: LookerDashboardSourceReport = LookerDashboardSourceReport()
|
||||
self.looker_api: LookerAPI = LookerAPI(self.source_config)
|
||||
self.user_registry: LookerUserRegistry = LookerUserRegistry(self.looker_api)
|
||||
self.user_registry: LookerUserRegistry = LookerUserRegistry(
|
||||
self.looker_api, self.reporter
|
||||
)
|
||||
self.explore_registry: LookerExploreRegistry = LookerExploreRegistry(
|
||||
self.looker_api, self.reporter, self.source_config
|
||||
)
|
||||
@ -1673,5 +1675,14 @@ class LookerDashboardSource(TestableSource, StatefulIngestionSourceBase):
|
||||
yield usage_mcp.as_workunit()
|
||||
self.reporter.report_stage_end("usage_extraction")
|
||||
|
||||
# Dump looker user resource mappings.
|
||||
logger.info("Ingesting looker user resource mapping workunits")
|
||||
self.reporter.report_stage_start("user_resource_extraction")
|
||||
yield from auto_workunit(
|
||||
self.user_registry.to_platform_resource(
|
||||
self.source_config.platform_instance
|
||||
)
|
||||
)
|
||||
|
||||
def get_report(self) -> SourceReport:
|
||||
return self.reporter
|
||||
|
||||
@ -9,7 +9,7 @@ from pydantic.class_validators import root_validator
|
||||
|
||||
import datahub.emitter.mce_builder as builder
|
||||
from datahub.configuration.common import AllowDenyPattern, ConfigModel
|
||||
from datahub.configuration.source_common import DatasetSourceConfigMixin
|
||||
from datahub.configuration.source_common import DatasetSourceConfigMixin, PlatformDetail
|
||||
from datahub.configuration.validate_field_deprecation import pydantic_field_deprecated
|
||||
from datahub.ingestion.source.common.subtypes import BIAssetSubTypes
|
||||
from datahub.ingestion.source.state.stale_entity_removal_handler import (
|
||||
@ -232,19 +232,6 @@ def default_for_dataset_type_mapping() -> Dict[str, str]:
|
||||
return dict_
|
||||
|
||||
|
||||
class PlatformDetail(ConfigModel):
|
||||
platform_instance: Optional[str] = pydantic.Field(
|
||||
default=None,
|
||||
description="DataHub platform instance name. To generate correct urn for upstream dataset, this should match "
|
||||
"with platform instance name used in ingestion "
|
||||
"recipe of other datahub sources.",
|
||||
)
|
||||
env: str = pydantic.Field(
|
||||
default=builder.DEFAULT_ENV,
|
||||
description="The environment that all assets produced by DataHub platform ingestion source belong to",
|
||||
)
|
||||
|
||||
|
||||
class DataBricksPlatformDetail(PlatformDetail):
|
||||
"""
|
||||
metastore is an additional field used in Databricks connector to generate the dataset urn
|
||||
|
||||
@ -2,8 +2,8 @@ import logging
|
||||
from abc import ABC, abstractmethod
|
||||
from typing import Union
|
||||
|
||||
from datahub.configuration.source_common import PlatformDetail
|
||||
from datahub.ingestion.source.powerbi.config import (
|
||||
PlatformDetail,
|
||||
PowerBiDashboardSourceConfig,
|
||||
PowerBIPlatformDetail,
|
||||
)
|
||||
|
||||
@ -5,13 +5,13 @@ from typing import Dict, List, Optional, Tuple, Type, cast
|
||||
|
||||
from lark import Tree
|
||||
|
||||
from datahub.configuration.source_common import PlatformDetail
|
||||
from datahub.emitter import mce_builder as builder
|
||||
from datahub.ingestion.api.common import PipelineContext
|
||||
from datahub.ingestion.source.powerbi.config import (
|
||||
Constant,
|
||||
DataBricksPlatformDetail,
|
||||
DataPlatformPair,
|
||||
PlatformDetail,
|
||||
PowerBiDashboardSourceConfig,
|
||||
PowerBiDashboardSourceReport,
|
||||
PowerBIPlatformDetail,
|
||||
|
||||
@ -540,6 +540,7 @@ class SnowflakeV2Source(
|
||||
identifiers=self.identifiers,
|
||||
schema_resolver=schema_resolver,
|
||||
discovered_tables=discovered_datasets,
|
||||
graph=self.ctx.graph,
|
||||
)
|
||||
|
||||
# TODO: This is slightly suboptimal because we create two SqlParsingAggregator instances with different configs
|
||||
|
||||
@ -490,7 +490,7 @@ class SqlParsingAggregator(Closeable):
|
||||
self._exit_stack.push(self._query_usage_counts)
|
||||
|
||||
# Tool Extractor
|
||||
self._tool_meta_extractor = ToolMetaExtractor()
|
||||
self._tool_meta_extractor = ToolMetaExtractor.create(graph)
|
||||
self.report.tool_meta_report = self._tool_meta_extractor.report
|
||||
|
||||
def close(self) -> None:
|
||||
|
||||
@ -1,3 +1,4 @@
|
||||
import contextlib
|
||||
import json
|
||||
import logging
|
||||
from dataclasses import dataclass, field
|
||||
@ -5,8 +6,15 @@ from typing import Callable, Dict, List, Optional, Tuple, Union
|
||||
|
||||
from typing_extensions import Protocol
|
||||
|
||||
from datahub.api.entities.platformresource.platform_resource import (
|
||||
ElasticPlatformResourceQuery,
|
||||
PlatformResource,
|
||||
PlatformResourceSearchFields,
|
||||
)
|
||||
from datahub.ingestion.api.report import Report
|
||||
from datahub.ingestion.graph.client import DataHubGraph
|
||||
from datahub.metadata.urns import CorpGroupUrn, CorpUserUrn
|
||||
from datahub.utilities.search_utils import LogicalOperator
|
||||
from datahub.utilities.stats_collections import int_top_k_dict
|
||||
|
||||
UrnStr = str
|
||||
@ -31,6 +39,7 @@ def _get_last_line(query: str) -> str:
|
||||
@dataclass
|
||||
class ToolMetaExtractorReport(Report):
|
||||
num_queries_meta_extracted: Dict[str, int] = field(default_factory=int_top_k_dict)
|
||||
failures: List[str] = field(default_factory=list)
|
||||
|
||||
|
||||
class ToolMetaExtractor:
|
||||
@ -42,14 +51,81 @@ class ToolMetaExtractor:
|
||||
by warehouse query logs.
|
||||
"""
|
||||
|
||||
def __init__(self) -> None:
|
||||
self.report = ToolMetaExtractorReport()
|
||||
def __init__(
|
||||
self,
|
||||
report: ToolMetaExtractorReport,
|
||||
looker_user_mapping: Optional[Dict[str, str]] = None,
|
||||
) -> None:
|
||||
self.report = report
|
||||
self.known_tool_extractors: List[Tuple[str, Callable[[QueryLog], bool]]] = [
|
||||
(
|
||||
"mode",
|
||||
self._extract_mode_query,
|
||||
)
|
||||
),
|
||||
(
|
||||
"looker",
|
||||
self._extract_looker_query,
|
||||
),
|
||||
]
|
||||
# maps user id (as string) to email address
|
||||
self.looker_user_mapping = looker_user_mapping
|
||||
|
||||
@classmethod
|
||||
def create(
|
||||
cls,
|
||||
graph: Optional[DataHubGraph] = None,
|
||||
) -> "ToolMetaExtractor":
|
||||
report = ToolMetaExtractorReport()
|
||||
looker_user_mapping = None
|
||||
if graph:
|
||||
try:
|
||||
looker_user_mapping = cls.extract_looker_user_mapping_from_graph(
|
||||
graph, report
|
||||
)
|
||||
except Exception as e:
|
||||
report.failures.append(
|
||||
f"Unexpected error during Looker user metadata extraction: {str(e)}"
|
||||
)
|
||||
|
||||
return cls(report, looker_user_mapping)
|
||||
|
||||
@classmethod
|
||||
def extract_looker_user_mapping_from_graph(
|
||||
cls, graph: DataHubGraph, report: ToolMetaExtractorReport
|
||||
) -> Optional[Dict[str, str]]:
|
||||
looker_user_mapping = None
|
||||
query = (
|
||||
ElasticPlatformResourceQuery.create_from()
|
||||
.group(LogicalOperator.AND)
|
||||
.add_field_match(PlatformResourceSearchFields.PLATFORM, "looker")
|
||||
.add_field_match(
|
||||
PlatformResourceSearchFields.RESOURCE_TYPE,
|
||||
"USER_ID_MAPPING",
|
||||
)
|
||||
.end()
|
||||
)
|
||||
platform_resources = list(
|
||||
PlatformResource.search_by_filters(query=query, graph_client=graph)
|
||||
)
|
||||
|
||||
if len(platform_resources) > 1:
|
||||
report.failures.append(
|
||||
"Looker user metadata extraction failed. Found more than one looker user id mappings."
|
||||
)
|
||||
else:
|
||||
platform_resource = platform_resources[0]
|
||||
|
||||
if (
|
||||
platform_resource
|
||||
and platform_resource.resource_info
|
||||
and platform_resource.resource_info.value
|
||||
):
|
||||
with contextlib.suppress(ValueError, AssertionError):
|
||||
value = platform_resource.resource_info.value.as_raw_json()
|
||||
if value:
|
||||
looker_user_mapping = value
|
||||
|
||||
return looker_user_mapping
|
||||
|
||||
def _extract_mode_query(self, entry: QueryLog) -> bool:
|
||||
"""
|
||||
@ -78,14 +154,49 @@ class ToolMetaExtractor:
|
||||
|
||||
return True
|
||||
|
||||
def _extract_looker_query(self, entry: QueryLog) -> bool:
|
||||
"""
|
||||
Returns:
|
||||
bool: whether QueryLog entry is that of looker and looker user info
|
||||
is extracted into entry.
|
||||
"""
|
||||
if not self.looker_user_mapping:
|
||||
return False
|
||||
|
||||
last_line = _get_last_line(entry.query_text)
|
||||
|
||||
if not (last_line.startswith("--") and "Looker Query Context" in last_line):
|
||||
return False
|
||||
|
||||
start_quote_idx = last_line.index("'")
|
||||
end_quote_idx = last_line.rindex("'")
|
||||
if start_quote_idx == -1 or end_quote_idx == -1:
|
||||
return False
|
||||
|
||||
looker_json_raw = last_line[start_quote_idx + 1 : end_quote_idx]
|
||||
looker_json = json.loads(looker_json_raw)
|
||||
|
||||
user_id = str(looker_json["user_id"])
|
||||
email = self.looker_user_mapping.get(user_id)
|
||||
if not email:
|
||||
return False
|
||||
|
||||
original_user = entry.user
|
||||
|
||||
entry.user = email_to_user_urn(email)
|
||||
entry.extra_info = entry.extra_info or {}
|
||||
entry.extra_info["user_via"] = original_user
|
||||
|
||||
return True
|
||||
|
||||
def extract_bi_metadata(self, entry: QueryLog) -> bool:
|
||||
for tool, meta_extractor in self.known_tool_extractors:
|
||||
try:
|
||||
if meta_extractor(entry):
|
||||
self.report.num_queries_meta_extracted[tool] += 1
|
||||
return True
|
||||
except Exception:
|
||||
logger.debug("Tool metadata extraction failed with error : {e}")
|
||||
except Exception as e:
|
||||
logger.debug(f"Tool metadata extraction failed with error : {e}")
|
||||
return False
|
||||
|
||||
|
||||
|
||||
@ -842,6 +842,62 @@
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.10)",
|
||||
|
||||
@ -497,6 +497,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.2)",
|
||||
|
||||
@ -735,6 +735,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -735,6 +735,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -828,6 +828,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.2)",
|
||||
|
||||
@ -464,6 +464,21 @@
|
||||
"/Folders/Shared"
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Ownership": {
|
||||
"owners": [
|
||||
{
|
||||
"owner": "urn:li:corpuser:test-1@looker.com",
|
||||
"type": "DATAOWNER"
|
||||
}
|
||||
],
|
||||
"ownerTypes": {},
|
||||
"lastModified": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -708,6 +723,21 @@
|
||||
"/Folders/Personal"
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Ownership": {
|
||||
"owners": [
|
||||
{
|
||||
"owner": "urn:li:corpuser:test-2@looker.com",
|
||||
"type": "DATAOWNER"
|
||||
}
|
||||
],
|
||||
"ownerTypes": {},
|
||||
"lastModified": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -1105,185 +1135,6 @@
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": {
|
||||
"urn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"aspects": [
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.BrowsePaths": {
|
||||
"paths": [
|
||||
"/Explore/sales_model"
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Status": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"project": "lkml_samples",
|
||||
"model": "sales_model",
|
||||
"looker.explore.label": "My Explore View",
|
||||
"looker.explore.name": "sales_explore",
|
||||
"looker.explore.file": "test_source_file.lkml"
|
||||
},
|
||||
"externalUrl": "https://looker.company.com/explore/sales_model/sales_explore",
|
||||
"name": "My Explore View",
|
||||
"description": "lorem ipsum",
|
||||
"tags": []
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.UpstreamLineage": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:datahub"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:looker,lkml_samples.view.underlying_view,PROD)",
|
||||
"type": "VIEW"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.schema.SchemaMetadata": {
|
||||
"schemaName": "sales_explore",
|
||||
"platform": "urn:li:dataPlatform:looker",
|
||||
"version": 0,
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
},
|
||||
"hash": "",
|
||||
"platformSchema": {
|
||||
"com.linkedin.pegasus2avro.schema.OtherSchema": {
|
||||
"rawSchema": ""
|
||||
}
|
||||
},
|
||||
"fields": [
|
||||
{
|
||||
"fieldPath": "dim1",
|
||||
"nullable": false,
|
||||
"description": "dimension one description",
|
||||
"label": "Dimensions One Label",
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.StringType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "string",
|
||||
"recursive": false,
|
||||
"globalTags": {
|
||||
"tags": [
|
||||
{
|
||||
"tag": "urn:li:tag:Dimension"
|
||||
}
|
||||
]
|
||||
},
|
||||
"isPartOfKey": false
|
||||
}
|
||||
],
|
||||
"primaryKeys": []
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "subTypes",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"typeNames": [
|
||||
"Explore"
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "embed",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"renderUrl": "https://looker.company.com/embed/explore/sales_model/sales_explore"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "browsePathsV2",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"path": [
|
||||
{
|
||||
"id": "Explore"
|
||||
},
|
||||
{
|
||||
"id": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5",
|
||||
"urn": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": {
|
||||
@ -1642,6 +1493,185 @@
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.DatasetSnapshot": {
|
||||
"urn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"aspects": [
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.BrowsePaths": {
|
||||
"paths": [
|
||||
"/Explore/sales_model"
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Status": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.DatasetProperties": {
|
||||
"customProperties": {
|
||||
"project": "lkml_samples",
|
||||
"model": "sales_model",
|
||||
"looker.explore.label": "My Explore View",
|
||||
"looker.explore.name": "sales_explore",
|
||||
"looker.explore.file": "test_source_file.lkml"
|
||||
},
|
||||
"externalUrl": "https://looker.company.com/explore/sales_model/sales_explore",
|
||||
"name": "My Explore View",
|
||||
"description": "lorem ipsum",
|
||||
"tags": []
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.dataset.UpstreamLineage": {
|
||||
"upstreams": [
|
||||
{
|
||||
"auditStamp": {
|
||||
"time": 1586847600000,
|
||||
"actor": "urn:li:corpuser:datahub"
|
||||
},
|
||||
"dataset": "urn:li:dataset:(urn:li:dataPlatform:looker,lkml_samples.view.underlying_view,PROD)",
|
||||
"type": "VIEW"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.schema.SchemaMetadata": {
|
||||
"schemaName": "sales_explore",
|
||||
"platform": "urn:li:dataPlatform:looker",
|
||||
"version": 0,
|
||||
"created": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
},
|
||||
"lastModified": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
},
|
||||
"hash": "",
|
||||
"platformSchema": {
|
||||
"com.linkedin.pegasus2avro.schema.OtherSchema": {
|
||||
"rawSchema": ""
|
||||
}
|
||||
},
|
||||
"fields": [
|
||||
{
|
||||
"fieldPath": "dim1",
|
||||
"nullable": false,
|
||||
"description": "dimension one description",
|
||||
"label": "Dimensions One Label",
|
||||
"type": {
|
||||
"type": {
|
||||
"com.linkedin.pegasus2avro.schema.StringType": {}
|
||||
}
|
||||
},
|
||||
"nativeDataType": "string",
|
||||
"recursive": false,
|
||||
"globalTags": {
|
||||
"tags": [
|
||||
{
|
||||
"tag": "urn:li:tag:Dimension"
|
||||
}
|
||||
]
|
||||
},
|
||||
"isPartOfKey": false
|
||||
}
|
||||
],
|
||||
"primaryKeys": []
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "subTypes",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"typeNames": [
|
||||
"Explore"
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "embed",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"renderUrl": "https://looker.company.com/embed/explore/sales_model/sales_explore"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "container",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"container": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,sales_model.explore.sales_explore,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "browsePathsV2",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"path": [
|
||||
{
|
||||
"id": "Explore"
|
||||
},
|
||||
{
|
||||
"id": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5",
|
||||
"urn": "urn:li:container:d38ab60586a6e39b4cf63f14946969c5"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"proposedSnapshot": {
|
||||
"com.linkedin.pegasus2avro.metadata.snapshot.TagSnapshot": {
|
||||
@ -1705,6 +1735,62 @@
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -793,6 +793,60 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:8436a2a37c4a7e81fb08c9c8415d2e4b",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:8436a2a37c4a7e81fb08c9c8415d2e4b",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker",
|
||||
"instance": "urn:li:dataPlatformInstance:(urn:li:dataPlatform:looker,ap-south-1)"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:8436a2a37c4a7e81fb08c9c8415d2e4b",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -759,6 +759,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -513,6 +513,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.2)",
|
||||
|
||||
@ -464,6 +464,21 @@
|
||||
"/Folders/Shared"
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"com.linkedin.pegasus2avro.common.Ownership": {
|
||||
"owners": [
|
||||
{
|
||||
"owner": "urn:li:corpuser:test-1@looker.com",
|
||||
"type": "DATAOWNER"
|
||||
}
|
||||
],
|
||||
"ownerTypes": {},
|
||||
"lastModified": {
|
||||
"time": 0,
|
||||
"actor": "urn:li:corpuser:unknown"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -1185,6 +1200,62 @@
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "execution-1"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
|
||||
@ -762,6 +762,62 @@
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided",
|
||||
"pipelineName": "stateful-looker-pipeline"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "tag",
|
||||
"entityUrn": "urn:li:tag:Dimension",
|
||||
@ -814,8 +870,8 @@
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dashboard",
|
||||
"entityUrn": "urn:li:dashboard:(looker,dashboards.11)",
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,bogus data.explore.my_view,PROD)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
@ -831,8 +887,8 @@
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.10)",
|
||||
"entityType": "dashboard",
|
||||
"entityUrn": "urn:li:dashboard:(looker,dashboards.11)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
@ -865,8 +921,8 @@
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "dataset",
|
||||
"entityUrn": "urn:li:dataset:(urn:li:dataPlatform:looker,bogus data.explore.my_view,PROD)",
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.10)",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
|
||||
@ -678,6 +678,59 @@
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "platformResourceInfo",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"resourceType": "USER_ID_MAPPING",
|
||||
"primaryKey": "",
|
||||
"value": {
|
||||
"blob": "{\"1\": \"test-1@looker.com\", \"2\": \"test-2@looker.com\", \"3\": \"test-3@looker.com\"}",
|
||||
"contentType": "JSON"
|
||||
}
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "dataPlatformInstance",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"platform": "urn:li:dataPlatform:looker"
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "platformResource",
|
||||
"entityUrn": "urn:li:platformResource:1cec84235c544a141e63dd2077da2562",
|
||||
"changeType": "UPSERT",
|
||||
"aspectName": "status",
|
||||
"aspect": {
|
||||
"json": {
|
||||
"removed": false
|
||||
}
|
||||
},
|
||||
"systemMetadata": {
|
||||
"lastObserved": 1586847600000,
|
||||
"runId": "looker-test",
|
||||
"lastRunId": "no-run-id-provided"
|
||||
}
|
||||
},
|
||||
{
|
||||
"entityType": "chart",
|
||||
"entityUrn": "urn:li:chart:(looker,dashboard_elements.2)",
|
||||
|
||||
@ -83,6 +83,7 @@ def test_looker_ingest(pytestconfig, tmp_path, mock_time):
|
||||
with mock.patch("looker_sdk.init40") as mock_sdk:
|
||||
mock_sdk.return_value = mocked_client
|
||||
setup_mock_dashboard(mocked_client)
|
||||
mocked_client.run_inline_query.side_effect = side_effect_query_inline
|
||||
setup_mock_explore(mocked_client)
|
||||
|
||||
test_resources_dir = pytestconfig.rootpath / "tests/integration/looker"
|
||||
@ -319,6 +320,7 @@ def setup_mock_look(mocked_client):
|
||||
mocked_client.all_looks.return_value = [
|
||||
Look(
|
||||
id="1",
|
||||
user_id="1",
|
||||
title="Outer Look",
|
||||
description="I am not part of any Dashboard",
|
||||
query_id="1",
|
||||
@ -327,6 +329,7 @@ def setup_mock_look(mocked_client):
|
||||
Look(
|
||||
id="2",
|
||||
title="Personal Look",
|
||||
user_id="2",
|
||||
description="I am not part of any Dashboard and in personal folder",
|
||||
query_id="2",
|
||||
folder=FolderBase(
|
||||
@ -561,6 +564,20 @@ def setup_mock_user(mocked_client):
|
||||
mocked_client.user.side_effect = get_user
|
||||
|
||||
|
||||
def setup_mock_all_user(mocked_client):
|
||||
def all_users(
|
||||
fields: Optional[str] = None,
|
||||
transport_options: Optional[transport.TransportOptions] = None,
|
||||
) -> List[User]:
|
||||
return [
|
||||
User(id="1", email="test-1@looker.com"),
|
||||
User(id="2", email="test-2@looker.com"),
|
||||
User(id="3", email="test-3@looker.com"),
|
||||
]
|
||||
|
||||
mocked_client.all_users.side_effect = all_users
|
||||
|
||||
|
||||
def side_effect_query_inline(
|
||||
result_format: str, body: WriteQuery, transport_options: Optional[TransportOptions]
|
||||
) -> str:
|
||||
@ -714,6 +731,7 @@ def test_looker_ingest_usage_history(pytestconfig, tmp_path, mock_time):
|
||||
mocked_client.run_inline_query.side_effect = side_effect_query_inline
|
||||
setup_mock_explore(mocked_client)
|
||||
setup_mock_user(mocked_client)
|
||||
setup_mock_all_user(mocked_client)
|
||||
|
||||
test_resources_dir = pytestconfig.rootpath / "tests/integration/looker"
|
||||
|
||||
@ -946,6 +964,8 @@ def ingest_independent_looks(
|
||||
mock_sdk.return_value = mocked_client
|
||||
setup_mock_dashboard(mocked_client)
|
||||
setup_mock_explore(mocked_client)
|
||||
setup_mock_user(mocked_client)
|
||||
setup_mock_all_user(mocked_client)
|
||||
setup_mock_look(mocked_client)
|
||||
|
||||
test_resources_dir = pytestconfig.rootpath / "tests/integration/looker"
|
||||
|
||||
@ -1,11 +1,14 @@
|
||||
from datahub.configuration.datetimes import parse_absolute_time
|
||||
from datahub.metadata.urns import CorpUserUrn
|
||||
from datahub.sql_parsing.sql_parsing_aggregator import PreparsedQuery
|
||||
from datahub.sql_parsing.tool_meta_extractor import ToolMetaExtractor
|
||||
from datahub.sql_parsing.tool_meta_extractor import (
|
||||
ToolMetaExtractor,
|
||||
ToolMetaExtractorReport,
|
||||
)
|
||||
|
||||
|
||||
def test_extract_mode_metadata() -> None:
|
||||
extractor = ToolMetaExtractor()
|
||||
extractor = ToolMetaExtractor(report=ToolMetaExtractorReport())
|
||||
query = """\
|
||||
select * from LONG_TAIL_COMPANIONS.ADOPTION.PET_PROFILES
|
||||
LIMIT 100
|
||||
@ -30,8 +33,42 @@ LIMIT 100
|
||||
assert extractor.report.num_queries_meta_extracted["mode"] == 1
|
||||
|
||||
|
||||
def test_extract_looker_metadata() -> None:
|
||||
extractor = ToolMetaExtractor(
|
||||
report=ToolMetaExtractorReport(), looker_user_mapping={"7": "john.doe@xyz.com"}
|
||||
)
|
||||
looker_query = """\
|
||||
SELECT
|
||||
all_entities_extended_sibling."ENTITY" AS "all_entities_extended_sibling.entity_type",
|
||||
COUNT(DISTINCT ( all_entities_extended_sibling."URN" )) AS "all_entities_extended_sibling.distinct_count"
|
||||
FROM "PUBLIC"."ALL_ENTITIES"
|
||||
AS all_entities_extended_sibling
|
||||
GROUP BY
|
||||
1
|
||||
ORDER BY
|
||||
1
|
||||
FETCH NEXT 50 ROWS ONLY
|
||||
-- Looker Query Context '{"user_id":7,"history_slug":"264797031bc403cf382cbefbe3700849","instance_slug":"32654f2ffadf10b1949d4009e52fc6a4"}'
|
||||
"""
|
||||
|
||||
entry = PreparsedQuery(
|
||||
query_id=None,
|
||||
query_text=looker_query,
|
||||
upstreams=[],
|
||||
downstream=None,
|
||||
column_lineage=None,
|
||||
column_usage=None,
|
||||
inferred_schema=None,
|
||||
user=CorpUserUrn("mode"),
|
||||
timestamp=parse_absolute_time("2021-08-01T01:02:03Z"),
|
||||
)
|
||||
assert extractor.extract_bi_metadata(entry)
|
||||
assert entry.user == CorpUserUrn("john.doe")
|
||||
assert extractor.report.num_queries_meta_extracted["looker"] == 1
|
||||
|
||||
|
||||
def test_extract_no_metadata() -> None:
|
||||
extractor = ToolMetaExtractor()
|
||||
extractor = ToolMetaExtractor(report=ToolMetaExtractorReport())
|
||||
query = """\
|
||||
select * from LONG_TAIL_COMPANIONS.ADOPTION.PET_PROFILES
|
||||
LIMIT 100
|
||||
@ -53,3 +90,4 @@ LIMIT 100
|
||||
assert not extractor.extract_bi_metadata(entry)
|
||||
|
||||
assert extractor.report.num_queries_meta_extracted["mode"] == 0
|
||||
assert extractor.report.num_queries_meta_extracted["looker"] == 0
|
||||
|
||||
@ -37,7 +37,11 @@ def stateful_source(mock_datahub_graph: DataHubGraph) -> Iterable[SnowflakeV2Sou
|
||||
),
|
||||
)
|
||||
|
||||
with mock.patch("snowflake.connector.connect"):
|
||||
with mock.patch(
|
||||
"datahub.sql_parsing.sql_parsing_aggregator.ToolMetaExtractor.create",
|
||||
) as mock_checkpoint, mock.patch("snowflake.connector.connect"):
|
||||
mock_checkpoint.return_value = mock.MagicMock()
|
||||
|
||||
yield SnowflakeV2Source(ctx=ctx, config=config)
|
||||
|
||||
|
||||
|
||||
@ -9,9 +9,13 @@ enum PlatformResourceType {
|
||||
/**
|
||||
* e.g. a Slack member resource, Looker user resource, etc.
|
||||
*/
|
||||
USER_INFO,
|
||||
USER_INFO,
|
||||
/**
|
||||
* e.g. a Slack channel
|
||||
*/
|
||||
CONVERSATION
|
||||
/**
|
||||
* e.g. Looker mapping of all user ids
|
||||
*/
|
||||
USER_ID_MAPPING
|
||||
}
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user