From d33bc9720880b46fa784d03e78a600a5fde05e88 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Thu, 18 Jul 2024 18:07:31 +0530 Subject: [PATCH 01/16] feat(ingest): add bigquery-queries source --- metadata-ingestion/setup.py | 2 + .../source/bigquery_v2/bigquery_config.py | 320 ++++++++++++----- .../source/bigquery_v2/bigquery_queries.py | 77 ++++ .../source/bigquery_v2/bigquery_report.py | 8 +- .../source/bigquery_v2/bigquery_schema.py | 4 +- .../source/bigquery_v2/queries_extractor.py | 340 ++++++++++++++++++ .../source/snowflake/snowflake_config.py | 13 +- .../ingestion/source_config/bigquery.py | 44 --- .../ingestion/source_config/usage/__init__.py | 0 .../source_config/usage/bigquery_usage.py | 163 --------- .../source_config/usage/snowflake_usage.py | 19 - .../sql_parsing/sql_parsing_aggregator.py | 16 +- .../integration/fivetran/test_fivetran.py | 2 +- 13 files changed, 679 insertions(+), 329 deletions(-) create mode 100644 metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py create mode 100644 metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py delete mode 100644 metadata-ingestion/src/datahub/ingestion/source_config/bigquery.py delete mode 100644 metadata-ingestion/src/datahub/ingestion/source_config/usage/__init__.py delete mode 100644 metadata-ingestion/src/datahub/ingestion/source_config/usage/bigquery_usage.py delete mode 100644 metadata-ingestion/src/datahub/ingestion/source_config/usage/snowflake_usage.py diff --git a/metadata-ingestion/setup.py b/metadata-ingestion/setup.py index 445600b8abd48..b930dbfdd688c 100644 --- a/metadata-ingestion/setup.py +++ b/metadata-ingestion/setup.py @@ -351,6 +351,7 @@ "google-cloud-datacatalog-lineage==0.2.2", } | classification_lib, + "bigquery-queries": sql_common | bigquery_common | sqlglot_lib, "clickhouse": sql_common | clickhouse_common, "clickhouse-usage": sql_common | usage_common | clickhouse_common, "cockroachdb": sql_common | postgres_common | {"sqlalchemy-cockroachdb<2.0.0"}, @@ -658,6 +659,7 @@ "athena = datahub.ingestion.source.sql.athena:AthenaSource", "azure-ad = datahub.ingestion.source.identity.azure_ad:AzureADSource", "bigquery = datahub.ingestion.source.bigquery_v2.bigquery:BigqueryV2Source", + "bigquery-queries = datahub.ingestion.source.bigquery_v2.bigquery_queries:BigQueryQueriesSource", "clickhouse = datahub.ingestion.source.sql.clickhouse:ClickHouseSource", "clickhouse-usage = datahub.ingestion.source.usage.clickhouse_usage:ClickHouseUsageSource", "cockroachdb = datahub.ingestion.source.sql.cockroachdb:CockroachDBSource", diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py index fe961dbd780f6..27cb0e3670adf 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py @@ -1,5 +1,8 @@ +import json import logging import os +import re +import tempfile from datetime import timedelta from typing import Any, Dict, List, Optional, Union @@ -8,19 +11,23 @@ from pydantic import Field, PositiveInt, PrivateAttr, root_validator, validator from datahub.configuration.common import AllowDenyPattern, ConfigModel +from datahub.configuration.source_common import ( + EnvConfigMixin, + LowerCaseDatasetUrnConfigMixin, + PlatformInstanceConfigMixin, +) from datahub.configuration.validate_field_removal import pydantic_removed_field +from datahub.configuration.validate_multiline_string import pydantic_multiline_string from datahub.ingestion.glossary.classification_mixin import ( ClassificationSourceConfigMixin, ) -from datahub.ingestion.source.sql.sql_config import SQLCommonConfig +from datahub.ingestion.source.sql.sql_config import SQLCommonConfig, SQLFilterConfig from datahub.ingestion.source.state.stateful_ingestion_base import ( StatefulLineageConfigMixin, StatefulProfilingConfigMixin, StatefulUsageConfigMixin, ) from datahub.ingestion.source.usage.usage_common import BaseUsageConfig -from datahub.ingestion.source_config.bigquery import BigQueryBaseConfig -from datahub.ingestion.source_config.usage.bigquery_usage import BigQueryCredential logger = logging.getLogger(__name__) @@ -28,6 +35,54 @@ os.getenv("DATAHUB_BIGQUERY_SCHEMA_PARALLELISM", 20) ) +# Regexp for sharded tables. +# A sharded table is a table that has a suffix of the form _yyyymmdd or yyyymmdd, where yyyymmdd is a date. +# The regexp checks for valid dates in the suffix (e.g. 20200101, 20200229, 20201231) and if the date is not valid +# then it is not a sharded table. +_BIGQUERY_DEFAULT_SHARDED_TABLE_REGEX: str = ( + "((.+\\D)[_$]?)?(\\d\\d\\d\\d(?:0[1-9]|1[0-2])(?:0[1-9]|[12][0-9]|3[01]))$" +) + + +class BigQueryBaseConfig(ConfigModel): + rate_limit: bool = Field( + default=False, description="Should we rate limit requests made to API." + ) + requests_per_min: int = Field( + default=60, + description="Used to control number of API calls made per min. Only used when `rate_limit` is set to `True`.", + ) + + temp_table_dataset_prefix: str = Field( + default="_", + description="If you are creating temp tables in a dataset with a particular prefix you can use this config to set the prefix for the dataset. This is to support workflows from before bigquery's introduction of temp tables. By default we use `_` because of datasets that begin with an underscore are hidden by default https://cloud.google.com/bigquery/docs/datasets#dataset-naming.", + ) + + sharded_table_pattern: str = Field( + deprecated=True, + default=_BIGQUERY_DEFAULT_SHARDED_TABLE_REGEX, + description="The regex pattern to match sharded tables and group as one table. This is a very low level config parameter, only change if you know what you are doing, ", + ) + + project_ids: List[str] = Field( + default_factory=list, + description=( + "Ingests specified project_ids. Use this property if you want to specify what projects to ingest or " + "don't want to give project resourcemanager.projects.list to your service account. " + "Overrides `project_id_pattern`." + ), + ) + + @validator("sharded_table_pattern") + def sharded_table_pattern_is_a_valid_regexp(cls, v): + try: + re.compile(v) + except Exception as e: + raise ValueError( + f"sharded_table_pattern configuration pattern is invalid. The exception was: {e}" + ) + return v + class BigQueryUsageConfig(BaseUsageConfig): _query_log_delay_removed = pydantic_removed_field("query_log_delay") @@ -43,6 +98,48 @@ class BigQueryUsageConfig(BaseUsageConfig): ) +class BigQueryCredential(ConfigModel): + project_id: str = Field(description="Project id to set the credentials") + private_key_id: str = Field(description="Private key id") + private_key: str = Field( + description="Private key in a form of '-----BEGIN PRIVATE KEY-----\\nprivate-key\\n-----END PRIVATE KEY-----\\n'" + ) + client_email: str = Field(description="Client email") + client_id: str = Field(description="Client Id") + auth_uri: str = Field( + default="https://accounts.google.com/o/oauth2/auth", + description="Authentication uri", + ) + token_uri: str = Field( + default="https://oauth2.googleapis.com/token", description="Token uri" + ) + auth_provider_x509_cert_url: str = Field( + default="https://www.googleapis.com/oauth2/v1/certs", + description="Auth provider x509 certificate url", + ) + type: str = Field(default="service_account", description="Authentication type") + client_x509_cert_url: Optional[str] = Field( + default=None, + description="If not set it will be default to https://www.googleapis.com/robot/v1/metadata/x509/client_email", + ) + + _fix_private_key_newlines = pydantic_multiline_string("private_key") + + @root_validator(skip_on_failure=True) + def validate_config(cls, values: Dict[str, Any]) -> Dict[str, Any]: + if values.get("client_x509_cert_url") is None: + values[ + "client_x509_cert_url" + ] = f'https://www.googleapis.com/robot/v1/metadata/x509/{values["client_email"]}' + return values + + def create_credential_temp_file(self) -> str: + with tempfile.NamedTemporaryFile(delete=False) as fp: + cred_json = json.dumps(self.dict(), indent=4, separators=(",", ": ")) + fp.write(cred_json.encode()) + return fp.name + + class BigQueryConnectionConfig(ConfigModel): credential: Optional[BigQueryCredential] = Field( default=None, description="BigQuery credential informations" @@ -98,19 +195,121 @@ def get_sql_alchemy_url(self) -> str: return "bigquery://" +class BigQueryConnection: + _connection: bigquery.Client + + def __init__(self, connection: bigquery.Client): + self._connection = connection + + def native_connection(self) -> bigquery.Client: + return self._connection + + def query(self, query: str) -> Any: + try: + logger.info(f"Query: {query}", stacklevel=2) + resp = self._connection.query(query) + return resp + + except Exception: + # TODO: is it possible to handle any generic error type here ? + raise + + def close(self): + self._connection.close() + + +class BigQueryFilterConfig(SQLFilterConfig): + + project_id_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + description="Regex patterns for project_id to filter in ingestion.", + ) + + dataset_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + description="Regex patterns for dataset to filter in ingestion. Specify regex to only match the schema name. e.g. to match all tables in schema analytics, use the regex 'analytics'", + ) + + match_fully_qualified_names: bool = Field( + default=True, + description="[deprecated] Whether `dataset_pattern` is matched against fully qualified dataset name `.`.", + ) + + table_snapshot_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + description="Regex patterns for table snapshots to filter in ingestion. Specify regex to match the entire snapshot name in database.schema.snapshot format. e.g. to match all snapshots starting with customer in Customer database and public schema, use the regex 'Customer.public.customer.*'", + ) + + @root_validator(pre=False, skip_on_failure=True) + def backward_compatibility_configs_set(cls, values: Dict) -> Dict: + dataset_pattern: Optional[AllowDenyPattern] = values.get("dataset_pattern") + schema_pattern = values.get("schema_pattern") + if ( + dataset_pattern == AllowDenyPattern.allow_all() + and schema_pattern != AllowDenyPattern.allow_all() + ): + logging.warning( + "dataset_pattern is not set but schema_pattern is set, using schema_pattern as dataset_pattern. schema_pattern will be deprecated, please use dataset_pattern instead." + ) + values["dataset_pattern"] = schema_pattern + dataset_pattern = schema_pattern + elif ( + dataset_pattern != AllowDenyPattern.allow_all() + and schema_pattern != AllowDenyPattern.allow_all() + ): + logging.warning( + "schema_pattern will be ignored in favour of dataset_pattern. schema_pattern will be deprecated, please use dataset_pattern only." + ) + + match_fully_qualified_names = values.get("match_fully_qualified_names") + + if ( + dataset_pattern is not None + and dataset_pattern != AllowDenyPattern.allow_all() + and match_fully_qualified_names is not None + and not match_fully_qualified_names + ): + logger.warning( + "Please update `dataset_pattern` to match against fully qualified schema name `.` and set config `match_fully_qualified_names : True`." + "The config option `match_fully_qualified_names` is deprecated and will be removed in a future release." + ) + elif match_fully_qualified_names and dataset_pattern is not None: + adjusted = False + for lst in [dataset_pattern.allow, dataset_pattern.deny]: + for i, pattern in enumerate(lst): + if "." not in pattern: + if pattern.startswith("^"): + lst[i] = r"^.*\." + pattern[1:] + else: + lst[i] = r".*\." + pattern + adjusted = True + if adjusted: + logger.warning( + "`dataset_pattern` was adjusted to match against fully qualified schema names," + " of the form `.`." + ) + + return values + + +class BigQueryIdentifierConfig( + PlatformInstanceConfigMixin, EnvConfigMixin, LowerCaseDatasetUrnConfigMixin +): + pass + + class BigQueryV2Config( BigQueryConnectionConfig, BigQueryBaseConfig, + BigQueryFilterConfig, + BigQueryIdentifierConfig, + # BigQueryFilterConfig must come before (higher precedence) the SQLCommon config, so that the documentation overrides are applied. SQLCommonConfig, StatefulUsageConfigMixin, StatefulLineageConfigMixin, StatefulProfilingConfigMixin, ClassificationSourceConfigMixin, ): - project_id_pattern: AllowDenyPattern = Field( - default=AllowDenyPattern.allow_all(), - description="Regex patterns for project_id to filter in ingestion.", - ) include_schema_metadata: bool = Field( default=True, @@ -141,16 +340,6 @@ class BigQueryV2Config( description="Capture BigQuery dataset labels as DataHub tag", ) - dataset_pattern: AllowDenyPattern = Field( - default=AllowDenyPattern.allow_all(), - description="Regex patterns for dataset to filter in ingestion. Specify regex to only match the schema name. e.g. to match all tables in schema analytics, use the regex 'analytics'", - ) - - match_fully_qualified_names: bool = Field( - default=True, - description="[deprecated] Whether `dataset_pattern` is matched against fully qualified dataset name `.`.", - ) - include_external_url: bool = Field( default=True, description="Whether to populate BigQuery Console url to Datasets/Tables", @@ -167,11 +356,6 @@ class BigQueryV2Config( default=True, description="Whether table snapshots should be ingested." ) - table_snapshot_pattern: AllowDenyPattern = Field( - default=AllowDenyPattern.allow_all(), - description="Regex patterns for table snapshots to filter in ingestion. Specify regex to match the entire snapshot name in database.schema.snapshot format. e.g. to match all snapshots starting with customer in Customer database and public schema, use the regex 'Customer.public.customer.*'", - ) - debug_include_full_payloads: bool = Field( default=False, description="Include full payload into events. It is only for debugging and internal use.", @@ -201,21 +385,13 @@ def have_table_data_read_permission(self) -> bool: default=300, description="Maximum number of columns to process in a table. This is a low level config property which should be touched with care. This restriction is needed because excessively wide tables can result in failure to ingest the schema.", ) + + # TODO- remove project_id # The inheritance hierarchy is wonky here, but these options need modifications. project_id: Optional[str] = Field( default=None, description="[deprecated] Use project_id_pattern or project_ids instead.", ) - project_ids: List[str] = Field( - default_factory=list, - description=( - "Ingests specified project_ids. Use this property if you want to specify what projects to ingest or " - "don't want to give project resourcemanager.projects.list to your service account. " - "Overrides `project_id_pattern`." - ), - ) - - storage_project_id: None = Field(default=None, hidden_from_docs=True) lineage_use_sql_parser: bool = Field( default=True, @@ -323,6 +499,22 @@ def have_table_data_read_permission(self) -> bool: " Set to 1 to disable.", ) + @root_validator(pre=False, skip_on_failure=True) + def project_id_backward_compatibility_configs_set(cls, values: Dict) -> Dict: + project_id = values.get("project_id") + project_id_pattern = values.get("project_id_pattern") + + if project_id_pattern == AllowDenyPattern.allow_all() and project_id: + logging.warning( + "project_id_pattern is not set but project_id is set, source will only ingest the project_id project. project_id will be deprecated, please use project_id_pattern instead." + ) + values["project_id_pattern"] = AllowDenyPattern(allow=[f"^{project_id}$"]) + elif project_id_pattern != AllowDenyPattern.allow_all() and project_id: + logging.warning( + "use project_id_pattern whenever possible. project_id will be deprecated, please use project_id_pattern only if possible." + ) + return values + @root_validator(skip_on_failure=True) def profile_default_settings(cls, values: Dict) -> Dict: # Extra default SQLAlchemy option for better connection pooling and threading. @@ -342,70 +534,6 @@ def validate_bigquery_audit_metadata_datasets( return v - @root_validator(pre=False, skip_on_failure=True) - def backward_compatibility_configs_set(cls, values: Dict) -> Dict: - project_id = values.get("project_id") - project_id_pattern = values.get("project_id_pattern") - - if project_id_pattern == AllowDenyPattern.allow_all() and project_id: - logging.warning( - "project_id_pattern is not set but project_id is set, source will only ingest the project_id project. project_id will be deprecated, please use project_id_pattern instead." - ) - values["project_id_pattern"] = AllowDenyPattern(allow=[f"^{project_id}$"]) - elif project_id_pattern != AllowDenyPattern.allow_all() and project_id: - logging.warning( - "use project_id_pattern whenever possible. project_id will be deprecated, please use project_id_pattern only if possible." - ) - - dataset_pattern: Optional[AllowDenyPattern] = values.get("dataset_pattern") - schema_pattern = values.get("schema_pattern") - if ( - dataset_pattern == AllowDenyPattern.allow_all() - and schema_pattern != AllowDenyPattern.allow_all() - ): - logging.warning( - "dataset_pattern is not set but schema_pattern is set, using schema_pattern as dataset_pattern. schema_pattern will be deprecated, please use dataset_pattern instead." - ) - values["dataset_pattern"] = schema_pattern - dataset_pattern = schema_pattern - elif ( - dataset_pattern != AllowDenyPattern.allow_all() - and schema_pattern != AllowDenyPattern.allow_all() - ): - logging.warning( - "schema_pattern will be ignored in favour of dataset_pattern. schema_pattern will be deprecated, please use dataset_pattern only." - ) - - match_fully_qualified_names = values.get("match_fully_qualified_names") - - if ( - dataset_pattern is not None - and dataset_pattern != AllowDenyPattern.allow_all() - and match_fully_qualified_names is not None - and not match_fully_qualified_names - ): - logger.warning( - "Please update `dataset_pattern` to match against fully qualified schema name `.` and set config `match_fully_qualified_names : True`." - "The config option `match_fully_qualified_names` is deprecated and will be removed in a future release." - ) - elif match_fully_qualified_names and dataset_pattern is not None: - adjusted = False - for lst in [dataset_pattern.allow, dataset_pattern.deny]: - for i, pattern in enumerate(lst): - if "." not in pattern: - if pattern.startswith("^"): - lst[i] = r"^.*\." + pattern[1:] - else: - lst[i] = r".*\." + pattern - adjusted = True - if adjusted: - logger.warning( - "`dataset_pattern` was adjusted to match against fully qualified schema names," - " of the form `.`." - ) - - return values - def get_table_pattern(self, pattern: List[str]) -> str: return "|".join(pattern) if pattern else "" diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py new file mode 100644 index 0000000000000..91a26b630aff7 --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py @@ -0,0 +1,77 @@ +import logging +from dataclasses import dataclass +from typing import Iterable, Optional + +from pydantic import Field +from typing_extensions import Self + +from datahub.configuration.time_window_config import BaseTimeWindowConfig +from datahub.ingestion.api.common import PipelineContext +from datahub.ingestion.api.source import Source, SourceReport +from datahub.ingestion.api.workunit import MetadataWorkUnit +from datahub.ingestion.source.bigquery_v2.bigquery_config import ( + BigQueryConnectionConfig, + BigQueryFilterConfig, + BigQueryIdentifierConfig, +) +from datahub.ingestion.source.bigquery_v2.bigquery_report import ( + BigQuerySchemaApiPerfReport, +) +from datahub.ingestion.source.bigquery_v2.bigquery_schema import BigQuerySchemaApi +from datahub.ingestion.source.bigquery_v2.queries_extractor import ( + BigQueryQueriesExtractor, + BigQueryQueriesExtractorConfig, + BigQueryQueriesExtractorReport, +) + +logger = logging.getLogger(__name__) + + +@dataclass +class BigQueryQueriesSourceReport(SourceReport): + window: Optional[BaseTimeWindowConfig] = None + queries_extractor: Optional[BigQueryQueriesExtractorReport] = None + + +class BigQueryQueriesSourceConfig( + BigQueryQueriesExtractorConfig, BigQueryFilterConfig, BigQueryIdentifierConfig +): + connection: BigQueryConnectionConfig = Field( + default_factory=BigQueryConnectionConfig + ) + + +class BigQueryQueriesSource(Source): + def __init__(self, ctx: PipelineContext, config: BigQueryQueriesSourceConfig): + self.ctx = ctx + self.config = config + self.report = BigQueryQueriesSourceReport() + + self.connection = self.config.connection.get_bigquery_client() + + self.queries_extractor = BigQueryQueriesExtractor( + connection=self.connection, + schema_api=BigQuerySchemaApi( + BigQuerySchemaApiPerfReport(), self.connection + ), + config=self.config, + structured_report=self.report, + filter_config=self.config, + identifier_config=self.config, + graph=self.ctx.graph, + ) + self.report.queries_extractor = self.queries_extractor.report + + @classmethod + def create(cls, config_dict: dict, ctx: PipelineContext) -> Self: + config = BigQueryQueriesSourceConfig.parse_obj(config_dict) + return cls(ctx, config) + + def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: + self.report.window = self.config.window + + # TODO: Disable auto status processor? + return self.queries_extractor.get_workunits_internal() + + def get_report(self) -> BigQueryQueriesSourceReport: + return self.report diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_report.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_report.py index 4cfcc3922ddc3..97b50f8d4ef69 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_report.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_report.py @@ -11,6 +11,7 @@ from datahub.ingestion.source.sql.sql_generic_profiler import ProfilingSqlReport from datahub.ingestion.source_report.ingestion_stage import IngestionStageReport from datahub.ingestion.source_report.time_window import BaseTimeWindowReport +from datahub.sql_parsing.sql_parsing_aggregator import SqlAggregatorReport from datahub.utilities.lossy_collections import LossyDict, LossyList from datahub.utilities.perf_timer import PerfTimer from datahub.utilities.stats_collections import TopKDict, int_top_k_dict @@ -30,8 +31,8 @@ class BigQuerySchemaApiPerfReport(Report): num_get_views_for_dataset_api_requests: int = 0 num_get_snapshots_for_dataset_api_requests: int = 0 - list_projects: PerfTimer = field(default_factory=PerfTimer) - list_datasets: PerfTimer = field(default_factory=PerfTimer) + list_projects_timer: PerfTimer = field(default_factory=PerfTimer) + list_datasets_timer: PerfTimer = field(default_factory=PerfTimer) get_columns_for_dataset_sec: float = 0 get_tables_for_dataset_sec: float = 0 @@ -169,5 +170,8 @@ class BigQueryV2Report( usage_end_time: Optional[datetime] = None stateful_usage_ingestion_enabled: bool = False + # lineage/usage v2 + sql_aggregator: Optional[SqlAggregatorReport] = None + def set_ingestion_stage(self, project_id: str, stage: str) -> None: self.report_ingestion_stage_start(f"{project_id}: {stage}") diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py index d73ac46c862ea..76ede5248c7c0 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py @@ -165,7 +165,7 @@ def _should_retry(exc: BaseException) -> bool: page_token = None projects: List[BigqueryProject] = [] - with self.report.list_projects: + with self.report.list_projects_timer: while True: try: self.report.num_list_projects_api_requests += 1 @@ -205,7 +205,7 @@ def _should_retry(exc: BaseException) -> bool: def get_datasets_for_project_id( self, project_id: str, maxResults: Optional[int] = None ) -> List[BigqueryDataset]: - with self.report.list_datasets: + with self.report.list_datasets_timer: self.report.num_list_datasets_api_requests += 1 datasets = self.bq_client.list_datasets(project_id, max_results=maxResults) return [ diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py new file mode 100644 index 0000000000000..547993777634b --- /dev/null +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -0,0 +1,340 @@ +import functools +import logging +import pathlib +import tempfile +from dataclasses import dataclass, field +from datetime import datetime, timezone +from typing import Iterable, List, Optional, TypedDict, Union + +from google.cloud.bigquery import Client +from pydantic import Field + +from datahub.configuration.time_window_config import BaseTimeWindowConfig +from datahub.emitter.mce_builder import make_user_urn +from datahub.ingestion.api.report import Report +from datahub.ingestion.api.source import SourceReport +from datahub.ingestion.api.source_helpers import auto_workunit +from datahub.ingestion.api.workunit import MetadataWorkUnit +from datahub.ingestion.graph.client import DataHubGraph +from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier +from datahub.ingestion.source.bigquery_v2.bigquery_config import ( + BigQueryBaseConfig, + BigQueryFilterConfig, + BigQueryIdentifierConfig, +) +from datahub.ingestion.source.bigquery_v2.bigquery_schema import ( + BigqueryProject, + BigQuerySchemaApi, +) +from datahub.ingestion.source.bigquery_v2.common import BQ_DATETIME_FORMAT +from datahub.ingestion.source.usage.usage_common import BaseUsageConfig +from datahub.sql_parsing.schema_resolver import SchemaResolver +from datahub.sql_parsing.sql_parsing_aggregator import ( + ObservedQuery, + PreparsedQuery, + SqlAggregatorReport, + SqlParsingAggregator, +) +from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedList +from datahub.utilities.perf_timer import PerfTimer + +logger = logging.getLogger(__name__) + + +class BigQueryTableReference(TypedDict): + projectId: str + datasetId: str + tableId: str + + +class DMLJobStatistics(TypedDict): + inserted_row_count: int + deleted_row_count: int + updated_row_count: int + + +class BigQueryJob(TypedDict): + job_id: str + project_id: str + creation_time: datetime + start_time: datetime + end_time: datetime + total_slot_ms: int + user_email: str + statement_type: str + job_type: Optional[str] + query: str + destination_table: Optional[BigQueryTableReference] + # NOTE: This does not capture referenced_view unlike GCP Logging Event + referenced_tables: List[BigQueryTableReference] + total_bytes_billed: int + total_bytes_processed: int + dml_statistics: Optional[DMLJobStatistics] + session_id: Optional[str] + + +class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): + # TODO: Support stateful ingestion for the time windows. + window: BaseTimeWindowConfig = BaseTimeWindowConfig() + + local_temp_path: Optional[pathlib.Path] = Field( + default=None, + description="Local path to store the audit log.", + # TODO: For now, this is simply an advanced config to make local testing easier. + # Eventually, we will want to store date-specific files in the directory and use it as a cache. + hidden_from_docs=True, + ) + + include_lineage: bool = True + include_queries: bool = True + include_usage_statistics: bool = True + include_query_usage_statistics: bool = False + include_operations: bool = True + + +@dataclass +class BigQueryQueriesExtractorReport(Report): + query_log_fetch_timer: PerfTimer = field(default_factory=PerfTimer) + audit_log_load_timer: PerfTimer = field(default_factory=PerfTimer) + sql_aggregator: Optional[SqlAggregatorReport] = None + + +class BigQueryQueriesExtractor: + def __init__( + self, + connection: Client, + schema_api: BigQuerySchemaApi, + config: BigQueryQueriesExtractorConfig, + structured_report: SourceReport, + # TODO: take mixins for filter and identifier instead of config itself + filter_config: BigQueryFilterConfig, + identifier_config: BigQueryIdentifierConfig, + graph: Optional[DataHubGraph] = None, + schema_resolver: Optional[SchemaResolver] = None, + discovered_tables: Optional[List[str]] = None, + ): + self.connection = connection + + self.config = config + self.filter_config = filter_config + self.identifier_config = identifier_config + self.schema_api = schema_api + self.report = BigQueryQueriesExtractorReport() + # self.filters = filters + self.discovered_tables = discovered_tables + + self._structured_report = structured_report + + self.aggregator = SqlParsingAggregator( + platform="bigquery", + platform_instance=self.identifier_config.platform_instance, + env=self.identifier_config.env, + schema_resolver=schema_resolver, + graph=graph, + eager_graph_load=False, + generate_lineage=self.config.include_lineage, + generate_queries=self.config.include_queries, + generate_usage_statistics=self.config.include_usage_statistics, + generate_query_usage_statistics=self.config.include_query_usage_statistics, + usage_config=BaseUsageConfig( + bucket_duration=self.config.window.bucket_duration, + start_time=self.config.window.start_time, + end_time=self.config.window.end_time, + # TODO make the rest of the fields configurable + ), + generate_operations=self.config.include_operations, + is_temp_table=self.is_temp_table, + is_allowed_table=self.is_allowed_table, + format_queries=False, + ) + self.report.sql_aggregator = self.aggregator.report + + @property + def structured_report(self) -> SourceReport: + return self._structured_report + + @functools.cached_property + def local_temp_path(self) -> pathlib.Path: + if self.config.local_temp_path: + assert self.config.local_temp_path.is_dir() + return self.config.local_temp_path + + path = pathlib.Path(tempfile.mkdtemp()) + path.mkdir(parents=True, exist_ok=True) + logger.info(f"Using local temp path: {path}") + return path + + def is_temp_table(self, name: str) -> bool: + return BigqueryTableIdentifier.from_string_name(name).dataset.startswith( + self.config.temp_table_dataset_prefix + ) + + def is_allowed_table(self, name: str) -> bool: + if ( + self.discovered_tables + and str(BigqueryTableIdentifier.from_string_name(name)) + not in self.discovered_tables + ): + return False + return True + + # TODO: Remove the code duplication. Also present in main bigquery source + def _get_projects(self) -> List[BigqueryProject]: + logger.info("Getting projects") + if self.config.project_ids: + return [ + BigqueryProject(id=project_id, name=project_id) + for project_id in self.config.project_ids + ] + else: + return list(self._query_project_list()) + + def _query_project_list(self) -> Iterable[BigqueryProject]: + try: + projects = self.schema_api.get_projects() + + if ( + not projects + ): # Report failure on exception and if empty list is returned + self.structured_report.failure( + title="Get projects didn't return any project. ", + message="Maybe resourcemanager.projects.get permission is missing for the service account. " + "You can assign predefined roles/bigquery.metadataViewer role to your service account.", + ) + except Exception as e: + self.structured_report.failure( + title="Failed to get BigQuery Projects", + message="Maybe resourcemanager.projects.get permission is missing for the service account. " + "You can assign predefined roles/bigquery.metadataViewer role to your service account.", + exc=e, + ) + projects = [] + + for project in projects: + if self.filter_config.project_id_pattern.allowed(project.id): + yield project + + def get_workunits_internal( + self, + ) -> Iterable[MetadataWorkUnit]: + # TODO: Add some logic to check if the cached audit log is stale or not. + audit_log_file = self.local_temp_path / "audit_log.sqlite" + use_cached_audit_log = audit_log_file.exists() + + queries: FileBackedList[Union[PreparsedQuery, ObservedQuery]] + if use_cached_audit_log: + logger.info("Using cached audit log") + shared_connection = ConnectionWrapper(audit_log_file) + queries = FileBackedList(shared_connection) + else: + audit_log_file.unlink(missing_ok=True) + + shared_connection = ConnectionWrapper(audit_log_file) + queries = FileBackedList(shared_connection) + entry: Union[PreparsedQuery, ObservedQuery] + + with self.report.query_log_fetch_timer: + for project in self._get_projects(): + for entry in self.fetch_query_log(project): + queries.append(entry) + + with self.report.audit_log_load_timer: + for query in queries: + self.aggregator.add(query) + + yield from auto_workunit(self.aggregator.gen_metadata()) + + def fetch_query_log( + self, project: BigqueryProject + ) -> Iterable[Union[PreparsedQuery, ObservedQuery]]: + + # TODO: support all regions - maybe a config + regions = ["region-us"] + + for region in regions: + # Each region needs to be a different query + query_log_query = _build_enriched_query_log_query( + project_id=project.id, + region=region, + start_time=self.config.window.start_time, + end_time=self.config.window.end_time, + # TODO: filters: deny users based on config + ) + + with self.structured_report.report_exc( + f"Error fetching query log from BigQuery Project {project.id}" + ): + logger.info(f"Fetching query log from BigQuery Project {project.id}") + resp = self.connection.query(query_log_query) + + for i, row in enumerate(resp): + if i % 1000 == 0: + logger.info(f"Processed {i} query log rows") + + try: + entry = self._parse_audit_log_row(row) + except Exception as e: + self.structured_report.warning( + "Error parsing query log row", + context=f"{row}", + exc=e, + ) + else: + yield entry + + def _parse_audit_log_row( + self, row: BigQueryJob + ) -> Union[ObservedQuery, PreparsedQuery]: + + timestamp: datetime = row["creation_time"] + timestamp = timestamp.astimezone(timezone.utc) + + entry = ObservedQuery( + query=row["query"], + session_id=row["session_id"], + timestamp=row["creation_time"], + user=make_user_urn(row["user_email"]) if row["user_email"] else None, + default_db=row["project_id"], + default_schema=None, + ) + + return entry + + +def _build_enriched_query_log_query( + project_id: str, + region: str, + start_time: datetime, + end_time: datetime, +) -> str: + + audit_start_time = start_time.strftime(BQ_DATETIME_FORMAT) + audit_end_time = end_time.strftime(BQ_DATETIME_FORMAT) + + # NOTE the use of creation_time instead of start_time + # as JOBS table is partitioned by creation_time + # Using this column significantly reduces processed bytes + return f""" + SELECT + job_id, + project_id, + creation_time, + start_time, + end_time, + total_slot_ms, + user_email, + statement_type, + job_type, + query, + destination_table, + referenced_tables, + total_bytes_billed, + total_bytes_processed, + dml_statistics, + session_info.session_id as session_id + FROM + `{project_id}`.`{region}`.INFORMATION_SCHEMA.JOBS + WHERE + creation_time >= '{audit_start_time}' AND + creation_time <= '{audit_end_time}' + """ diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_config.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_config.py index ac9164cd0a000..9e74fb8b496aa 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_config.py @@ -29,7 +29,7 @@ StatefulProfilingConfigMixin, StatefulUsageConfigMixin, ) -from datahub.ingestion.source_config.usage.snowflake_usage import SnowflakeUsageConfig +from datahub.ingestion.source.usage.usage_common import BaseUsageConfig from datahub.utilities.global_warning_util import add_global_warning logger = logging.Logger(__name__) @@ -135,6 +135,17 @@ class SnowflakeIdentifierConfig( ) +class SnowflakeUsageConfig(BaseUsageConfig): + email_domain: Optional[str] = pydantic.Field( + default=None, + description="Email domain of your organization so users can be displayed on UI appropriately.", + ) + apply_view_usage_to_tables: bool = pydantic.Field( + default=False, + description="Whether to apply view's usage to its base tables. If set to True, usage is applied to base tables only.", + ) + + # TODO: SnowflakeConfig is unused except for this inheritance. We should collapse the config inheritance hierarchy. class SnowflakeConfig( SnowflakeIdentifierConfig, diff --git a/metadata-ingestion/src/datahub/ingestion/source_config/bigquery.py b/metadata-ingestion/src/datahub/ingestion/source_config/bigquery.py deleted file mode 100644 index 1b96da4fecbc9..0000000000000 --- a/metadata-ingestion/src/datahub/ingestion/source_config/bigquery.py +++ /dev/null @@ -1,44 +0,0 @@ -import re - -import pydantic - -from datahub.configuration.common import ConfigModel - -# Regexp for sharded tables. -# A sharded table is a table that has a suffix of the form _yyyymmdd or yyyymmdd, where yyyymmdd is a date. -# The regexp checks for valid dates in the suffix (e.g. 20200101, 20200229, 20201231) and if the date is not valid -# then it is not a sharded table. -_BIGQUERY_DEFAULT_SHARDED_TABLE_REGEX: str = ( - "((.+\\D)[_$]?)?(\\d\\d\\d\\d(?:0[1-9]|1[0-2])(?:0[1-9]|[12][0-9]|3[01]))$" -) - - -class BigQueryBaseConfig(ConfigModel): - rate_limit: bool = pydantic.Field( - default=False, description="Should we rate limit requests made to API." - ) - requests_per_min: int = pydantic.Field( - default=60, - description="Used to control number of API calls made per min. Only used when `rate_limit` is set to `True`.", - ) - - temp_table_dataset_prefix: str = pydantic.Field( - default="_", - description="If you are creating temp tables in a dataset with a particular prefix you can use this config to set the prefix for the dataset. This is to support workflows from before bigquery's introduction of temp tables. By default we use `_` because of datasets that begin with an underscore are hidden by default https://cloud.google.com/bigquery/docs/datasets#dataset-naming.", - ) - - sharded_table_pattern: str = pydantic.Field( - deprecated=True, - default=_BIGQUERY_DEFAULT_SHARDED_TABLE_REGEX, - description="The regex pattern to match sharded tables and group as one table. This is a very low level config parameter, only change if you know what you are doing, ", - ) - - @pydantic.validator("sharded_table_pattern") - def sharded_table_pattern_is_a_valid_regexp(cls, v): - try: - re.compile(v) - except Exception as e: - raise ValueError( - f"sharded_table_pattern configuration pattern is invalid. The exception was: {e}" - ) - return v diff --git a/metadata-ingestion/src/datahub/ingestion/source_config/usage/__init__.py b/metadata-ingestion/src/datahub/ingestion/source_config/usage/__init__.py deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/metadata-ingestion/src/datahub/ingestion/source_config/usage/bigquery_usage.py b/metadata-ingestion/src/datahub/ingestion/source_config/usage/bigquery_usage.py deleted file mode 100644 index cc87c791120a8..0000000000000 --- a/metadata-ingestion/src/datahub/ingestion/source_config/usage/bigquery_usage.py +++ /dev/null @@ -1,163 +0,0 @@ -import json -import logging -import os -import tempfile -from datetime import timedelta -from typing import Any, Dict, List, Optional - -import pydantic - -from datahub.configuration import ConfigModel -from datahub.configuration.common import AllowDenyPattern -from datahub.configuration.source_common import EnvConfigMixin -from datahub.configuration.validate_field_removal import pydantic_removed_field -from datahub.configuration.validate_multiline_string import pydantic_multiline_string -from datahub.ingestion.source.usage.usage_common import BaseUsageConfig -from datahub.ingestion.source_config.bigquery import BigQueryBaseConfig - -logger = logging.getLogger(__name__) - - -class BigQueryCredential(ConfigModel): - project_id: str = pydantic.Field(description="Project id to set the credentials") - private_key_id: str = pydantic.Field(description="Private key id") - private_key: str = pydantic.Field( - description="Private key in a form of '-----BEGIN PRIVATE KEY-----\\nprivate-key\\n-----END PRIVATE KEY-----\\n'" - ) - client_email: str = pydantic.Field(description="Client email") - client_id: str = pydantic.Field(description="Client Id") - auth_uri: str = pydantic.Field( - default="https://accounts.google.com/o/oauth2/auth", - description="Authentication uri", - ) - token_uri: str = pydantic.Field( - default="https://oauth2.googleapis.com/token", description="Token uri" - ) - auth_provider_x509_cert_url: str = pydantic.Field( - default="https://www.googleapis.com/oauth2/v1/certs", - description="Auth provider x509 certificate url", - ) - type: str = pydantic.Field( - default="service_account", description="Authentication type" - ) - client_x509_cert_url: Optional[str] = pydantic.Field( - default=None, - description="If not set it will be default to https://www.googleapis.com/robot/v1/metadata/x509/client_email", - ) - - _fix_private_key_newlines = pydantic_multiline_string("private_key") - - @pydantic.root_validator(skip_on_failure=True) - def validate_config(cls, values: Dict[str, Any]) -> Dict[str, Any]: - if values.get("client_x509_cert_url") is None: - values[ - "client_x509_cert_url" - ] = f'https://www.googleapis.com/robot/v1/metadata/x509/{values["client_email"]}' - return values - - def create_credential_temp_file(self) -> str: - with tempfile.NamedTemporaryFile(delete=False) as fp: - cred_json = json.dumps(self.dict(), indent=4, separators=(",", ": ")) - fp.write(cred_json.encode()) - return fp.name - - -class BigQueryUsageConfig(BigQueryBaseConfig, EnvConfigMixin, BaseUsageConfig): - projects: Optional[List[str]] = pydantic.Field( - default=None, - description="List of project ids to ingest usage from. If not specified, will infer from environment.", - ) - project_id: Optional[str] = pydantic.Field( - default=None, - description="Project ID to ingest usage from. If not specified, will infer from environment. Deprecated in favour of projects ", - ) - extra_client_options: dict = pydantic.Field( - default_factory=dict, - description="Additional options to pass to google.cloud.logging_v2.client.Client.", - ) - use_v2_audit_metadata: Optional[bool] = pydantic.Field( - default=False, - description="Whether to ingest logs using the v2 format. Required if use_exported_bigquery_audit_metadata is set to True.", - ) - - bigquery_audit_metadata_datasets: Optional[List[str]] = pydantic.Field( - description="A list of datasets that contain a table named cloudaudit_googleapis_com_data_access which contain BigQuery audit logs, specifically, those containing BigQueryAuditMetadata. It is recommended that the project of the dataset is also specified, for example, projectA.datasetB.", - ) - use_exported_bigquery_audit_metadata: bool = pydantic.Field( - default=False, - description="When configured, use BigQueryAuditMetadata in bigquery_audit_metadata_datasets to compute usage information.", - ) - - use_date_sharded_audit_log_tables: bool = pydantic.Field( - default=False, - description="Whether to read date sharded tables or time partitioned tables when extracting usage from exported audit logs.", - ) - - table_pattern: AllowDenyPattern = pydantic.Field( - default=AllowDenyPattern.allow_all(), - description="List of regex patterns for tables to include/exclude from ingestion.", - ) - dataset_pattern: AllowDenyPattern = pydantic.Field( - default=AllowDenyPattern.allow_all(), - description="List of regex patterns for datasets to include/exclude from ingestion.", - ) - log_page_size: pydantic.PositiveInt = pydantic.Field( - default=1000, - description="", - ) - - query_log_delay: Optional[pydantic.PositiveInt] = pydantic.Field( - default=None, - description="To account for the possibility that the query event arrives after the read event in the audit logs, we wait for at least query_log_delay additional events to be processed before attempting to resolve BigQuery job information from the logs. If query_log_delay is None, it gets treated as an unlimited delay, which prioritizes correctness at the expense of memory usage.", - ) - - max_query_duration: timedelta = pydantic.Field( - default=timedelta(minutes=15), - description="Correction to pad start_time and end_time with. For handling the case where the read happens within our time range but the query completion event is delayed and happens after the configured end time.", - ) - - credential: Optional[BigQueryCredential] = pydantic.Field( - default=None, - description="Bigquery credential. Required if GOOGLE_APPLICATION_CREDENTIALS environment variable is not set. See this example recipe for details", - ) - _credentials_path: Optional[str] = pydantic.PrivateAttr(None) - temp_table_dataset_prefix: str = pydantic.Field( - default="_", - description="If you are creating temp tables in a dataset with a particular prefix you can use this config to set the prefix for the dataset. This is to support workflows from before bigquery's introduction of temp tables. By default we use `_` because of datasets that begin with an underscore are hidden by default https://cloud.google.com/bigquery/docs/datasets#dataset-naming.", - ) - - def __init__(self, **data: Any): - super().__init__(**data) - if self.credential: - self._credentials_path = self.credential.create_credential_temp_file() - logger.debug( - f"Creating temporary credential file at {self._credentials_path}" - ) - os.environ["GOOGLE_APPLICATION_CREDENTIALS"] = self._credentials_path - - @pydantic.validator("project_id") - def note_project_id_deprecation(cls, v, values, **kwargs): - logger.warning( - "bigquery-usage project_id option is deprecated; use projects instead" - ) - values["projects"] = [v] - return None - - # BigQuery project-ids are globally unique. - platform_instance_not_supported_for_bigquery = pydantic_removed_field( - "platform_instance" - ) - - @pydantic.validator("use_exported_bigquery_audit_metadata") - def use_exported_bigquery_audit_metadata_uses_v2(cls, v, values): - if v is True and not values["use_v2_audit_metadata"]: - raise ValueError( - "To use exported BigQuery audit metadata, you must also use v2 audit metadata" - ) - return v - - def get_allow_pattern_string(self) -> str: - return "|".join(self.table_pattern.allow) if self.table_pattern else "" - - def get_deny_pattern_string(self) -> str: - return "|".join(self.table_pattern.deny) if self.table_pattern else "" diff --git a/metadata-ingestion/src/datahub/ingestion/source_config/usage/snowflake_usage.py b/metadata-ingestion/src/datahub/ingestion/source_config/usage/snowflake_usage.py deleted file mode 100644 index a0e79f62240ee..0000000000000 --- a/metadata-ingestion/src/datahub/ingestion/source_config/usage/snowflake_usage.py +++ /dev/null @@ -1,19 +0,0 @@ -import logging -from typing import Optional - -import pydantic - -from datahub.ingestion.source.usage.usage_common import BaseUsageConfig - -logger = logging.getLogger(__name__) - - -class SnowflakeUsageConfig(BaseUsageConfig): - email_domain: Optional[str] = pydantic.Field( - default=None, - description="Email domain of your organization so users can be displayed on UI appropriately.", - ) - apply_view_usage_to_tables: bool = pydantic.Field( - default=False, - description="Whether to apply view's usage to its base tables. If set to True, usage is applied to base tables only.", - ) diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py index fbf6f954f82bb..0774f0efcb46d 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py @@ -83,6 +83,9 @@ class LoggedQuery: default_schema: Optional[str] +ObservedQuery = LoggedQuery + + @dataclasses.dataclass class ViewDefinition: view_definition: str @@ -469,7 +472,10 @@ def is_allowed_table(self, urn: UrnStr) -> bool: return self._is_allowed_table(self._name_from_urn(urn)) def add( - self, item: Union[KnownQueryLineageInfo, KnownLineageMapping, PreparsedQuery] + self, + item: Union[ + KnownQueryLineageInfo, KnownLineageMapping, PreparsedQuery, ObservedQuery + ], ) -> None: if isinstance(item, KnownQueryLineageInfo): self.add_known_query_lineage(item) @@ -477,6 +483,14 @@ def add( self.add_known_lineage_mapping(item.upstream_urn, item.downstream_urn) elif isinstance(item, PreparsedQuery): self.add_preparsed_query(item) + elif isinstance(item, ObservedQuery): + self.add_observed_query( + query=item.query, + default_db=item.default_db, + default_schema=item.default_schema, + session_id=item.session_id, + query_timestamp=item.timestamp, + ) else: raise ValueError(f"Cannot add unknown item type: {type(item)}") diff --git a/metadata-ingestion/tests/integration/fivetran/test_fivetran.py b/metadata-ingestion/tests/integration/fivetran/test_fivetran.py index 887dcce4b7e9b..5e0e20234cc99 100644 --- a/metadata-ingestion/tests/integration/fivetran/test_fivetran.py +++ b/metadata-ingestion/tests/integration/fivetran/test_fivetran.py @@ -9,6 +9,7 @@ from datahub.configuration.common import ConfigurationWarning from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.run.pipeline import Pipeline +from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryCredential from datahub.ingestion.source.fivetran.config import ( BigQueryDestinationConfig, FivetranSourceConfig, @@ -16,7 +17,6 @@ ) from datahub.ingestion.source.fivetran.fivetran import FivetranSource from datahub.ingestion.source.fivetran.fivetran_query import FivetranLogQuery -from datahub.ingestion.source_config.usage.bigquery_usage import BigQueryCredential from tests.test_helpers import mce_helpers FROZEN_TIME = "2022-06-07 17:00:00" From 19d9b61e712cdfc11bc101edb854bbd89305fc72 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Mon, 22 Jul 2024 19:50:47 +0530 Subject: [PATCH 02/16] refractor to separate BigQueryFilter, BigQueryIdentifierBuilder - update filtering in bigquery-queries source to use allow deny patterns --- .../ingestion/source/bigquery_v2/bigquery.py | 87 ++++------------ .../source/bigquery_v2/bigquery_audit.py | 98 ++++++++++++------- .../source/bigquery_v2/bigquery_config.py | 60 +++--------- .../source/bigquery_v2/bigquery_queries.py | 12 ++- .../source/bigquery_v2/bigquery_schema.py | 56 +++++++++++ .../source/bigquery_v2/bigquery_schema_gen.py | 43 ++++---- .../bigquery_v2/bigquery_test_connection.py | 5 +- .../ingestion/source/bigquery_v2/common.py | 67 +++++++++++++ .../ingestion/source/bigquery_v2/lineage.py | 15 ++- .../source/bigquery_v2/queries_extractor.py | 98 +++++++------------ .../ingestion/source/bigquery_v2/usage.py | 15 ++- .../bigquery/test_bigquery_usage.py | 6 +- .../tests/unit/test_bigquery_lineage.py | 6 +- .../tests/unit/test_bigquery_source.py | 70 ++++++++++--- .../tests/unit/test_bigquery_usage.py | 6 +- .../unit/test_bigqueryv2_usage_source.py | 6 +- 16 files changed, 373 insertions(+), 277 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py index 7a96b2f0643ab..36022ca8d7305 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py @@ -4,7 +4,6 @@ import os from typing import Iterable, List, Optional -from datahub.emitter.mce_builder import make_dataset_urn from datahub.ingestion.api.common import PipelineContext from datahub.ingestion.api.decorators import ( SupportStatus, @@ -21,15 +20,12 @@ TestConnectionReport, ) from datahub.ingestion.api.workunit import MetadataWorkUnit -from datahub.ingestion.source.bigquery_v2.bigquery_audit import ( - BigqueryTableIdentifier, - BigQueryTableRef, -) +from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryV2Config from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report from datahub.ingestion.source.bigquery_v2.bigquery_schema import ( - BigqueryProject, BigQuerySchemaApi, + get_projects, ) from datahub.ingestion.source.bigquery_v2.bigquery_schema_gen import ( BigQuerySchemaGenerator, @@ -37,6 +33,10 @@ from datahub.ingestion.source.bigquery_v2.bigquery_test_connection import ( BigQueryTestConnection, ) +from datahub.ingestion.source.bigquery_v2.common import ( + BigQueryFilter, + BigQueryIdentifierBuilder, +) from datahub.ingestion.source.bigquery_v2.lineage import BigqueryLineageExtractor from datahub.ingestion.source.bigquery_v2.profiler import BigqueryProfiler from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor @@ -122,6 +122,8 @@ def __init__(self, ctx: PipelineContext, config: BigQueryV2Config): ) self.sql_parser_schema_resolver = self._init_schema_resolver() + self.filters = BigQueryFilter(self.config, self.report) + self.identifiers = BigQueryIdentifierBuilder(self.config, self.report) redundant_lineage_run_skip_handler: Optional[ RedundantLineageRunSkipHandler @@ -138,7 +140,7 @@ def __init__(self, ctx: PipelineContext, config: BigQueryV2Config): self.lineage_extractor = BigqueryLineageExtractor( config, self.report, - dataset_urn_builder=self.gen_dataset_urn_from_raw_ref, + identifiers=self.identifiers, redundant_run_skip_handler=redundant_lineage_run_skip_handler, ) @@ -155,7 +157,7 @@ def __init__(self, ctx: PipelineContext, config: BigQueryV2Config): config, self.report, schema_resolver=self.sql_parser_schema_resolver, - dataset_urn_builder=self.gen_dataset_urn_from_raw_ref, + identifiers=self.identifiers, redundant_run_skip_handler=redundant_usage_run_skip_handler, ) @@ -178,7 +180,7 @@ def __init__(self, ctx: PipelineContext, config: BigQueryV2Config): self.domain_registry, self.sql_parser_schema_resolver, self.profiler, - self.gen_dataset_urn, + self.identifiers, ) self.add_config_to_report() @@ -231,7 +233,12 @@ def get_workunit_processors(self) -> List[Optional[MetadataWorkUnitProcessor]]: ] def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: - projects = self._get_projects() + projects = get_projects( + self.config, + self.bq_schema_extractor.schema_api, + self.report, + self.filters, + ) if not projects: return @@ -255,66 +262,6 @@ def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: self.bq_schema_extractor.table_refs, ) - def _get_projects(self) -> List[BigqueryProject]: - logger.info("Getting projects") - if self.config.project_ids or self.config.project_id: - project_ids = self.config.project_ids or [self.config.project_id] # type: ignore - return [ - BigqueryProject(id=project_id, name=project_id) - for project_id in project_ids - ] - else: - return list(self._query_project_list()) - - def _query_project_list(self) -> Iterable[BigqueryProject]: - try: - projects = self.bigquery_data_dictionary.get_projects() - - if ( - not projects - ): # Report failure on exception and if empty list is returned - self.report.failure( - title="Get projects didn't return any project. ", - message="Maybe resourcemanager.projects.get permission is missing for the service account. " - "You can assign predefined roles/bigquery.metadataViewer role to your service account.", - ) - except Exception as e: - self.report.failure( - title="Failed to get BigQuery Projects", - message="Maybe resourcemanager.projects.get permission is missing for the service account. " - "You can assign predefined roles/bigquery.metadataViewer role to your service account.", - exc=e, - ) - projects = [] - - for project in projects: - if self.config.project_id_pattern.allowed(project.id): - yield project - else: - self.report.report_dropped(project.id) - - def gen_dataset_urn( - self, project_id: str, dataset_name: str, table: str, use_raw_name: bool = False - ) -> str: - datahub_dataset_name = BigqueryTableIdentifier(project_id, dataset_name, table) - return make_dataset_urn( - self.platform, - ( - str(datahub_dataset_name) - if not use_raw_name - else datahub_dataset_name.raw_table_name() - ), - self.config.env, - ) - - def gen_dataset_urn_from_raw_ref(self, ref: BigQueryTableRef) -> str: - return self.gen_dataset_urn( - ref.table_identifier.project_id, - ref.table_identifier.dataset, - ref.table_identifier.table, - use_raw_name=True, - ) - def get_report(self) -> BigQueryV2Report: return self.report diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py index bc00517567bbd..c0afdf65cee5d 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py @@ -294,19 +294,27 @@ def from_entry( job.get("jobName", {}).get("jobId"), ), project_id=job.get("jobName", {}).get("projectId"), - default_dataset=job_query_conf["defaultDataset"] - if job_query_conf["defaultDataset"] - else None, - start_time=parser.parse(job["jobStatistics"]["startTime"]) - if job["jobStatistics"]["startTime"] - else None, - end_time=parser.parse(job["jobStatistics"]["endTime"]) - if job["jobStatistics"]["endTime"] - else None, - numAffectedRows=int(job["jobStatistics"]["queryOutputRowCount"]) - if "queryOutputRowCount" in job["jobStatistics"] - and job["jobStatistics"]["queryOutputRowCount"] - else None, + default_dataset=( + job_query_conf["defaultDataset"] + if job_query_conf["defaultDataset"] + else None + ), + start_time=( + parser.parse(job["jobStatistics"]["startTime"]) + if job["jobStatistics"]["startTime"] + else None + ), + end_time=( + parser.parse(job["jobStatistics"]["endTime"]) + if job["jobStatistics"]["endTime"] + else None + ), + numAffectedRows=( + int(job["jobStatistics"]["queryOutputRowCount"]) + if "queryOutputRowCount" in job["jobStatistics"] + and job["jobStatistics"]["queryOutputRowCount"] + else None + ), statementType=job_query_conf.get("statementType", "UNKNOWN"), ) # destinationTable @@ -376,18 +384,26 @@ def from_exported_bigquery_audit_metadata( query=query_config["query"], job_name=job["jobName"], project_id=QueryEvent._get_project_id_from_job_name(job["jobName"]), - default_dataset=query_config["defaultDataset"] - if query_config.get("defaultDataset") - else None, - start_time=parser.parse(job["jobStats"]["startTime"]) - if job["jobStats"]["startTime"] - else None, - end_time=parser.parse(job["jobStats"]["endTime"]) - if job["jobStats"]["endTime"] - else None, - numAffectedRows=int(query_stats["outputRowCount"]) - if query_stats.get("outputRowCount") - else None, + default_dataset=( + query_config["defaultDataset"] + if query_config.get("defaultDataset") + else None + ), + start_time=( + parser.parse(job["jobStats"]["startTime"]) + if job["jobStats"]["startTime"] + else None + ), + end_time=( + parser.parse(job["jobStats"]["endTime"]) + if job["jobStats"]["endTime"] + else None + ), + numAffectedRows=( + int(query_stats["outputRowCount"]) + if query_stats.get("outputRowCount") + else None + ), statementType=query_config.get("statementType", "UNKNOWN"), ) # jobName @@ -445,18 +461,26 @@ def from_entry_v2( timestamp=row.timestamp, actor_email=payload["authenticationInfo"]["principalEmail"], query=query_config["query"], - default_dataset=query_config["defaultDataset"] - if "defaultDataset" in query_config and query_config["defaultDataset"] - else None, - start_time=parser.parse(job["jobStats"]["startTime"]) - if job["jobStats"]["startTime"] - else None, - end_time=parser.parse(job["jobStats"]["endTime"]) - if job["jobStats"]["endTime"] - else None, - numAffectedRows=int(query_stats["outputRowCount"]) - if "outputRowCount" in query_stats and query_stats["outputRowCount"] - else None, + default_dataset=( + query_config["defaultDataset"] + if "defaultDataset" in query_config and query_config["defaultDataset"] + else None + ), + start_time=( + parser.parse(job["jobStats"]["startTime"]) + if job["jobStats"]["startTime"] + else None + ), + end_time=( + parser.parse(job["jobStats"]["endTime"]) + if job["jobStats"]["endTime"] + else None + ), + numAffectedRows=( + int(query_stats["outputRowCount"]) + if "outputRowCount" in query_stats and query_stats["outputRowCount"] + else None + ), statementType=query_config.get("statementType", "UNKNOWN"), ) query_event.job_name = job.get("jobName") diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py index 27cb0e3670adf..4db76b6c918e2 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py @@ -83,6 +83,19 @@ def sharded_table_pattern_is_a_valid_regexp(cls, v): ) return v + @root_validator(pre=True, skip_on_failure=True) + def project_id_backward_compatibility_configs_set(cls, values: Dict) -> Dict: + project_id = values.pop("project_id", None) + project_ids = values.get("project_ids") + + if not project_ids and project_id: + values["project_ids"] = [project_id] + elif project_ids and project_id: + logging.warning( + "Please use `project_ids` config. Config `project_id` will be ignored." + ) + return values + class BigQueryUsageConfig(BaseUsageConfig): _query_log_delay_removed = pydantic_removed_field("query_log_delay") @@ -195,29 +208,6 @@ def get_sql_alchemy_url(self) -> str: return "bigquery://" -class BigQueryConnection: - _connection: bigquery.Client - - def __init__(self, connection: bigquery.Client): - self._connection = connection - - def native_connection(self) -> bigquery.Client: - return self._connection - - def query(self, query: str) -> Any: - try: - logger.info(f"Query: {query}", stacklevel=2) - resp = self._connection.query(query) - return resp - - except Exception: - # TODO: is it possible to handle any generic error type here ? - raise - - def close(self): - self._connection.close() - - class BigQueryFilterConfig(SQLFilterConfig): project_id_pattern: AllowDenyPattern = Field( @@ -303,7 +293,6 @@ class BigQueryV2Config( BigQueryBaseConfig, BigQueryFilterConfig, BigQueryIdentifierConfig, - # BigQueryFilterConfig must come before (higher precedence) the SQLCommon config, so that the documentation overrides are applied. SQLCommonConfig, StatefulUsageConfigMixin, StatefulLineageConfigMixin, @@ -386,13 +375,6 @@ def have_table_data_read_permission(self) -> bool: description="Maximum number of columns to process in a table. This is a low level config property which should be touched with care. This restriction is needed because excessively wide tables can result in failure to ingest the schema.", ) - # TODO- remove project_id - # The inheritance hierarchy is wonky here, but these options need modifications. - project_id: Optional[str] = Field( - default=None, - description="[deprecated] Use project_id_pattern or project_ids instead.", - ) - lineage_use_sql_parser: bool = Field( default=True, description="Use sql parser to resolve view/table lineage.", @@ -499,22 +481,6 @@ def have_table_data_read_permission(self) -> bool: " Set to 1 to disable.", ) - @root_validator(pre=False, skip_on_failure=True) - def project_id_backward_compatibility_configs_set(cls, values: Dict) -> Dict: - project_id = values.get("project_id") - project_id_pattern = values.get("project_id_pattern") - - if project_id_pattern == AllowDenyPattern.allow_all() and project_id: - logging.warning( - "project_id_pattern is not set but project_id is set, source will only ingest the project_id project. project_id will be deprecated, please use project_id_pattern instead." - ) - values["project_id_pattern"] = AllowDenyPattern(allow=[f"^{project_id}$"]) - elif project_id_pattern != AllowDenyPattern.allow_all() and project_id: - logging.warning( - "use project_id_pattern whenever possible. project_id will be deprecated, please use project_id_pattern only if possible." - ) - return values - @root_validator(skip_on_failure=True) def profile_default_settings(cls, values: Dict) -> Dict: # Extra default SQLAlchemy option for better connection pooling and threading. diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py index 91a26b630aff7..3f253f1e0e161 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py @@ -18,6 +18,10 @@ BigQuerySchemaApiPerfReport, ) from datahub.ingestion.source.bigquery_v2.bigquery_schema import BigQuerySchemaApi +from datahub.ingestion.source.bigquery_v2.common import ( + BigQueryFilter, + BigQueryIdentifierBuilder, +) from datahub.ingestion.source.bigquery_v2.queries_extractor import ( BigQueryQueriesExtractor, BigQueryQueriesExtractorConfig, @@ -47,6 +51,9 @@ def __init__(self, ctx: PipelineContext, config: BigQueryQueriesSourceConfig): self.config = config self.report = BigQueryQueriesSourceReport() + self.filters = BigQueryFilter(self.config, self.report) + self.identifiers = BigQueryIdentifierBuilder(self.config, self.report) + self.connection = self.config.connection.get_bigquery_client() self.queries_extractor = BigQueryQueriesExtractor( @@ -56,8 +63,8 @@ def __init__(self, ctx: PipelineContext, config: BigQueryQueriesSourceConfig): ), config=self.config, structured_report=self.report, - filter_config=self.config, - identifier_config=self.config, + filters=self.filters, + identifiers=self.identifiers, graph=self.ctx.graph, ) self.report.queries_extractor = self.queries_extractor.report @@ -71,6 +78,7 @@ def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: self.report.window = self.config.window # TODO: Disable auto status processor? + # TODO: Don't emit lineage, usage, operations for ghost entities return self.queries_extractor.get_workunits_internal() def get_report(self) -> BigQueryQueriesSourceReport: diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py index 76ede5248c7c0..3e7bb2794262f 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py @@ -13,12 +13,15 @@ TimePartitioningType, ) +from datahub.ingestion.api.source import SourceReport from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier +from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryBaseConfig from datahub.ingestion.source.bigquery_v2.bigquery_helper import parse_labels from datahub.ingestion.source.bigquery_v2.bigquery_report import ( BigQuerySchemaApiPerfReport, BigQueryV2Report, ) +from datahub.ingestion.source.bigquery_v2.common import BigQueryFilter from datahub.ingestion.source.bigquery_v2.queries import ( BigqueryQuery, BigqueryTableType, @@ -579,3 +582,56 @@ def _make_bigquery_table_snapshot(snapshot: bigquery.Row) -> BigqueryTableSnapsh table=snapshot.base_table_name, ), ) + + +def query_project_list( + schema_api: BigQuerySchemaApi, + report: SourceReport, + filters: BigQueryFilter, +) -> Iterable[BigqueryProject]: + try: + projects = schema_api.get_projects() + + if not projects: # Report failure on exception and if empty list is returned + report.failure( + title="Get projects didn't return any project. ", + message="Maybe resourcemanager.projects.get permission is missing for the service account. " + "You can assign predefined roles/bigquery.metadataViewer role to your service account.", + ) + except Exception as e: + report.failure( + title="Failed to get BigQuery Projects", + message="Maybe resourcemanager.projects.get permission is missing for the service account. " + "You can assign predefined roles/bigquery.metadataViewer role to your service account.", + exc=e, + ) + projects = [] + + for project in projects: + if filters.filter_config.project_id_pattern.allowed(project.id): + yield project + # TODO: suppport reporting dropped projects for queries extractor report + # else: + # report.report_dropped(project.id) + + +def get_projects( + config: BigQueryBaseConfig, + schema_api: BigQuerySchemaApi, + report: SourceReport, + filters: BigQueryFilter, +) -> List[BigqueryProject]: + logger.info("Getting projects") + if config.project_ids: + return [ + BigqueryProject(id=project_id, name=project_id) + for project_id in config.project_ids + ] + else: + return list( + query_project_list( + schema_api, + report, + filters, + ) + ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py index 46ec75edb9734..f0829dddd9fa0 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py @@ -1,7 +1,7 @@ import logging import re from collections import defaultdict -from typing import Callable, Dict, Iterable, List, Optional, Set, Type, Union, cast +from typing import Dict, Iterable, List, Optional, Set, Type, Union, cast from google.cloud.bigquery.table import TableListItem @@ -41,6 +41,7 @@ from datahub.ingestion.source.bigquery_v2.common import ( BQ_EXTERNAL_DATASET_URL_TEMPLATE, BQ_EXTERNAL_TABLE_URL_TEMPLATE, + BigQueryIdentifierBuilder, ) from datahub.ingestion.source.bigquery_v2.profiler import BigqueryProfiler from datahub.ingestion.source.common.subtypes import ( @@ -160,15 +161,15 @@ def __init__( domain_registry: Optional[DomainRegistry], sql_parser_schema_resolver: SchemaResolver, profiler: BigqueryProfiler, - dataset_urn_builder: Callable[[str, str, str], str], + identifiers: BigQueryIdentifierBuilder, ): self.config = config self.report = report - self.bigquery_data_dictionary = bigquery_data_dictionary + self.schema_api = bigquery_data_dictionary self.domain_registry = domain_registry self.sql_parser_schema_resolver = sql_parser_schema_resolver self.profiler = profiler - self.gen_dataset_urn = dataset_urn_builder + self.identifiers = identifiers self.platform: str = "bigquery" self.classification_handler = ClassificationHandler(self.config, self.report) @@ -285,14 +286,12 @@ def _process_project( project_id = bigquery_project.id try: - bigquery_project.datasets = ( - self.bigquery_data_dictionary.get_datasets_for_project_id(project_id) + bigquery_project.datasets = self.schema_api.get_datasets_for_project_id( + project_id ) except Exception as e: - if ( - self.config.project_id or self.config.project_ids - ) and "not enabled BigQuery." in str(e): + if self.config.project_ids and "not enabled BigQuery." in str(e): action_mesage = ( "The project has not enabled BigQuery API. " "Did you mistype project id in recipe ?" @@ -419,7 +418,7 @@ def _process_schema( or self.config.include_views or self.config.include_table_snapshots ): - columns = self.bigquery_data_dictionary.get_columns_for_dataset( + columns = self.schema_api.get_columns_for_dataset( project_id=project_id, dataset_name=dataset_name, column_limit=self.config.column_limit, @@ -459,9 +458,7 @@ def _process_schema( ) elif self.store_table_refs: # Need table_refs to calculate lineage and usage - for table_item in self.bigquery_data_dictionary.list_tables( - dataset_name, project_id - ): + for table_item in self.schema_api.list_tables(dataset_name, project_id): identifier = BigqueryTableIdentifier( project_id=project_id, dataset=dataset_name, @@ -481,7 +478,7 @@ def _process_schema( if self.config.include_views: db_views[dataset_name] = list( - self.bigquery_data_dictionary.get_views_for_dataset( + self.schema_api.get_views_for_dataset( project_id, dataset_name, self.config.is_profiling_enabled(), @@ -500,7 +497,7 @@ def _process_schema( if self.config.include_table_snapshots: db_snapshots[dataset_name] = list( - self.bigquery_data_dictionary.get_snapshots_for_dataset( + self.schema_api.get_snapshots_for_dataset( project_id, dataset_name, self.config.is_profiling_enabled(), @@ -747,7 +744,9 @@ def gen_view_dataset_workunits( viewLogic=view_definition_string or "", ) yield MetadataChangeProposalWrapper( - entityUrn=self.gen_dataset_urn(project_id, dataset_name, table.name), + entityUrn=self.identifiers.gen_dataset_urn( + project_id, dataset_name, table.name + ), aspect=view_properties_aspect, ).as_workunit() @@ -786,7 +785,9 @@ def gen_dataset_workunits( tags_to_add: Optional[List[str]] = None, custom_properties: Optional[Dict[str, str]] = None, ) -> Iterable[MetadataWorkUnit]: - dataset_urn = self.gen_dataset_urn(project_id, dataset_name, table.name) + dataset_urn = self.identifiers.gen_dataset_urn( + project_id, dataset_name, table.name + ) status = Status(removed=False) yield MetadataChangeProposalWrapper( @@ -999,7 +1000,7 @@ def get_tables_for_dataset( for table_item in table_items: items_to_get[table_item] = table_items[table_item] if len(items_to_get) % max_batch_size == 0: - yield from self.bigquery_data_dictionary.get_tables_for_dataset( + yield from self.schema_api.get_tables_for_dataset( project_id, dataset_name, items_to_get, @@ -1009,7 +1010,7 @@ def get_tables_for_dataset( items_to_get.clear() if items_to_get: - yield from self.bigquery_data_dictionary.get_tables_for_dataset( + yield from self.schema_api.get_tables_for_dataset( project_id, dataset_name, items_to_get, @@ -1028,9 +1029,7 @@ def get_core_table_details( # Dict to store sharded table and the last seen max shard id sharded_tables: Dict[str, TableListItem] = {} - for table in self.bigquery_data_dictionary.list_tables( - dataset_name, project_id - ): + for table in self.schema_api.list_tables(dataset_name, project_id): table_identifier = BigqueryTableIdentifier( project_id=project_id, dataset=dataset_name, diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_test_connection.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_test_connection.py index 3aac78c154b2e..362511b92a9d3 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_test_connection.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_test_connection.py @@ -12,6 +12,7 @@ from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryV2Config from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report from datahub.ingestion.source.bigquery_v2.bigquery_schema import BigQuerySchemaApi +from datahub.ingestion.source.bigquery_v2.common import BigQueryIdentifierBuilder from datahub.ingestion.source.bigquery_v2.lineage import BigqueryLineageExtractor from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor from datahub.sql_parsing.schema_resolver import SchemaResolver @@ -134,7 +135,7 @@ def lineage_capability_test( report: BigQueryV2Report, ) -> CapabilityReport: lineage_extractor = BigqueryLineageExtractor( - connection_conf, report, lambda ref: "" + connection_conf, report, BigQueryIdentifierBuilder(connection_conf, report) ) for project_id in project_ids: try: @@ -158,7 +159,7 @@ def usage_capability_test( connection_conf, report, schema_resolver=SchemaResolver(platform="bigquery"), - dataset_urn_builder=lambda ref: "", + identifiers=BigQueryIdentifierBuilder(connection_conf, report), ) for project_id in project_ids: try: diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py index e38ab07855b8b..5c8c844763c91 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py @@ -1,5 +1,72 @@ +from datahub.configuration.pattern_utils import is_schema_allowed +from datahub.emitter.mce_builder import make_dataset_urn +from datahub.ingestion.api.source import SourceReport +from datahub.ingestion.source.bigquery_v2.bigquery_audit import ( + BigqueryTableIdentifier, + BigQueryTableRef, +) +from datahub.ingestion.source.bigquery_v2.bigquery_config import ( + BigQueryFilterConfig, + BigQueryIdentifierConfig, +) + BQ_DATETIME_FORMAT = "%Y-%m-%dT%H:%M:%SZ" BQ_DATE_SHARD_FORMAT = "%Y%m%d" BQ_EXTERNAL_TABLE_URL_TEMPLATE = "https://console.cloud.google.com/bigquery?project={project}&ws=!1m5!1m4!4m3!1s{project}!2s{dataset}!3s{table}" BQ_EXTERNAL_DATASET_URL_TEMPLATE = "https://console.cloud.google.com/bigquery?project={project}&ws=!1m4!1m3!3m2!1s{project}!2s{dataset}" + + +class BigQueryIdentifierBuilder: + platform = "bigquery" + + def __init__( + self, + identifier_config: BigQueryIdentifierConfig, + structured_reporter: SourceReport, + ) -> None: + self.identifier_config = identifier_config + self.structured_reporter = structured_reporter + + def gen_dataset_urn( + self, project_id: str, dataset_name: str, table: str, use_raw_name: bool = False + ) -> str: + datahub_dataset_name = BigqueryTableIdentifier(project_id, dataset_name, table) + return make_dataset_urn( + self.platform, + ( + str(datahub_dataset_name) + if not use_raw_name + else datahub_dataset_name.raw_table_name() + ), + self.identifier_config.env, + ) + + def gen_dataset_urn_from_raw_ref(self, ref: BigQueryTableRef) -> str: + return self.gen_dataset_urn( + ref.table_identifier.project_id, + ref.table_identifier.dataset, + ref.table_identifier.table, + use_raw_name=True, + ) + + +class BigQueryFilter: + def __init__( + self, filter_config: BigQueryFilterConfig, structured_reporter: SourceReport + ) -> None: + self.filter_config = filter_config + self.structured_reporter = structured_reporter + + def is_allowed(self, table_id: BigqueryTableIdentifier) -> bool: + return ( + self.filter_config.project_id_pattern.allowed(table_id.project_id) + and is_schema_allowed( + self.filter_config.dataset_pattern, + table_id.dataset, + table_id.project_id, + self.filter_config.match_fully_qualified_names, + ) + and self.filter_config.table_pattern.allowed(str(table_id)) + # TODO: use view_pattern ? + ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py index 496bd64d3b4fe..aff73961e8c9a 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/lineage.py @@ -42,7 +42,10 @@ BigQuerySchemaApi, BigqueryTableSnapshot, ) -from datahub.ingestion.source.bigquery_v2.common import BQ_DATETIME_FORMAT +from datahub.ingestion.source.bigquery_v2.common import ( + BQ_DATETIME_FORMAT, + BigQueryIdentifierBuilder, +) from datahub.ingestion.source.bigquery_v2.queries import ( BQ_FILTER_RULE_TEMPLATE_V2_LINEAGE, bigquery_audit_metadata_query_template_lineage, @@ -225,12 +228,12 @@ def __init__( self, config: BigQueryV2Config, report: BigQueryV2Report, - dataset_urn_builder: Callable[[BigQueryTableRef], str], + identifiers: BigQueryIdentifierBuilder, redundant_run_skip_handler: Optional[RedundantLineageRunSkipHandler] = None, ): self.config = config self.report = report - self.dataset_urn_builder = dataset_urn_builder + self.identifiers = identifiers self.audit_log_api = BigQueryAuditLogApi( report.audit_log_api_perf, self.config.rate_limit, @@ -427,7 +430,7 @@ def populate_snapshot_lineage( def gen_lineage_workunits_for_table( self, lineage: Dict[str, Set[LineageEdge]], table_ref: BigQueryTableRef ) -> Iterable[MetadataWorkUnit]: - dataset_urn = self.dataset_urn_builder(table_ref) + dataset_urn = self.identifiers.gen_dataset_urn_from_raw_ref(table_ref) lineage_info = self.get_lineage_for_table( bq_table=table_ref, @@ -870,7 +873,9 @@ def get_lineage_for_table( # even if the lineage is same but the order is different. for upstream in sorted(self.get_upstream_tables(bq_table, lineage_metadata)): upstream_table = BigQueryTableRef.from_string_name(upstream.table) - upstream_table_urn = self.dataset_urn_builder(upstream_table) + upstream_table_urn = self.identifiers.gen_dataset_urn_from_raw_ref( + upstream_table + ) # Generate table-level lineage. upstream_table_class = UpstreamClass( diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 547993777634b..6064b729c8e13 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -9,6 +9,7 @@ from google.cloud.bigquery import Client from pydantic import Field +from datahub.configuration.common import AllowDenyPattern from datahub.configuration.time_window_config import BaseTimeWindowConfig from datahub.emitter.mce_builder import make_user_urn from datahub.ingestion.api.report import Report @@ -17,16 +18,17 @@ from datahub.ingestion.api.workunit import MetadataWorkUnit from datahub.ingestion.graph.client import DataHubGraph from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier -from datahub.ingestion.source.bigquery_v2.bigquery_config import ( - BigQueryBaseConfig, - BigQueryFilterConfig, - BigQueryIdentifierConfig, -) +from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryBaseConfig from datahub.ingestion.source.bigquery_v2.bigquery_schema import ( BigqueryProject, BigQuerySchemaApi, + get_projects, +) +from datahub.ingestion.source.bigquery_v2.common import ( + BQ_DATETIME_FORMAT, + BigQueryFilter, + BigQueryIdentifierBuilder, ) -from datahub.ingestion.source.bigquery_v2.common import BQ_DATETIME_FORMAT from datahub.ingestion.source.usage.usage_common import BaseUsageConfig from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.sql_parsing.sql_parsing_aggregator import ( @@ -85,6 +87,11 @@ class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): hidden_from_docs=True, ) + user_email_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + description="regex patterns for user emails to filter in usage.", + ) + include_lineage: bool = True include_queries: bool = True include_usage_statistics: bool = True @@ -106,9 +113,8 @@ def __init__( schema_api: BigQuerySchemaApi, config: BigQueryQueriesExtractorConfig, structured_report: SourceReport, - # TODO: take mixins for filter and identifier instead of config itself - filter_config: BigQueryFilterConfig, - identifier_config: BigQueryIdentifierConfig, + filters: BigQueryFilter, + identifiers: BigQueryIdentifierBuilder, graph: Optional[DataHubGraph] = None, schema_resolver: Optional[SchemaResolver] = None, discovered_tables: Optional[List[str]] = None, @@ -116,8 +122,8 @@ def __init__( self.connection = connection self.config = config - self.filter_config = filter_config - self.identifier_config = identifier_config + self.filters = filters + self.identifiers = identifiers self.schema_api = schema_api self.report = BigQueryQueriesExtractorReport() # self.filters = filters @@ -126,9 +132,9 @@ def __init__( self._structured_report = structured_report self.aggregator = SqlParsingAggregator( - platform="bigquery", - platform_instance=self.identifier_config.platform_instance, - env=self.identifier_config.env, + platform=self.identifiers.platform, + platform_instance=self.identifiers.identifier_config.platform_instance, + env=self.identifiers.identifier_config.env, schema_resolver=schema_resolver, graph=graph, eager_graph_load=False, @@ -140,7 +146,7 @@ def __init__( bucket_duration=self.config.window.bucket_duration, start_time=self.config.window.start_time, end_time=self.config.window.end_time, - # TODO make the rest of the fields configurable + user_email_pattern=self.config.user_email_pattern, ), generate_operations=self.config.include_operations, is_temp_table=self.is_temp_table, @@ -170,49 +176,11 @@ def is_temp_table(self, name: str) -> bool: ) def is_allowed_table(self, name: str) -> bool: - if ( - self.discovered_tables - and str(BigqueryTableIdentifier.from_string_name(name)) - not in self.discovered_tables - ): + table_id = BigqueryTableIdentifier.from_string_name(name) + if self.discovered_tables and str(table_id) not in self.discovered_tables: return False - return True - - # TODO: Remove the code duplication. Also present in main bigquery source - def _get_projects(self) -> List[BigqueryProject]: - logger.info("Getting projects") - if self.config.project_ids: - return [ - BigqueryProject(id=project_id, name=project_id) - for project_id in self.config.project_ids - ] - else: - return list(self._query_project_list()) - - def _query_project_list(self) -> Iterable[BigqueryProject]: - try: - projects = self.schema_api.get_projects() - - if ( - not projects - ): # Report failure on exception and if empty list is returned - self.structured_report.failure( - title="Get projects didn't return any project. ", - message="Maybe resourcemanager.projects.get permission is missing for the service account. " - "You can assign predefined roles/bigquery.metadataViewer role to your service account.", - ) - except Exception as e: - self.structured_report.failure( - title="Failed to get BigQuery Projects", - message="Maybe resourcemanager.projects.get permission is missing for the service account. " - "You can assign predefined roles/bigquery.metadataViewer role to your service account.", - exc=e, - ) - projects = [] - for project in projects: - if self.filter_config.project_id_pattern.allowed(project.id): - yield project + return self.filters.is_allowed(table_id) def get_workunits_internal( self, @@ -234,7 +202,9 @@ def get_workunits_internal( entry: Union[PreparsedQuery, ObservedQuery] with self.report.query_log_fetch_timer: - for project in self._get_projects(): + for project in get_projects( + self.config, self.schema_api, self.structured_report, self.filters + ): for entry in self.fetch_query_log(project): queries.append(entry) @@ -248,8 +218,9 @@ def fetch_query_log( self, project: BigqueryProject ) -> Iterable[Union[PreparsedQuery, ObservedQuery]]: - # TODO: support all regions - maybe a config - regions = ["region-us"] + # Multi-regions from https://cloud.google.com/bigquery/docs/locations#supported_locations + regions = ["region-us", "region-eu"] + # TODO: support other regions as required - via a config for region in regions: # Each region needs to be a different query @@ -311,9 +282,9 @@ def _build_enriched_query_log_query( audit_start_time = start_time.strftime(BQ_DATETIME_FORMAT) audit_end_time = end_time.strftime(BQ_DATETIME_FORMAT) - # NOTE the use of creation_time instead of start_time - # as JOBS table is partitioned by creation_time - # Using this column significantly reduces processed bytes + # NOTE the use of creation_time as timestamp here + # as JOBS table is partitioned by creation_time. + # Using this column filter significantly reduces processed bytes. return f""" SELECT job_id, @@ -336,5 +307,6 @@ def _build_enriched_query_log_query( `{project_id}`.`{region}`.INFORMATION_SCHEMA.JOBS WHERE creation_time >= '{audit_start_time}' AND - creation_time <= '{audit_end_time}' + creation_time <= '{audit_end_time}' AND + error_result is not null """ diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py index 6824d630a2277..1135a0956a2a7 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py @@ -44,7 +44,10 @@ ) from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryV2Config from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report -from datahub.ingestion.source.bigquery_v2.common import BQ_DATETIME_FORMAT +from datahub.ingestion.source.bigquery_v2.common import ( + BQ_DATETIME_FORMAT, + BigQueryIdentifierBuilder, +) from datahub.ingestion.source.bigquery_v2.queries import ( BQ_FILTER_RULE_TEMPLATE_V2_USAGE, bigquery_audit_metadata_query_template_usage, @@ -313,13 +316,13 @@ def __init__( report: BigQueryV2Report, *, schema_resolver: SchemaResolver, - dataset_urn_builder: Callable[[BigQueryTableRef], str], + identifiers: BigQueryIdentifierBuilder, redundant_run_skip_handler: Optional[RedundantUsageRunSkipHandler] = None, ): self.config: BigQueryV2Config = config self.report: BigQueryV2Report = report self.schema_resolver = schema_resolver - self.dataset_urn_builder = dataset_urn_builder + self.identifiers = identifiers # Replace hash of query with uuid if there are hash conflicts self.uuid_to_query: Dict[str, str] = {} @@ -404,7 +407,9 @@ def _get_workunits_internal( bucket_duration=self.config.bucket_duration, ), dataset_urns={ - self.dataset_urn_builder(BigQueryTableRef.from_string_name(ref)) + self.identifiers.gen_dataset_urn_from_raw_ref( + BigQueryTableRef.from_string_name(ref) + ) for ref in table_refs }, ) @@ -535,7 +540,7 @@ def _generate_usage_workunits( user_freq=entry.user_freq, column_freq=entry.column_freq, bucket_duration=self.config.bucket_duration, - resource_urn_builder=self.dataset_urn_builder, + resource_urn_builder=self.identifiers.gen_dataset_urn_from_raw_ref, top_n_queries=self.config.usage.top_n_queries, format_sql_queries=self.config.usage.format_sql_queries, queries_character_limit=self.config.usage.queries_character_limit, diff --git a/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py b/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py index 92ad7b383dc1c..9cb80ff02657b 100644 --- a/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py +++ b/metadata-ingestion/tests/performance/bigquery/test_bigquery_usage.py @@ -6,12 +6,12 @@ import humanfriendly import psutil -from datahub.emitter.mce_builder import make_dataset_urn from datahub.ingestion.source.bigquery_v2.bigquery_config import ( BigQueryUsageConfig, BigQueryV2Config, ) from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report +from datahub.ingestion.source.bigquery_v2.common import BigQueryIdentifierBuilder from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.utilities.perf_timer import PerfTimer @@ -49,9 +49,7 @@ def run_test(): config, report, schema_resolver=SchemaResolver(platform="bigquery"), - dataset_urn_builder=lambda ref: make_dataset_urn( - "bigquery", str(ref.table_identifier) - ), + identifiers=BigQueryIdentifierBuilder(config, report), ) report.set_ingestion_stage("All", "Event Generation") diff --git a/metadata-ingestion/tests/unit/test_bigquery_lineage.py b/metadata-ingestion/tests/unit/test_bigquery_lineage.py index 5d8c040b4123b..6bd5cc4d3226e 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_lineage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_lineage.py @@ -3,13 +3,13 @@ import pytest -import datahub.emitter.mce_builder as builder from datahub.ingestion.source.bigquery_v2.bigquery_audit import ( BigQueryTableRef, QueryEvent, ) from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryV2Config from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report +from datahub.ingestion.source.bigquery_v2.common import BigQueryIdentifierBuilder from datahub.ingestion.source.bigquery_v2.lineage import ( BigqueryLineageExtractor, LineageEdge, @@ -83,7 +83,7 @@ def test_lineage_with_timestamps(lineage_entries: List[QueryEvent]) -> None: config = BigQueryV2Config() report = BigQueryV2Report() extractor: BigqueryLineageExtractor = BigqueryLineageExtractor( - config, report, lambda x: builder.make_dataset_urn("bigquery", str(x)) + config, report, BigQueryIdentifierBuilder(config, report) ) bq_table = BigQueryTableRef.from_string_name( @@ -108,7 +108,7 @@ def test_column_level_lineage(lineage_entries: List[QueryEvent]) -> None: config = BigQueryV2Config(extract_column_lineage=True, incremental_lineage=False) report = BigQueryV2Report() extractor: BigqueryLineageExtractor = BigqueryLineageExtractor( - config, report, lambda x: builder.make_dataset_urn("bigquery", str(x)) + config, report, BigQueryIdentifierBuilder(config, report) ) bq_table = BigQueryTableRef.from_string_name( diff --git a/metadata-ingestion/tests/unit/test_bigquery_source.py b/metadata-ingestion/tests/unit/test_bigquery_source.py index 746cf9b0acfc3..7edfd8586f1df 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_source.py +++ b/metadata-ingestion/tests/unit/test_bigquery_source.py @@ -31,6 +31,7 @@ BigqueryTable, BigqueryTableSnapshot, BigqueryView, + get_projects, ) from datahub.ingestion.source.bigquery_v2.bigquery_schema_gen import ( BigQuerySchemaGenerator, @@ -179,7 +180,12 @@ def test_get_projects_with_project_ids(get_bq_client_mock): } ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) - assert source._get_projects() == [ + assert get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) == [ BigqueryProject("test-1", "test-1"), BigqueryProject("test-2", "test-2"), ] @@ -189,7 +195,12 @@ def test_get_projects_with_project_ids(get_bq_client_mock): {"project_ids": ["test-1", "test-2"], "project_id": "test-3"} ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test2")) - assert source._get_projects() == [ + assert get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) == [ BigqueryProject("test-1", "test-1"), BigqueryProject("test-2", "test-2"), ] @@ -207,7 +218,12 @@ def test_get_projects_with_project_ids_overrides_project_id_pattern( } ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) - projects = source._get_projects() + projects = get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) assert projects == [ BigqueryProject(id="test-project", name="test-project"), BigqueryProject(id="test-project-2", name="test-project-2"), @@ -220,8 +236,10 @@ def test_platform_instance_config_always_none(): ) assert config.platform_instance is None - config = BigQueryV2Config(platform_instance="something", project_id="project_id") - assert config.project_id == "project_id" + config = BigQueryV2Config.parse_obj( + dict(platform_instance="something", project_id="project_id") + ) + assert config.project_ids == ["project_id"] assert config.platform_instance is None @@ -268,7 +286,12 @@ def test_get_projects_with_single_project_id(get_bq_client_mock): get_bq_client_mock.return_value = client_mock config = BigQueryV2Config.parse_obj({"project_id": "test-3"}) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) - assert source._get_projects() == [ + assert get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) == [ BigqueryProject("test-3", "test-3"), ] assert client_mock.list_projects.call_count == 0 @@ -300,7 +323,12 @@ def test_get_projects_by_list(get_bq_client_mock): config = BigQueryV2Config.parse_obj({}) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) - assert source._get_projects() == [ + assert get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) == [ BigqueryProject("test-1", "one"), BigqueryProject("test-2", "two"), BigqueryProject("test-3", "three"), @@ -321,7 +349,12 @@ def test_get_projects_filter_by_pattern(get_bq_client_mock, get_projects_mock): {"project_id_pattern": {"deny": ["^test-project$"]}} ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) - projects = source._get_projects() + projects = get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) assert projects == [ BigqueryProject(id="test-project-2", name="Test Project 2"), ] @@ -336,7 +369,12 @@ def test_get_projects_list_empty(get_bq_client_mock, get_projects_mock): {"project_id_pattern": {"deny": ["^test-project$"]}} ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) - projects = source._get_projects() + projects = get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) assert len(source.report.failures) == 1 assert projects == [] @@ -357,7 +395,12 @@ def test_get_projects_list_failure( source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) caplog.clear() with caplog.at_level(logging.ERROR): - projects = source._get_projects() + projects = get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) assert len(caplog.records) == 2 assert error_str in caplog.records[0].msg assert len(source.report.failures) == 1 @@ -373,7 +416,12 @@ def test_get_projects_list_fully_filtered(get_projects_mock, get_bq_client_mock) {"project_id_pattern": {"deny": ["^test-project$"]}} ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) - projects = source._get_projects() + projects = get_projects( + source.config, + source.bq_schema_extractor.schema_api, + source.report, + source.filters, + ) assert len(source.report.failures) == 0 assert projects == [] diff --git a/metadata-ingestion/tests/unit/test_bigquery_usage.py b/metadata-ingestion/tests/unit/test_bigquery_usage.py index f476e62dd0170..1c2b08915691f 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_usage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_usage.py @@ -8,7 +8,6 @@ from freezegun import freeze_time from datahub.configuration.time_window_config import BucketDuration -from datahub.emitter.mce_builder import make_dataset_urn from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.ingestion.api.workunit import MetadataWorkUnit from datahub.ingestion.source.bigquery_v2.bigquery_audit import ( @@ -23,6 +22,7 @@ BigQueryV2Config, ) from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report +from datahub.ingestion.source.bigquery_v2.common import BigQueryIdentifierBuilder from datahub.ingestion.source.bigquery_v2.usage import ( OPERATION_STATEMENT_TYPES, BigQueryUsageExtractor, @@ -204,9 +204,7 @@ def usage_extractor(config: BigQueryV2Config) -> BigQueryUsageExtractor: config, report, schema_resolver=SchemaResolver(platform="bigquery"), - dataset_urn_builder=lambda ref: make_dataset_urn( - "bigquery", str(ref.table_identifier) - ), + identifiers=BigQueryIdentifierBuilder(config, report), ) diff --git a/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py b/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py index 21787af1b0cb9..63de742b201a9 100644 --- a/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py +++ b/metadata-ingestion/tests/unit/test_bigqueryv2_usage_source.py @@ -12,6 +12,7 @@ unquote_and_decode_unicode_escape_seq, ) from datahub.ingestion.source.bigquery_v2.bigquery_report import BigQueryV2Report +from datahub.ingestion.source.bigquery_v2.common import BigQueryIdentifierBuilder from datahub.ingestion.source.bigquery_v2.usage import BigQueryUsageExtractor from datahub.sql_parsing.schema_resolver import SchemaResolver @@ -117,11 +118,12 @@ def test_bigqueryv2_filters(): corrected_start_time = config.start_time - config.max_query_duration corrected_end_time = config.end_time + config.max_query_duration + report = BigQueryV2Report() filter: str = BigQueryUsageExtractor( config, - BigQueryV2Report(), + report, schema_resolver=SchemaResolver(platform="bigquery"), - dataset_urn_builder=lambda x: "", + identifiers=BigQueryIdentifierBuilder(config, report), )._generate_filter(corrected_start_time, corrected_end_time) assert filter == expected_filter From 139c040977327d537a97801a06dfc4a18bb1b8ce Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Wed, 24 Jul 2024 08:13:39 +0530 Subject: [PATCH 03/16] minor changes --- .../src/datahub/ingestion/source/bigquery_v2/common.py | 10 +++++++--- .../ingestion/source/bigquery_v2/queries_extractor.py | 8 +++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py index 5c8c844763c91..58cd1965ab7e6 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py @@ -1,3 +1,4 @@ +from datahub.configuration.common import AllowDenyPattern from datahub.configuration.pattern_utils import is_schema_allowed from datahub.emitter.mce_builder import make_dataset_urn from datahub.ingestion.api.source import SourceReport @@ -16,6 +17,8 @@ BQ_EXTERNAL_TABLE_URL_TEMPLATE = "https://console.cloud.google.com/bigquery?project={project}&ws=!1m5!1m4!4m3!1s{project}!2s{dataset}!3s{table}" BQ_EXTERNAL_DATASET_URL_TEMPLATE = "https://console.cloud.google.com/bigquery?project={project}&ws=!1m4!1m3!3m2!1s{project}!2s{dataset}" +BQ_SYSTEM_TABLES_PATTERN = [r".*\.INFORMATION_SCHEMA\..*", r".*\.__TABLES__.*"] + class BigQueryIdentifierBuilder: platform = "bigquery" @@ -59,7 +62,9 @@ def __init__( self.structured_reporter = structured_reporter def is_allowed(self, table_id: BigqueryTableIdentifier) -> bool: - return ( + return AllowDenyPattern(deny=BQ_SYSTEM_TABLES_PATTERN).allowed( + str(table_id) + ) and ( self.filter_config.project_id_pattern.allowed(table_id.project_id) and is_schema_allowed( self.filter_config.dataset_pattern, @@ -68,5 +73,4 @@ def is_allowed(self, table_id: BigqueryTableIdentifier) -> bool: self.filter_config.match_fully_qualified_names, ) and self.filter_config.table_pattern.allowed(str(table_id)) - # TODO: use view_pattern ? - ) + ) # TODO: use view_pattern ? diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 6064b729c8e13..92a376914d632 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -233,9 +233,11 @@ def fetch_query_log( ) with self.structured_report.report_exc( - f"Error fetching query log from BigQuery Project {project.id}" + f"Error fetching query log from BigQuery Project {project.id} Region {region}" ): - logger.info(f"Fetching query log from BigQuery Project {project.id}") + logger.info( + f"Fetching query log from BigQuery Project {project.id} Region {region}" + ) resp = self.connection.query(query_log_query) for i, row in enumerate(resp): @@ -308,5 +310,5 @@ def _build_enriched_query_log_query( WHERE creation_time >= '{audit_start_time}' AND creation_time <= '{audit_end_time}' AND - error_result is not null + error_result is null """ From ae21deb64f05a92f6db3ae231c16b0ed3b4af699 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Thu, 25 Jul 2024 16:56:31 +0530 Subject: [PATCH 04/16] fix user urn, more reporting, some refractors --- .../ingestion/source/bigquery_v2/bigquery.py | 1 - .../source/bigquery_v2/bigquery_audit.py | 8 --- .../source/bigquery_v2/bigquery_config.py | 31 +++++----- .../source/bigquery_v2/bigquery_schema.py | 13 ++--- .../source/bigquery_v2/bigquery_schema_gen.py | 21 ++----- .../ingestion/source/bigquery_v2/common.py | 29 +++++++++- .../source/bigquery_v2/queries_extractor.py | 14 +++-- .../ingestion/source/bigquery_v2/usage.py | 9 +-- .../sql_parsing/sql_parsing_aggregator.py | 1 + .../tests/unit/test_bigquery_source.py | 9 --- .../tests/unit/test_bigquery_usage.py | 57 +++++++++++++------ 11 files changed, 110 insertions(+), 83 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py index 36022ca8d7305..1ef8157eb2d17 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py @@ -234,7 +234,6 @@ def get_workunit_processors(self) -> List[Optional[MetadataWorkUnitProcessor]]: def get_workunits_internal(self) -> Iterable[MetadataWorkUnit]: projects = get_projects( - self.config, self.bq_schema_extractor.schema_api, self.report, self.filters, diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py index c0afdf65cee5d..319c838d2658a 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_audit.py @@ -7,7 +7,6 @@ from dateutil import parser -from datahub.emitter.mce_builder import make_dataset_urn from datahub.utilities.parsing_util import ( get_first_missing_key, get_first_missing_key_any, @@ -213,13 +212,6 @@ def get_sanitized_table_ref(self) -> "BigQueryTableRef": BigqueryTableIdentifier.from_string_name(sanitized_table) ) - def to_urn(self, env: str) -> str: - return make_dataset_urn( - "bigquery", - f"{self.table_identifier.project_id}.{self.table_identifier.dataset}.{self.table_identifier.table}", - env, - ) - def __str__(self) -> str: return f"projects/{self.table_identifier.project_id}/datasets/{self.table_identifier.dataset}/tables/{self.table_identifier.table}" diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py index 4db76b6c918e2..b8ed6ee75d2ff 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py @@ -64,15 +64,6 @@ class BigQueryBaseConfig(ConfigModel): description="The regex pattern to match sharded tables and group as one table. This is a very low level config parameter, only change if you know what you are doing, ", ) - project_ids: List[str] = Field( - default_factory=list, - description=( - "Ingests specified project_ids. Use this property if you want to specify what projects to ingest or " - "don't want to give project resourcemanager.projects.list to your service account. " - "Overrides `project_id_pattern`." - ), - ) - @validator("sharded_table_pattern") def sharded_table_pattern_is_a_valid_regexp(cls, v): try: @@ -209,6 +200,14 @@ def get_sql_alchemy_url(self) -> str: class BigQueryFilterConfig(SQLFilterConfig): + project_ids: List[str] = Field( + default_factory=list, + description=( + "Ingests specified project_ids. Use this property if you want to specify what projects to ingest or " + "don't want to give project resourcemanager.projects.list to your service account. " + "Overrides `project_id_pattern`." + ), + ) project_id_pattern: AllowDenyPattern = Field( default=AllowDenyPattern.allow_all(), @@ -285,7 +284,12 @@ def backward_compatibility_configs_set(cls, values: Dict) -> Dict: class BigQueryIdentifierConfig( PlatformInstanceConfigMixin, EnvConfigMixin, LowerCaseDatasetUrnConfigMixin ): - pass + include_data_platform_instance: bool = Field( + default=False, + description="Whether to create a DataPlatformInstance aspect, equal to the BigQuery project id." + " If enabled, will cause redundancy in the browse path for BigQuery entities in the UI," + " because the project id is represented as the top-level container.", + ) class BigQueryV2Config( @@ -334,13 +338,6 @@ class BigQueryV2Config( description="Whether to populate BigQuery Console url to Datasets/Tables", ) - include_data_platform_instance: bool = Field( - default=False, - description="Whether to create a DataPlatformInstance aspect, equal to the BigQuery project id." - " If enabled, will cause redundancy in the browse path for BigQuery entities in the UI," - " because the project id is represented as the top-level container.", - ) - include_table_snapshots: Optional[bool] = Field( default=True, description="Whether table snapshots should be ingested." ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py index 3e7bb2794262f..84672134e3c6c 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema.py @@ -15,7 +15,6 @@ from datahub.ingestion.api.source import SourceReport from datahub.ingestion.source.bigquery_v2.bigquery_audit import BigqueryTableIdentifier -from datahub.ingestion.source.bigquery_v2.bigquery_config import BigQueryBaseConfig from datahub.ingestion.source.bigquery_v2.bigquery_helper import parse_labels from datahub.ingestion.source.bigquery_v2.bigquery_report import ( BigQuerySchemaApiPerfReport, @@ -610,22 +609,22 @@ def query_project_list( for project in projects: if filters.filter_config.project_id_pattern.allowed(project.id): yield project - # TODO: suppport reporting dropped projects for queries extractor report - # else: - # report.report_dropped(project.id) + else: + logger.debug( + f"Ignoring project {project.id} as it's not allowed by project_id_pattern" + ) def get_projects( - config: BigQueryBaseConfig, schema_api: BigQuerySchemaApi, report: SourceReport, filters: BigQueryFilter, ) -> List[BigqueryProject]: logger.info("Getting projects") - if config.project_ids: + if filters.filter_config.project_ids: return [ BigqueryProject(id=project_id, name=project_id) - for project_id in config.project_ids + for project_id in filters.filter_config.project_ids ] else: return list( diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py index f0829dddd9fa0..dbb56a842a186 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_schema_gen.py @@ -6,11 +6,7 @@ from google.cloud.bigquery.table import TableListItem from datahub.configuration.pattern_utils import is_schema_allowed, is_tag_allowed -from datahub.emitter.mce_builder import ( - make_data_platform_urn, - make_dataplatform_instance_urn, - make_tag_urn, -) +from datahub.emitter.mce_builder import make_tag_urn from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.mcp_builder import BigQueryDatasetKey, ContainerKey, ProjectIdKey from datahub.ingestion.api.workunit import MetadataWorkUnit @@ -170,7 +166,6 @@ def __init__( self.sql_parser_schema_resolver = sql_parser_schema_resolver self.profiler = profiler self.identifiers = identifiers - self.platform: str = "bigquery" self.classification_handler = ClassificationHandler(self.config, self.report) self.data_reader: Optional[BigQueryDataReader] = None @@ -206,12 +201,8 @@ def get_dataplatform_instance_aspect( self, dataset_urn: str, project_id: str ) -> MetadataWorkUnit: aspect = DataPlatformInstanceClass( - platform=make_data_platform_urn(self.platform), - instance=( - make_dataplatform_instance_urn(self.platform, project_id) - if self.config.include_data_platform_instance - else None - ), + platform=self.identifiers.make_data_platform_urn(), + instance=self.identifiers.make_dataplatform_instance_urn(project_id), ) return MetadataChangeProposalWrapper( entityUrn=dataset_urn, aspect=aspect @@ -221,7 +212,7 @@ def gen_dataset_key(self, db_name: str, schema: str) -> ContainerKey: return BigQueryDatasetKey( project_id=db_name, dataset_id=schema, - platform=self.platform, + platform=self.identifiers.platform, env=self.config.env, backcompat_env_as_instance=True, ) @@ -229,7 +220,7 @@ def gen_dataset_key(self, db_name: str, schema: str) -> ContainerKey: def gen_project_id_key(self, database: str) -> ContainerKey: return ProjectIdKey( project_id=database, - platform=self.platform, + platform=self.identifiers.platform, env=self.config.env, backcompat_env_as_instance=True, ) @@ -955,7 +946,7 @@ def gen_schema_metadata( ) -> MetadataWorkUnit: schema_metadata = SchemaMetadata( schemaName=str(dataset_name), - platform=make_data_platform_urn(self.platform), + platform=self.identifiers.make_data_platform_urn(), version=0, hash="", platformSchema=MySqlDDL(tableSchema=""), diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py index 58cd1965ab7e6..6ada35aef8df5 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py @@ -1,6 +1,13 @@ +from typing import Optional + from datahub.configuration.common import AllowDenyPattern from datahub.configuration.pattern_utils import is_schema_allowed -from datahub.emitter.mce_builder import make_dataset_urn +from datahub.emitter.mce_builder import ( + make_data_platform_urn, + make_dataplatform_instance_urn, + make_dataset_urn, + make_user_urn, +) from datahub.ingestion.api.source import SourceReport from datahub.ingestion.source.bigquery_v2.bigquery_audit import ( BigqueryTableIdentifier, @@ -53,6 +60,19 @@ def gen_dataset_urn_from_raw_ref(self, ref: BigQueryTableRef) -> str: use_raw_name=True, ) + def gen_user_urn(self, user_email: str) -> str: + return make_user_urn(user_email.split("@")[0]) + + def make_data_platform_urn(self) -> str: + return make_data_platform_urn(self.platform) + + def make_dataplatform_instance_urn(self, project_id: str) -> Optional[str]: + return ( + make_dataplatform_instance_urn(self.platform, project_id) + if self.identifier_config.include_data_platform_instance + else None + ) + class BigQueryFilter: def __init__( @@ -65,7 +85,7 @@ def is_allowed(self, table_id: BigqueryTableIdentifier) -> bool: return AllowDenyPattern(deny=BQ_SYSTEM_TABLES_PATTERN).allowed( str(table_id) ) and ( - self.filter_config.project_id_pattern.allowed(table_id.project_id) + self.is_project_allowed(table_id.project_id) and is_schema_allowed( self.filter_config.dataset_pattern, table_id.dataset, @@ -74,3 +94,8 @@ def is_allowed(self, table_id: BigqueryTableIdentifier) -> bool: ) and self.filter_config.table_pattern.allowed(str(table_id)) ) # TODO: use view_pattern ? + + def is_project_allowed(self, project_id: str) -> bool: + if self.filter_config.project_ids: + return project_id in self.filter_config.project_ids + return self.filter_config.project_id_pattern.allowed(project_id) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 92a376914d632..c9dec25d7aba4 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -11,7 +11,6 @@ from datahub.configuration.common import AllowDenyPattern from datahub.configuration.time_window_config import BaseTimeWindowConfig -from datahub.emitter.mce_builder import make_user_urn from datahub.ingestion.api.report import Report from datahub.ingestion.api.source import SourceReport from datahub.ingestion.api.source_helpers import auto_workunit @@ -39,6 +38,7 @@ ) from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedList from datahub.utilities.perf_timer import PerfTimer +from datahub.utilities.stats_collections import TopKDict, int_top_k_dict logger = logging.getLogger(__name__) @@ -104,6 +104,7 @@ class BigQueryQueriesExtractorReport(Report): query_log_fetch_timer: PerfTimer = field(default_factory=PerfTimer) audit_log_load_timer: PerfTimer = field(default_factory=PerfTimer) sql_aggregator: Optional[SqlAggregatorReport] = None + num_queries_by_project: TopKDict[str, int] = field(default_factory=int_top_k_dict) class BigQueryQueriesExtractor: @@ -203,9 +204,10 @@ def get_workunits_internal( with self.report.query_log_fetch_timer: for project in get_projects( - self.config, self.schema_api, self.structured_report, self.filters + self.schema_api, self.structured_report, self.filters ): for entry in self.fetch_query_log(project): + self.report.num_queries_by_project[project.id] += 1 queries.append(entry) with self.report.audit_log_load_timer: @@ -229,7 +231,6 @@ def fetch_query_log( region=region, start_time=self.config.window.start_time, end_time=self.config.window.end_time, - # TODO: filters: deny users based on config ) with self.structured_report.report_exc( @@ -266,7 +267,12 @@ def _parse_audit_log_row( query=row["query"], session_id=row["session_id"], timestamp=row["creation_time"], - user=make_user_urn(row["user_email"]) if row["user_email"] else None, + # TODO: Move user urn generation to BigQueryIdentifierBuilder + user=( + self.identifiers.gen_user_urn(row["user_email"]) + if row["user_email"] + else None + ), default_db=row["project_id"], default_schema=None, ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py index 1135a0956a2a7..876ffab85ba31 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/usage.py @@ -26,7 +26,6 @@ BaseTimeWindowConfig, get_time_bucket, ) -from datahub.emitter.mce_builder import make_user_urn from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.ingestion.api.closeable import Closeable from datahub.ingestion.api.source_helpers import auto_empty_dataset_usage_statistics @@ -715,12 +714,14 @@ def _create_operation_workunit( affected_datasets = [] if event.query_event and event.query_event.referencedTables: for table in event.query_event.referencedTables: - affected_datasets.append(table.to_urn(self.config.env)) + affected_datasets.append( + self.identifiers.gen_dataset_urn_from_raw_ref(table) + ) operation_aspect = OperationClass( timestampMillis=reported_time, lastUpdatedTimestamp=operational_meta.last_updated_timestamp, - actor=make_user_urn(operational_meta.actor_email.split("@")[0]), + actor=self.identifiers.gen_user_urn(operational_meta.actor_email), operationType=operational_meta.statement_type, customOperationType=operational_meta.custom_type, affectedDatasets=affected_datasets, @@ -734,7 +735,7 @@ def _create_operation_workunit( operation_aspect.numAffectedRows = event.query_event.numAffectedRows return MetadataChangeProposalWrapper( - entityUrn=destination_table.to_urn(env=self.config.env), + entityUrn=self.identifiers.gen_dataset_urn_from_raw_ref(destination_table), aspect=operation_aspect, ).as_workunit() diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py index 0774f0efcb46d..09885e435162d 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py @@ -490,6 +490,7 @@ def add( default_schema=item.default_schema, session_id=item.session_id, query_timestamp=item.timestamp, + user=CorpUserUrn.from_string(item.user) if item.user else None, ) else: raise ValueError(f"Cannot add unknown item type: {type(item)}") diff --git a/metadata-ingestion/tests/unit/test_bigquery_source.py b/metadata-ingestion/tests/unit/test_bigquery_source.py index 7edfd8586f1df..b3146bfabff9f 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_source.py +++ b/metadata-ingestion/tests/unit/test_bigquery_source.py @@ -181,7 +181,6 @@ def test_get_projects_with_project_ids(get_bq_client_mock): ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) assert get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -196,7 +195,6 @@ def test_get_projects_with_project_ids(get_bq_client_mock): ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test2")) assert get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -219,7 +217,6 @@ def test_get_projects_with_project_ids_overrides_project_id_pattern( ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) projects = get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -287,7 +284,6 @@ def test_get_projects_with_single_project_id(get_bq_client_mock): config = BigQueryV2Config.parse_obj({"project_id": "test-3"}) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) assert get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -324,7 +320,6 @@ def test_get_projects_by_list(get_bq_client_mock): config = BigQueryV2Config.parse_obj({}) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test1")) assert get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -350,7 +345,6 @@ def test_get_projects_filter_by_pattern(get_bq_client_mock, get_projects_mock): ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) projects = get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -370,7 +364,6 @@ def test_get_projects_list_empty(get_bq_client_mock, get_projects_mock): ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) projects = get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -396,7 +389,6 @@ def test_get_projects_list_failure( caplog.clear() with caplog.at_level(logging.ERROR): projects = get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, @@ -417,7 +409,6 @@ def test_get_projects_list_fully_filtered(get_projects_mock, get_bq_client_mock) ) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test")) projects = get_projects( - source.config, source.bq_schema_extractor.schema_api, source.report, source.filters, diff --git a/metadata-ingestion/tests/unit/test_bigquery_usage.py b/metadata-ingestion/tests/unit/test_bigquery_usage.py index 1c2b08915691f..7ff83bff4a72a 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_usage.py +++ b/metadata-ingestion/tests/unit/test_bigquery_usage.py @@ -162,21 +162,23 @@ def query_view_1_and_table_1(timestamp: datetime = TS_1, actor: str = ACTOR_1) - def make_usage_workunit( - table: Table, dataset_usage_statistics: DatasetUsageStatisticsClass + table: Table, + dataset_usage_statistics: DatasetUsageStatisticsClass, + identifiers: BigQueryIdentifierBuilder, ) -> MetadataWorkUnit: resource = BigQueryTableRef.from_string_name(TABLE_REFS[table.name]) return MetadataChangeProposalWrapper( - entityUrn=resource.to_urn("PROD"), + entityUrn=identifiers.gen_dataset_urn_from_raw_ref(resource), aspectName=dataset_usage_statistics.get_aspect_name(), aspect=dataset_usage_statistics, ).as_workunit() def make_operational_workunit( - resource: str, operation: OperationClass + resource_urn: str, operation: OperationClass ) -> MetadataWorkUnit: return MetadataChangeProposalWrapper( - entityUrn=BigQueryTableRef.from_string_name(resource).to_urn("PROD"), + entityUrn=resource_urn, aspectName=operation.get_aspect_name(), aspect=operation, ).as_workunit() @@ -209,7 +211,10 @@ def usage_extractor(config: BigQueryV2Config) -> BigQueryUsageExtractor: def make_zero_usage_workunit( - table: Table, time: datetime, bucket_duration: BucketDuration = BucketDuration.DAY + table: Table, + time: datetime, + identifiers: BigQueryIdentifierBuilder, + bucket_duration: BucketDuration = BucketDuration.DAY, ) -> MetadataWorkUnit: return make_usage_workunit( table=table, @@ -222,6 +227,7 @@ def make_zero_usage_workunit( userCounts=[], fieldCounts=[], ), + identifiers=identifiers, ) @@ -292,9 +298,10 @@ def test_usage_counts_single_bucket_resource_project( ), ], ), + identifiers=usage_extractor.identifiers, ), - make_zero_usage_workunit(TABLE_2, TS_1), - make_zero_usage_workunit(VIEW_1, TS_1), + make_zero_usage_workunit(TABLE_2, TS_1, usage_extractor.identifiers), + make_zero_usage_workunit(VIEW_1, TS_1, usage_extractor.identifiers), ] compare_workunits(workunits, expected) @@ -375,6 +382,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=VIEW_1, @@ -402,6 +410,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ], fieldCounts=[], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=TABLE_2, @@ -433,6 +442,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), # TS 2 make_usage_workunit( @@ -477,6 +487,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=VIEW_1, @@ -497,6 +508,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ], fieldCounts=[], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=TABLE_2, @@ -530,6 +542,7 @@ def test_usage_counts_multiple_buckets_and_resources_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), ] compare_workunits(workunits, expected) @@ -620,6 +633,7 @@ def test_usage_counts_multiple_buckets_and_resources_no_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=TABLE_2, @@ -662,6 +676,7 @@ def test_usage_counts_multiple_buckets_and_resources_no_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), # TS 2 make_usage_workunit( @@ -711,6 +726,7 @@ def test_usage_counts_multiple_buckets_and_resources_no_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), make_usage_workunit( table=TABLE_2, @@ -762,8 +778,9 @@ def test_usage_counts_multiple_buckets_and_resources_no_view_usage( ), ], ), + identifiers=usage_extractor.identifiers, ), - make_zero_usage_workunit(VIEW_1, TS_1), + make_zero_usage_workunit(VIEW_1, TS_1, usage_extractor.identifiers), # TS_2 not included as only 1 minute of it was ingested ] compare_workunits(workunits, expected) @@ -791,7 +808,7 @@ def test_usage_counts_no_query_event( workunits = usage_extractor._get_workunits_internal([event], [str(ref)]) expected = [ MetadataChangeProposalWrapper( - entityUrn=ref.to_urn("PROD"), + entityUrn=usage_extractor.identifiers.gen_dataset_urn_from_raw_ref(ref), aspect=DatasetUsageStatisticsClass( timestampMillis=int(TS_1.timestamp() * 1000), eventGranularity=TimeWindowSizeClass( @@ -870,6 +887,7 @@ def test_usage_counts_no_columns( ], fieldCounts=[], ), + identifiers=usage_extractor.identifiers, ) ] compare_workunits(workunits, expected) @@ -989,6 +1007,7 @@ def test_usage_counts_no_columns_and_top_n_limit_hit( ], fieldCounts=[], ), + identifiers=usage_extractor.identifiers, ) ] compare_workunits(workunits, expected) @@ -1034,7 +1053,11 @@ def test_operational_stats( workunits = usage_extractor._get_workunits_internal(events, table_refs.values()) expected = [ make_operational_workunit( - table_refs[query.object_modified.name], + usage_extractor.identifiers.gen_dataset_urn_from_raw_ref( + BigQueryTableRef.from_string_name( + table_refs[query.object_modified.name] + ) + ), OperationClass( timestampMillis=int(FROZEN_TIME.timestamp() * 1000), lastUpdatedTimestamp=int(query.timestamp.timestamp() * 1000), @@ -1051,18 +1074,20 @@ def test_operational_stats( ), affectedDatasets=list( dict.fromkeys( # Preserve order - BigQueryTableRef.from_string_name( - table_refs[field.table.name] - ).to_urn("PROD") + usage_extractor.identifiers.gen_dataset_urn_from_raw_ref( + BigQueryTableRef.from_string_name( + table_refs[field.table.name] + ) + ) for field in query.fields_accessed if not field.table.is_view() ) ) + list( dict.fromkeys( # Preserve order - BigQueryTableRef.from_string_name( - table_refs[parent.name] - ).to_urn("PROD") + usage_extractor.identifiers.gen_dataset_urn_from_raw_ref( + BigQueryTableRef.from_string_name(table_refs[parent.name]) + ) for field in query.fields_accessed if field.table.is_view() for parent in field.table.upstreams From 13d5f9ac86fac762090e6aa14b9871e44791288f Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Fri, 26 Jul 2024 14:57:02 +0530 Subject: [PATCH 05/16] more logs, sharded table legacy support --- .../datahub/ingestion/source/bigquery_v2/bigquery.py | 2 -- .../ingestion/source/bigquery_v2/bigquery_config.py | 10 +++++----- .../src/datahub/ingestion/source/bigquery_v2/common.py | 2 ++ .../ingestion/source/bigquery_v2/queries_extractor.py | 7 +++++-- .../ingestion/source/snowflake/snowflake_queries.py | 4 +++- 5 files changed, 15 insertions(+), 10 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py index 1ef8157eb2d17..02d1eb4ec5d6a 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery.py @@ -109,8 +109,6 @@ def __init__(self, ctx: PipelineContext, config: BigQueryV2Config): BigqueryTableIdentifier._BIGQUERY_DEFAULT_SHARDED_TABLE_REGEX = ( self.config.sharded_table_pattern ) - if self.config.enable_legacy_sharded_table_support: - BigqueryTableIdentifier._BQ_SHARDED_TABLE_SUFFIX = "" self.bigquery_data_dictionary = BigQuerySchemaApi( self.report.schema_api_perf, diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py index b8ed6ee75d2ff..fb033ce420c45 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py @@ -291,6 +291,11 @@ class BigQueryIdentifierConfig( " because the project id is represented as the top-level container.", ) + enable_legacy_sharded_table_support: bool = Field( + default=True, + description="Use the legacy sharded table urn suffix added.", + ) + class BigQueryV2Config( BigQueryConnectionConfig, @@ -397,11 +402,6 @@ def have_table_data_read_permission(self) -> bool: description="This flag enables the data lineage extraction from Data Lineage API exposed by Google Data Catalog. NOTE: This extractor can't build views lineage. It's recommended to enable the view's DDL parsing. Read the docs to have more information about: https://cloud.google.com/data-catalog/docs/concepts/about-data-lineage", ) - enable_legacy_sharded_table_support: bool = Field( - default=True, - description="Use the legacy sharded table urn suffix added.", - ) - extract_policy_tags_from_catalog: bool = Field( default=False, description=( diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py index 6ada35aef8df5..5d2358fb8d05e 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/common.py @@ -36,6 +36,8 @@ def __init__( structured_reporter: SourceReport, ) -> None: self.identifier_config = identifier_config + if self.identifier_config.enable_legacy_sharded_table_support: + BigqueryTableIdentifier._BQ_SHARDED_TABLE_SUFFIX = "" self.structured_reporter = structured_reporter def gen_dataset_urn( diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index c9dec25d7aba4..6eeda50a22d23 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -211,7 +211,9 @@ def get_workunits_internal( queries.append(entry) with self.report.audit_log_load_timer: - for query in queries: + for i, query in enumerate(queries): + if i % 1000 == 0: + logger.info(f"Added {i} query log entries to SQL aggregator") self.aggregator.add(query) yield from auto_workunit(self.aggregator.gen_metadata()) @@ -316,5 +318,6 @@ def _build_enriched_query_log_query( WHERE creation_time >= '{audit_start_time}' AND creation_time <= '{audit_end_time}' AND - error_result is null + error_result is null AND + not CONTAINS_SUBSTR(query, '.INFORMATION_SCHEMA.') """ diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py index d5b8f98e40075..7a8df94cc61e6 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py @@ -219,7 +219,9 @@ def get_workunits_internal( queries.append(entry) with self.report.audit_log_load_timer: - for query in queries: + for i, query in enumerate(queries): + if i % 1000 == 0: + logger.info(f"Added {i} query log entries to SQL aggregator") self.aggregator.add(query) yield from auto_workunit(self.aggregator.gen_metadata()) From e96febb274337d44d36edafdac0b98076c5e9674 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Fri, 26 Jul 2024 15:58:16 +0530 Subject: [PATCH 06/16] add list projects timer details --- .../ingestion/source/bigquery_v2/bigquery_queries.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py index 3f253f1e0e161..45136efecbac7 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_queries.py @@ -1,5 +1,5 @@ import logging -from dataclasses import dataclass +from dataclasses import dataclass, field from typing import Iterable, Optional from pydantic import Field @@ -35,6 +35,9 @@ class BigQueryQueriesSourceReport(SourceReport): window: Optional[BaseTimeWindowConfig] = None queries_extractor: Optional[BigQueryQueriesExtractorReport] = None + schema_api_perf: BigQuerySchemaApiPerfReport = field( + default_factory=BigQuerySchemaApiPerfReport + ) class BigQueryQueriesSourceConfig( @@ -58,9 +61,7 @@ def __init__(self, ctx: PipelineContext, config: BigQueryQueriesSourceConfig): self.queries_extractor = BigQueryQueriesExtractor( connection=self.connection, - schema_api=BigQuerySchemaApi( - BigQuerySchemaApiPerfReport(), self.connection - ), + schema_api=BigQuerySchemaApi(self.report.schema_api_perf, self.connection), config=self.config, structured_report=self.report, filters=self.filters, From a7af11eedd678ef26fb7b96328f1b88331eecc0f Mon Sep 17 00:00:00 2001 From: Harshal Sheth Date: Fri, 26 Jul 2024 14:00:19 -0700 Subject: [PATCH 07/16] tweak log statement --- .../datahub/ingestion/source/bigquery_v2/queries_extractor.py | 4 ++-- .../datahub/ingestion/source/snowflake/snowflake_queries.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 6eeda50a22d23..dcd4520aaff6e 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -244,8 +244,8 @@ def fetch_query_log( resp = self.connection.query(query_log_query) for i, row in enumerate(resp): - if i % 1000 == 0: - logger.info(f"Processed {i} query log rows") + if i > 0 and i % 1000 == 0: + logger.info(f"Processed {i} query log rows so far") try: entry = self._parse_audit_log_row(row) diff --git a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py index 7a8df94cc61e6..c4f5cf0ce4116 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py +++ b/metadata-ingestion/src/datahub/ingestion/source/snowflake/snowflake_queries.py @@ -277,8 +277,8 @@ def fetch_query_log( resp = self.connection.query(query_log_query) for i, row in enumerate(resp): - if i % 1000 == 0: - logger.info(f"Processed {i} query log rows") + if i > 0 and i % 1000 == 0: + logger.info(f"Processed {i} query log rows so far") assert isinstance(row, dict) try: From 04257a8d421fb5ee8b0246df3320c8df42cc93d8 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Mon, 29 Jul 2024 11:11:44 +0530 Subject: [PATCH 08/16] safeguard boolean checks otherwise exception is thrown during parsing.e.g. list index out of range --- .../source/bigquery_v2/queries_extractor.py | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index dcd4520aaff6e..8b18d1c5f6ae9 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -172,17 +172,24 @@ def local_temp_path(self) -> pathlib.Path: return path def is_temp_table(self, name: str) -> bool: - return BigqueryTableIdentifier.from_string_name(name).dataset.startswith( - self.config.temp_table_dataset_prefix - ) + try: + return BigqueryTableIdentifier.from_string_name(name).dataset.startswith( + self.config.temp_table_dataset_prefix + ) + except Exception: + logger.warning(f"Error parsing table name {name} ") + return False def is_allowed_table(self, name: str) -> bool: - table_id = BigqueryTableIdentifier.from_string_name(name) - if self.discovered_tables and str(table_id) not in self.discovered_tables: + try: + table_id = BigqueryTableIdentifier.from_string_name(name) + if self.discovered_tables and str(table_id) not in self.discovered_tables: + return False + return self.filters.is_allowed(table_id) + except Exception: + logger.warning(f"Error parsing table name {name} ") return False - return self.filters.is_allowed(table_id) - def get_workunits_internal( self, ) -> Iterable[MetadataWorkUnit]: From 8f8b7f590e1fa40b02546666fe52f2355b277bdb Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Wed, 7 Aug 2024 19:11:16 +0530 Subject: [PATCH 09/16] deduplicate queries, fix empty upstream lineage --- .../source/bigquery_v2/queries_extractor.py | 99 +++++++++++++++---- .../sql_parsing/sql_parsing_aggregator.py | 14 ++- .../unit/sql_parsing/test_sql_aggregator.py | 25 +++++ 3 files changed, 115 insertions(+), 23 deletions(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 8b18d1c5f6ae9..30777e2d675d4 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -4,13 +4,16 @@ import tempfile from dataclasses import dataclass, field from datetime import datetime, timezone -from typing import Iterable, List, Optional, TypedDict, Union +from typing import Dict, Iterable, List, MutableMapping, Optional, TypedDict from google.cloud.bigquery import Client from pydantic import Field from datahub.configuration.common import AllowDenyPattern -from datahub.configuration.time_window_config import BaseTimeWindowConfig +from datahub.configuration.time_window_config import ( + BaseTimeWindowConfig, + get_time_bucket, +) from datahub.ingestion.api.report import Report from datahub.ingestion.api.source import SourceReport from datahub.ingestion.api.source_helpers import auto_workunit @@ -32,13 +35,18 @@ from datahub.sql_parsing.schema_resolver import SchemaResolver from datahub.sql_parsing.sql_parsing_aggregator import ( ObservedQuery, - PreparsedQuery, SqlAggregatorReport, SqlParsingAggregator, ) -from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedList +from datahub.sql_parsing.sqlglot_utils import get_query_fingerprint +from datahub.utilities.file_backed_collections import ( + ConnectionWrapper, + FileBackedDict, + FileBackedList, +) from datahub.utilities.perf_timer import PerfTimer from datahub.utilities.stats_collections import TopKDict, int_top_k_dict +from datahub.utilities.time import datetime_to_ts_millis logger = logging.getLogger(__name__) @@ -73,6 +81,7 @@ class BigQueryJob(TypedDict): total_bytes_processed: int dml_statistics: Optional[DMLJobStatistics] session_id: Optional[str] + query_hash: Optional[str] class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): @@ -98,14 +107,23 @@ class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): include_query_usage_statistics: bool = False include_operations: bool = True + region_qualifiers: List[str] = Field( + default=["region-us", "region-eu"], + description="BigQuery regions to be scanned for bigquery jobs. See [this](https://cloud.google.com/bigquery/docs/information-schema-jobs) for details.", + ) + @dataclass class BigQueryQueriesExtractorReport(Report): query_log_fetch_timer: PerfTimer = field(default_factory=PerfTimer) + audit_log_preprocessing_timer: PerfTimer = field(default_factory=PerfTimer) audit_log_load_timer: PerfTimer = field(default_factory=PerfTimer) sql_aggregator: Optional[SqlAggregatorReport] = None num_queries_by_project: TopKDict[str, int] = field(default_factory=int_top_k_dict) + num_total_queries: int = 0 + num_unique_queries: int = 0 + class BigQueryQueriesExtractor: def __init__( @@ -197,7 +215,7 @@ def get_workunits_internal( audit_log_file = self.local_temp_path / "audit_log.sqlite" use_cached_audit_log = audit_log_file.exists() - queries: FileBackedList[Union[PreparsedQuery, ObservedQuery]] + queries: FileBackedList[ObservedQuery] if use_cached_audit_log: logger.info("Using cached audit log") shared_connection = ConnectionWrapper(audit_log_file) @@ -207,7 +225,7 @@ def get_workunits_internal( shared_connection = ConnectionWrapper(audit_log_file) queries = FileBackedList(shared_connection) - entry: Union[PreparsedQuery, ObservedQuery] + entry: ObservedQuery with self.report.query_log_fetch_timer: for project in get_projects( @@ -216,22 +234,62 @@ def get_workunits_internal( for entry in self.fetch_query_log(project): self.report.num_queries_by_project[project.id] += 1 queries.append(entry) + self.report.num_total_queries = len(queries) + + with self.report.audit_log_preprocessing_timer: + # Preprocessing stage that deduplicates the queries using query hash per usage bucket + queries_deduped: MutableMapping[str, Dict[int, ObservedQuery]] + queries_deduped = self.deduplicate_queries(queries) + self.report.num_unique_queries = len(queries_deduped) with self.report.audit_log_load_timer: - for i, query in enumerate(queries): - if i % 1000 == 0: - logger.info(f"Added {i} query log entries to SQL aggregator") - self.aggregator.add(query) + i = 0 + # Is FileBackedDict OrderedDict ? i.e. keys / values are retrieved in same order as added ? + # Does aggregator expect to see queries in same order as they were executed ? + for query_instances in queries_deduped.values(): + for _, query in query_instances.items(): + if i > 0 and i % 1000 == 0: + logger.info(f"Added {i} query log entries to SQL aggregator") + + self.aggregator.add(query) + i += 1 yield from auto_workunit(self.aggregator.gen_metadata()) - def fetch_query_log( - self, project: BigqueryProject - ) -> Iterable[Union[PreparsedQuery, ObservedQuery]]: + def deduplicate_queries( + self, queries: FileBackedList[ObservedQuery] + ) -> MutableMapping[str, Dict[int, ObservedQuery]]: + queries_deduped: FileBackedDict[Dict[int, ObservedQuery]] = FileBackedDict() + for query in queries: + time_bucket = ( + datetime_to_ts_millis( + get_time_bucket(query.timestamp, self.config.window.bucket_duration) + ) + if query.timestamp + else 0 + ) + query_hash = get_query_fingerprint( + query.query, self.identifiers.platform, fast=True + ) + query.query_hash = query_hash + if query_hash not in queries_deduped: + queries_deduped[query_hash] = {time_bucket: query} + else: + seen_query = queries_deduped[query_hash] + if time_bucket not in seen_query: + seen_query[time_bucket] = query + else: + observed_query = seen_query[time_bucket] + observed_query.usage_multiplier += 1 + observed_query.timestamp = query.timestamp + queries_deduped[query_hash] = seen_query + + return queries_deduped + + def fetch_query_log(self, project: BigqueryProject) -> Iterable[ObservedQuery]: # Multi-regions from https://cloud.google.com/bigquery/docs/locations#supported_locations - regions = ["region-us", "region-eu"] - # TODO: support other regions as required - via a config + regions = self.config.region_qualifiers for region in regions: # Each region needs to be a different query @@ -265,10 +323,7 @@ def fetch_query_log( else: yield entry - def _parse_audit_log_row( - self, row: BigQueryJob - ) -> Union[ObservedQuery, PreparsedQuery]: - + def _parse_audit_log_row(self, row: BigQueryJob) -> ObservedQuery: timestamp: datetime = row["creation_time"] timestamp = timestamp.astimezone(timezone.utc) @@ -284,6 +339,8 @@ def _parse_audit_log_row( ), default_db=row["project_id"], default_schema=None, + # Not using BQ query hash as it's not always present + # query_hash=row["query_hash"], ) return entry @@ -319,7 +376,8 @@ def _build_enriched_query_log_query( total_bytes_billed, total_bytes_processed, dml_statistics, - session_info.session_id as session_id + session_info.session_id as session_id, + query_info.query_hashes.normalized_literals as query_hash FROM `{project_id}`.`{region}`.INFORMATION_SCHEMA.JOBS WHERE @@ -327,4 +385,5 @@ def _build_enriched_query_log_query( creation_time <= '{audit_end_time}' AND error_result is null AND not CONTAINS_SUBSTR(query, '.INFORMATION_SCHEMA.') + ORDER BY creation_time """ diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py index 09885e435162d..54f7b37f6b60c 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py @@ -83,7 +83,10 @@ class LoggedQuery: default_schema: Optional[str] -ObservedQuery = LoggedQuery +@dataclasses.dataclass +class ObservedQuery(LoggedQuery): + query_hash: Optional[str] = None + usage_multiplier: int = 1 @dataclasses.dataclass @@ -489,8 +492,10 @@ def add( default_db=item.default_db, default_schema=item.default_schema, session_id=item.session_id, + usage_multiplier=item.usage_multiplier, query_timestamp=item.timestamp, user=CorpUserUrn.from_string(item.user) if item.user else None, + query_hash=item.query_hash, ) else: raise ValueError(f"Cannot add unknown item type: {type(item)}") @@ -634,6 +639,7 @@ def add_observed_query( usage_multiplier: int = 1, is_known_temp_table: bool = False, require_out_table_schema: bool = False, + query_hash: Optional[str] = None, ) -> None: """Add an observed query to the aggregator. @@ -677,8 +683,7 @@ def add_observed_query( if isinstance(parsed.debug_info.column_error, CooperativeTimeoutError): self.report.num_observed_queries_column_timeout += 1 - query_fingerprint = parsed.query_fingerprint - + query_fingerprint = query_hash or parsed.query_fingerprint self.add_preparsed_query( PreparsedQuery( query_id=query_fingerprint, @@ -1150,6 +1155,9 @@ def _gen_lineage_for_downstream( upstream_aspect.fineGrainedLineages or None ) + if not upstream_aspect.upstreams and not upstream_aspect.fineGrainedLineages: + return + yield MetadataChangeProposalWrapper( entityUrn=downstream_urn, aspect=upstream_aspect, diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py index 5c6abf4c9371d..0655ef5842441 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py @@ -499,3 +499,28 @@ def test_table_rename(pytestconfig: pytest.Config) -> None: outputs=mcps, golden_path=RESOURCE_DIR / "test_table_rename.json", ) + + +@freeze_time(FROZEN_TIME) +def test_create_table_query_mcps(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="bigquery", + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=True, + ) + + aggregator.add_observed_query( + query="create or replace table `dataset.foo` (date_utc timestamp, revenue int);", + default_db="dev", + default_schema="public", + query_timestamp=datetime.now(), + ) + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_create_table_query_mcps.json", + ) From d997bb6df2982222cccc983f93378ef979d0d492 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Thu, 8 Aug 2024 11:39:16 +0530 Subject: [PATCH 10/16] add golden file, refractors --- .../source/bigquery_v2/queries_extractor.py | 178 +++++++++++------- .../test_create_table_query_mcps.json | 22 +++ ...lineage_via_temp_table_disordered_add.json | 79 ++++++++ .../unit/sql_parsing/test_sql_aggregator.py | 33 ++++ 4 files changed, 240 insertions(+), 72 deletions(-) create mode 100644 metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json create mode 100644 metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_lineage_via_temp_table_disordered_add.json diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 30777e2d675d4..34ca77a62d342 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -4,7 +4,7 @@ import tempfile from dataclasses import dataclass, field from datetime import datetime, timezone -from typing import Dict, Iterable, List, MutableMapping, Optional, TypedDict +from typing import Dict, Iterable, List, Optional, TypedDict from google.cloud.bigquery import Client from pydantic import Field @@ -109,7 +109,8 @@ class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): region_qualifiers: List[str] = Field( default=["region-us", "region-eu"], - description="BigQuery regions to be scanned for bigquery jobs. See [this](https://cloud.google.com/bigquery/docs/information-schema-jobs) for details.", + description="BigQuery regions to be scanned for bigquery jobs. " + "See [this](https://cloud.google.com/bigquery/docs/information-schema-jobs#scope_and_syntax) for details.", ) @@ -234,23 +235,22 @@ def get_workunits_internal( for entry in self.fetch_query_log(project): self.report.num_queries_by_project[project.id] += 1 queries.append(entry) - self.report.num_total_queries = len(queries) + self.report.num_total_queries = len(queries) with self.report.audit_log_preprocessing_timer: # Preprocessing stage that deduplicates the queries using query hash per usage bucket - queries_deduped: MutableMapping[str, Dict[int, ObservedQuery]] + queries_deduped: FileBackedDict[Dict[int, ObservedQuery]] queries_deduped = self.deduplicate_queries(queries) self.report.num_unique_queries = len(queries_deduped) with self.report.audit_log_load_timer: i = 0 - # Is FileBackedDict OrderedDict ? i.e. keys / values are retrieved in same order as added ? - # Does aggregator expect to see queries in same order as they were executed ? for query_instances in queries_deduped.values(): for _, query in query_instances.items(): - if i > 0 and i % 1000 == 0: + if i > 0 and i % 10000 == 0: logger.info(f"Added {i} query log entries to SQL aggregator") + logger.info(f"{query.query_hash}, {query.timestamp}") self.aggregator.add(query) i += 1 @@ -258,31 +258,41 @@ def get_workunits_internal( def deduplicate_queries( self, queries: FileBackedList[ObservedQuery] - ) -> MutableMapping[str, Dict[int, ObservedQuery]]: + ) -> FileBackedDict[Dict[int, ObservedQuery]]: + + # This fingerprint based deduplication is done here to reduce performance hit due to + # repetitive sql parsing while adding observed query to aggregator that would otherwise + # parse same query multiple times. In future, aggregator may absorb this deduplication. + # With current implementation, it is possible that "Operation"(e.g. INSERT) is reported + # only once per day, although it may have happened multiple times throughout the day. + queries_deduped: FileBackedDict[Dict[int, ObservedQuery]] = FileBackedDict() - for query in queries: - time_bucket = ( - datetime_to_ts_millis( + + for i, query in enumerate(queries): + if i > 0 and i % 10000 == 0: + logger.info(f"Preprocessing completed for {i} query log entries") + + # query = ObservedQuery(**asdict(query)) + + time_bucket = 0 + if query.timestamp: + time_bucket = datetime_to_ts_millis( get_time_bucket(query.timestamp, self.config.window.bucket_duration) ) - if query.timestamp - else 0 - ) - query_hash = get_query_fingerprint( + + # Not using original BQ query hash as it's not always present + query.query_hash = get_query_fingerprint( query.query, self.identifiers.platform, fast=True ) - query.query_hash = query_hash - if query_hash not in queries_deduped: - queries_deduped[query_hash] = {time_bucket: query} - else: - seen_query = queries_deduped[query_hash] - if time_bucket not in seen_query: - seen_query[time_bucket] = query - else: - observed_query = seen_query[time_bucket] - observed_query.usage_multiplier += 1 - observed_query.timestamp = query.timestamp - queries_deduped[query_hash] = seen_query + + query_instances = queries_deduped.setdefault(query.query_hash, {}) + + observed_query = query_instances.setdefault(time_bucket, query) + + # If the query already exists for this time bucket, update its attributes + if observed_query is not query: + observed_query.usage_multiplier += 1 + observed_query.timestamp = query.timestamp return queries_deduped @@ -292,55 +302,60 @@ def fetch_query_log(self, project: BigqueryProject) -> Iterable[ObservedQuery]: regions = self.config.region_qualifiers for region in regions: - # Each region needs to be a different query - query_log_query = _build_enriched_query_log_query( - project_id=project.id, - region=region, - start_time=self.config.window.start_time, - end_time=self.config.window.end_time, - ) - with self.structured_report.report_exc( - f"Error fetching query log from BigQuery Project {project.id} Region {region}" + f"Error fetching query log from BQ Project {project.id} for {region}" ): - logger.info( - f"Fetching query log from BigQuery Project {project.id} Region {region}" + yield from self.fetch_region_query_log(project, region) + + def fetch_region_query_log( + self, project: BigqueryProject, region: str + ) -> Iterable[ObservedQuery]: + + # Each region needs to be a different query + query_log_query = _build_enriched_query_log_query( + project_id=project.id, + region=region, + start_time=self.config.window.start_time, + end_time=self.config.window.end_time, + ) + + logger.info(f"Fetching query log from BQ Project {project.id} for {region}") + resp = self.connection.query(query_log_query) + + for i, row in enumerate(resp): + if i > 0 and i % 1000 == 0: + logger.info(f"Processed {i} query log rows so far") + try: + entry = self._parse_audit_log_row(row) + except Exception as e: + self.structured_report.warning( + "Error parsing query log row", + context=f"{row}", + exc=e, ) - resp = self.connection.query(query_log_query) - - for i, row in enumerate(resp): - if i > 0 and i % 1000 == 0: - logger.info(f"Processed {i} query log rows so far") - - try: - entry = self._parse_audit_log_row(row) - except Exception as e: - self.structured_report.warning( - "Error parsing query log row", - context=f"{row}", - exc=e, - ) - else: - yield entry + else: + yield entry def _parse_audit_log_row(self, row: BigQueryJob) -> ObservedQuery: timestamp: datetime = row["creation_time"] timestamp = timestamp.astimezone(timezone.utc) + # https://cloud.google.com/bigquery/docs/multi-statement-queries + # Also _ at start considers this as temp dataset as per `temp_table_dataset_prefix` config + TEMP_TABLE_QUALIFIER = "_SESSION" + entry = ObservedQuery( query=row["query"], session_id=row["session_id"], timestamp=row["creation_time"], - # TODO: Move user urn generation to BigQueryIdentifierBuilder user=( self.identifiers.gen_user_urn(row["user_email"]) if row["user_email"] else None ), default_db=row["project_id"], - default_schema=None, - # Not using BQ query hash as it's not always present - # query_hash=row["query_hash"], + default_schema=TEMP_TABLE_QUALIFIER, + query_hash=row["query_hash"], ) return entry @@ -356,26 +371,44 @@ def _build_enriched_query_log_query( audit_start_time = start_time.strftime(BQ_DATETIME_FORMAT) audit_end_time = end_time.strftime(BQ_DATETIME_FORMAT) - # NOTE the use of creation_time as timestamp here - # as JOBS table is partitioned by creation_time. - # Using this column filter significantly reduces processed bytes. + # List of all statement types + # https://cloud.google.com/bigquery/docs/reference/auditlogs/rest/Shared.Types/BigQueryAuditMetadata.QueryStatementType + unsupported_statement_types = ",".join( + [ + f"'{statement_type}'" + for statement_type in [ + # procedure + "CREATE_PROCEDURE", + "DROP_PROCEDURE", + "CALL", + "SCRIPT", # individual statements in executed procedure are present as separate jobs + # schema + "CREATE_SCHEMA", + "DROP_SCHEMA", + # function + "CREATE_FUNCTION", + "CREATE_TABLE_FUNCTION", + "DROP_FUNCTION", + # policies + "CREATE_ROW_ACCESS_POLICY", + "DROP_ROW_ACCESS_POLICY", + ] + ] + ) + + # NOTE the use of partition column creation_time as timestamp here. + # Currently, only required columns are fetched. There are more columns such as + # total_slot_ms, statement_type, job_type, destination_table, referenced_tables, + # total_bytes_billed, dml_statistics(inserted_row_count, etc) that may be fetched + # as required in future. Refer below link for list of all columns + # https://cloud.google.com/bigquery/docs/information-schema-jobs#schema return f""" SELECT job_id, project_id, creation_time, - start_time, - end_time, - total_slot_ms, user_email, - statement_type, - job_type, query, - destination_table, - referenced_tables, - total_bytes_billed, - total_bytes_processed, - dml_statistics, session_info.session_id as session_id, query_info.query_hashes.normalized_literals as query_hash FROM @@ -384,6 +417,7 @@ def _build_enriched_query_log_query( creation_time >= '{audit_start_time}' AND creation_time <= '{audit_end_time}' AND error_result is null AND - not CONTAINS_SUBSTR(query, '.INFORMATION_SCHEMA.') + not CONTAINS_SUBSTR(query, '.INFORMATION_SCHEMA.') AND + statement_type not in ({unsupported_statement_types}) ORDER BY creation_time """ diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json new file mode 100644 index 0000000000000..6a5cf51dd7522 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json @@ -0,0 +1,22 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,dev.dataset.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1707182625000, + "partitionSpec": { + "type": "FULL_TABLE", + "partition": "FULL_TABLE_SNAPSHOT" + }, + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:f2e61c641cf14eae74147b6280ae40648516c4b7b58cfca6c4f7fb14ab255ce2" + }, + "lastUpdatedTimestamp": 1707182625000 + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_lineage_via_temp_table_disordered_add.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_lineage_via_temp_table_disordered_add.json new file mode 100644 index 0000000000000..51a732e7a2494 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_lineage_via_temp_table_disordered_add.json @@ -0,0 +1,79 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.derived_from_foo,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:composite_39f4adf89c8ad4d6d307b628c82d8260e1c5cd7eb6fb3a8cbb437421f970c16f" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_39f4adf89c8ad4d6d307b628c82d8260e1c5cd7eb6fb3a8cbb437421f970c16f", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "CREATE TEMPORARY TABLE foo AS\nSELECT\n a,\n b + c AS c\nFROM bar;\n\nCREATE TABLE derived_from_foo AS\nSELECT\n *\nFROM foo", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 0, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_39f4adf89c8ad4d6d307b628c82d8260e1c5cd7eb6fb3a8cbb437421f970c16f", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.bar,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.derived_from_foo,PROD)" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_39f4adf89c8ad4d6d307b628c82d8260e1c5cd7eb6fb3a8cbb437421f970c16f", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:redshift" + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py index 0655ef5842441..823de771fba2e 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py @@ -524,3 +524,36 @@ def test_create_table_query_mcps(pytestconfig: pytest.Config) -> None: outputs=mcps, golden_path=RESOURCE_DIR / "test_create_table_query_mcps.json", ) + + +@freeze_time(FROZEN_TIME) +def test_lineage_via_temp_table_disordered_add(pytestconfig: pytest.Config) -> None: + aggregator = SqlParsingAggregator( + platform="redshift", + generate_lineage=True, + generate_usage_statistics=False, + generate_operations=False, + ) + + aggregator._schema_resolver.add_raw_schema_info( + DatasetUrn("redshift", "dev.public.bar").urn(), + {"a": "int", "b": "int", "c": "int"}, + ) + aggregator.add_observed_query( + query="create table derived_from_foo as select * from foo", + default_db="dev", + default_schema="public", + ) + aggregator.add_observed_query( + query="create temp table foo as select a, b+c as c from bar", + default_db="dev", + default_schema="public", + ) + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_lineage_via_temp_table_disordered_add.json", + ) From 4153ac545bfe3df3ea2d973c424cbd9090430346 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Tue, 13 Aug 2024 18:20:31 +0530 Subject: [PATCH 11/16] review comments + queries for operation without lineage --- .../docs/sources/bigquery/bigquery_recipe.yml | 3 +- .../src/datahub/emitter/mce_builder.py | 16 +++++ .../source/bigquery_v2/bigquery_config.py | 11 +++- .../source/bigquery_v2/queries_extractor.py | 52 ++++++++-------- .../sql_parsing/sql_parsing_aggregator.py | 25 +++++--- .../test_create_table_query_mcps.json | 59 ++++++++++++++++++- .../unit/sql_parsing/test_sql_aggregator.py | 4 -- .../tests/unit/test_bigquery_source.py | 26 ++++++++ 8 files changed, 151 insertions(+), 45 deletions(-) diff --git a/metadata-ingestion/docs/sources/bigquery/bigquery_recipe.yml b/metadata-ingestion/docs/sources/bigquery/bigquery_recipe.yml index c91aef929889e..2efa1e5513caa 100644 --- a/metadata-ingestion/docs/sources/bigquery/bigquery_recipe.yml +++ b/metadata-ingestion/docs/sources/bigquery/bigquery_recipe.yml @@ -1,8 +1,7 @@ source: type: bigquery config: - # `schema_pattern` for BQ Datasets - schema_pattern: + dataset_pattern: allow: - finance_bq_dataset table_pattern: diff --git a/metadata-ingestion/src/datahub/emitter/mce_builder.py b/metadata-ingestion/src/datahub/emitter/mce_builder.py index a0694983660e4..02a0049bf8964 100644 --- a/metadata-ingestion/src/datahub/emitter/mce_builder.py +++ b/metadata-ingestion/src/datahub/emitter/mce_builder.py @@ -26,6 +26,7 @@ from avrogen.dict_wrapper import DictWrapper from datahub.configuration.source_common import DEFAULT_ENV as DEFAULT_ENV_CONFIGURATION +from datahub.metadata._urns.urn_defs import CorpGroupUrn, CorpUserUrn from datahub.metadata.schema_classes import ( AssertionKeyClass, AuditStampClass, @@ -224,6 +225,21 @@ def make_user_urn(username: str) -> str: ) +def make_actor_urn(actor: str) -> Union[CorpUserUrn, CorpGroupUrn]: + """ + Makes a user urn if the input is not a user or group urn already + """ + return ( + CorpUserUrn(actor) + if not actor.startswith(("urn:li:corpuser:", "urn:li:corpGroup:")) + else ( + CorpUserUrn.create_from_string(actor) + if actor.startswith("urn:li:corpuser:") + else CorpGroupUrn.create_from_string(actor) + ) + ) + + def make_group_urn(groupname: str) -> str: """ Makes a group urn if the input is not a user or group urn already diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py index fb033ce420c45..3a18ee6edb894 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/bigquery_config.py @@ -70,8 +70,8 @@ def sharded_table_pattern_is_a_valid_regexp(cls, v): re.compile(v) except Exception as e: raise ValueError( - f"sharded_table_pattern configuration pattern is invalid. The exception was: {e}" - ) + "sharded_table_pattern configuration pattern is invalid." + ) from e return v @root_validator(pre=True, skip_on_failure=True) @@ -229,6 +229,12 @@ class BigQueryFilterConfig(SQLFilterConfig): description="Regex patterns for table snapshots to filter in ingestion. Specify regex to match the entire snapshot name in database.schema.snapshot format. e.g. to match all snapshots starting with customer in Customer database and public schema, use the regex 'Customer.public.customer.*'", ) + # NOTE: `schema_pattern` is added here only to hide it from docs. + schema_pattern: AllowDenyPattern = Field( + default=AllowDenyPattern.allow_all(), + hidden_from_docs=True, + ) + @root_validator(pre=False, skip_on_failure=True) def backward_compatibility_configs_set(cls, values: Dict) -> Dict: dataset_pattern: Optional[AllowDenyPattern] = values.get("dataset_pattern") @@ -301,6 +307,7 @@ class BigQueryV2Config( BigQueryConnectionConfig, BigQueryBaseConfig, BigQueryFilterConfig, + # BigQueryFilterConfig must come before (higher precedence) the SQLCommon config, so that the documentation overrides are applied. BigQueryIdentifierConfig, SQLCommonConfig, StatefulUsageConfigMixin, diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 34ca77a62d342..d7adc0b161bf8 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -149,7 +149,7 @@ def __init__( # self.filters = filters self.discovered_tables = discovered_tables - self._structured_report = structured_report + self.structured_report = structured_report self.aggregator = SqlParsingAggregator( platform=self.identifiers.platform, @@ -175,10 +175,6 @@ def __init__( ) self.report.sql_aggregator = self.aggregator.report - @property - def structured_report(self) -> SourceReport: - return self._structured_report - @functools.cached_property def local_temp_path(self) -> pathlib.Path: if self.config.local_temp_path: @@ -340,7 +336,10 @@ def _parse_audit_log_row(self, row: BigQueryJob) -> ObservedQuery: timestamp: datetime = row["creation_time"] timestamp = timestamp.astimezone(timezone.utc) - # https://cloud.google.com/bigquery/docs/multi-statement-queries + # Usually bigquery identifiers are always referred as . and only + # temporary tables are referred as
alone without project or dataset name. + # Note that temporary tables can also be referenced using _SESSION.
+ # More details here - https://cloud.google.com/bigquery/docs/multi-statement-queries # Also _ at start considers this as temp dataset as per `temp_table_dataset_prefix` config TEMP_TABLE_QUALIFIER = "_SESSION" @@ -373,27 +372,26 @@ def _build_enriched_query_log_query( # List of all statement types # https://cloud.google.com/bigquery/docs/reference/auditlogs/rest/Shared.Types/BigQueryAuditMetadata.QueryStatementType + UNSUPPORTED_STATEMENT_TYPES = [ + # procedure + "CREATE_PROCEDURE", + "DROP_PROCEDURE", + "CALL", + "SCRIPT", # individual statements in executed procedure are present as separate jobs + # schema + "CREATE_SCHEMA", + "DROP_SCHEMA", + # function + "CREATE_FUNCTION", + "CREATE_TABLE_FUNCTION", + "DROP_FUNCTION", + # policies + "CREATE_ROW_ACCESS_POLICY", + "DROP_ROW_ACCESS_POLICY", + ] + unsupported_statement_types = ",".join( - [ - f"'{statement_type}'" - for statement_type in [ - # procedure - "CREATE_PROCEDURE", - "DROP_PROCEDURE", - "CALL", - "SCRIPT", # individual statements in executed procedure are present as separate jobs - # schema - "CREATE_SCHEMA", - "DROP_SCHEMA", - # function - "CREATE_FUNCTION", - "CREATE_TABLE_FUNCTION", - "DROP_FUNCTION", - # policies - "CREATE_ROW_ACCESS_POLICY", - "DROP_ROW_ACCESS_POLICY", - ] - ] + [f"'{statement_type}'" for statement_type in UNSUPPORTED_STATEMENT_TYPES] ) # NOTE the use of partition column creation_time as timestamp here. @@ -402,7 +400,7 @@ def _build_enriched_query_log_query( # total_bytes_billed, dml_statistics(inserted_row_count, etc) that may be fetched # as required in future. Refer below link for list of all columns # https://cloud.google.com/bigquery/docs/information-schema-jobs#schema - return f""" + return f"""\ SELECT job_id, project_id, diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py index 54f7b37f6b60c..90195929bb658 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py @@ -16,7 +16,7 @@ import datahub.emitter.mce_builder as builder import datahub.metadata.schema_classes as models from datahub.configuration.time_window_config import get_time_bucket -from datahub.emitter.mce_builder import get_sys_time, make_ts_millis +from datahub.emitter.mce_builder import get_sys_time, make_actor_urn, make_ts_millis from datahub.emitter.mcp import MetadataChangeProposalWrapper from datahub.emitter.sql_parsing_builder import compute_upstream_fields from datahub.ingestion.api.closeable import Closeable @@ -25,6 +25,7 @@ from datahub.ingestion.graph.client import DataHubGraph from datahub.ingestion.source.usage.usage_common import BaseUsageConfig, UsageAggregator from datahub.metadata.urns import ( + CorpGroupUrn, CorpUserUrn, DataPlatformUrn, DatasetUrn, @@ -107,7 +108,7 @@ class QueryMetadata: query_type: QueryType lineage_type: str # from models.DatasetLineageTypeClass latest_timestamp: Optional[datetime] - actor: Optional[CorpUserUrn] + actor: Optional[Union[CorpUserUrn, CorpGroupUrn]] upstreams: List[UrnStr] # this is direct upstreams, which may be temp tables column_lineage: List[ColumnLineageInfo] @@ -166,7 +167,7 @@ class PreparsedQuery: confidence_score: float = 1.0 query_count: int = 1 - user: Optional[CorpUserUrn] = None + user: Optional[Union[CorpUserUrn, CorpGroupUrn]] = None timestamp: Optional[datetime] = None session_id: str = _MISSING_SESSION_ID query_type: QueryType = QueryType.UNKNOWN @@ -494,7 +495,7 @@ def add( session_id=item.session_id, usage_multiplier=item.usage_multiplier, query_timestamp=item.timestamp, - user=CorpUserUrn.from_string(item.user) if item.user else None, + user=make_actor_urn(item.user) if item.user else None, query_hash=item.query_hash, ) else: @@ -632,7 +633,7 @@ def add_observed_query( default_db: Optional[str] = None, default_schema: Optional[str] = None, query_timestamp: Optional[datetime] = None, - user: Optional[CorpUserUrn] = None, + user: Optional[Union[CorpUserUrn, CorpGroupUrn]] = None, session_id: Optional[ str ] = None, # can only see temp tables with the same session @@ -929,7 +930,7 @@ def _run_sql_parser( schema_resolver: SchemaResolverInterface, session_id: str = _MISSING_SESSION_ID, timestamp: Optional[datetime] = None, - user: Optional[CorpUserUrn] = None, + user: Optional[Union[CorpUserUrn, CorpGroupUrn]] = None, ) -> SqlParsingResult: with self.report.sql_parsing_timer: parsed = sqlglot_lineage( @@ -1015,7 +1016,7 @@ def gen_metadata(self) -> Iterable[MetadataChangeProposalWrapper]: yield from self._gen_lineage_mcps(queries_generated) yield from self._gen_remaining_queries(queries_generated) yield from self._gen_usage_statistics_mcps() - yield from self._gen_operation_mcps() + yield from self._gen_operation_mcps(queries_generated) def _gen_lineage_mcps( self, queries_generated: Set[QueryId] @@ -1455,7 +1456,9 @@ def _gen_usage_statistics_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: # TODO: We should change the usage aggregator to return MCPWs directly. yield cast(MetadataChangeProposalWrapper, wu.metadata) - def _gen_operation_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: + def _gen_operation_mcps( + self, queries_generated: Set[QueryId] + ) -> Iterable[MetadataChangeProposalWrapper]: if not self.generate_operations: return @@ -1463,6 +1466,12 @@ def _gen_operation_mcps(self) -> Iterable[MetadataChangeProposalWrapper]: for query_id in query_ids: yield from self._gen_operation_for_downstream(downstream_urn, query_id) + # Avoid generating the same query twice. + if query_id in queries_generated: + continue + queries_generated.add(query_id) + yield from self._gen_query(self._query_map[query_id], downstream_urn) + def _gen_operation_for_downstream( self, downstream_urn: UrnStr, query_id: QueryId ) -> Iterable[MetadataChangeProposalWrapper]: diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json index 6a5cf51dd7522..ddb6d931ac9a5 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_create_table_query_mcps.json @@ -8,8 +8,8 @@ "json": { "timestampMillis": 1707182625000, "partitionSpec": { - "type": "FULL_TABLE", - "partition": "FULL_TABLE_SNAPSHOT" + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" }, "operationType": "CREATE", "customProperties": { @@ -18,5 +18,60 @@ "lastUpdatedTimestamp": 1707182625000 } } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f2e61c641cf14eae74147b6280ae40648516c4b7b58cfca6c4f7fb14ab255ce2", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "CREATE OR REPLACE TABLE `dataset.foo` (\n date_utc TIMESTAMP,\n revenue INT64\n)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + }, + "lastModified": { + "time": 1707182625000, + "actor": "urn:li:corpuser:_ingestion" + } + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f2e61c641cf14eae74147b6280ae40648516c4b7b58cfca6c4f7fb14ab255ce2", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,dev.dataset.foo,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,dev.dataset.foo,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,dev.dataset.foo,PROD),revenue)" + } + ] + } + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f2e61c641cf14eae74147b6280ae40648516c4b7b58cfca6c4f7fb14ab255ce2", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + } } ] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py index 823de771fba2e..2b8bc68ee6dc5 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py @@ -535,10 +535,6 @@ def test_lineage_via_temp_table_disordered_add(pytestconfig: pytest.Config) -> N generate_operations=False, ) - aggregator._schema_resolver.add_raw_schema_info( - DatasetUrn("redshift", "dev.public.bar").urn(), - {"a": "int", "b": "int", "c": "int"}, - ) aggregator.add_observed_query( query="create table derived_from_foo as select * from foo", default_db="dev", diff --git a/metadata-ingestion/tests/unit/test_bigquery_source.py b/metadata-ingestion/tests/unit/test_bigquery_source.py index b3146bfabff9f..683712a1bfbbf 100644 --- a/metadata-ingestion/tests/unit/test_bigquery_source.py +++ b/metadata-ingestion/tests/unit/test_bigquery_source.py @@ -1212,3 +1212,29 @@ def get_datasets_for_project_id_side_effect( config = BigQueryV2Config.parse_obj({**base_config, "exclude_empty_projects": True}) source = BigqueryV2Source(config=config, ctx=PipelineContext(run_id="test-2")) assert len({wu.metadata.entityUrn for wu in source.get_workunits()}) == 1 # type: ignore + + +def test_bigquery_config_deprecated_schema_pattern(): + base_config = { + "include_usage_statistics": False, + "include_table_lineage": False, + } + + config = BigQueryV2Config.parse_obj(base_config) + assert config.dataset_pattern == AllowDenyPattern(allow=[".*"]) # default + + config_with_schema_pattern = { + **base_config, + "schema_pattern": AllowDenyPattern(deny=[".*"]), + } + config = BigQueryV2Config.parse_obj(config_with_schema_pattern) + assert config.dataset_pattern == AllowDenyPattern(deny=[".*"]) # schema_pattern + + config_with_dataset_pattern = { + **base_config, + "dataset_pattern": AllowDenyPattern(deny=["temp.*"]), + } + config = BigQueryV2Config.parse_obj(config_with_dataset_pattern) + assert config.dataset_pattern == AllowDenyPattern( + deny=["temp.*"] + ) # dataset_pattern From a6e455fd464402a8b238faf0b74082ddfb3f4a63 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Wed, 14 Aug 2024 19:40:16 +0530 Subject: [PATCH 12/16] fix import --- metadata-ingestion/src/datahub/emitter/mce_builder.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/metadata-ingestion/src/datahub/emitter/mce_builder.py b/metadata-ingestion/src/datahub/emitter/mce_builder.py index 02a0049bf8964..8b41366bd60dc 100644 --- a/metadata-ingestion/src/datahub/emitter/mce_builder.py +++ b/metadata-ingestion/src/datahub/emitter/mce_builder.py @@ -26,7 +26,6 @@ from avrogen.dict_wrapper import DictWrapper from datahub.configuration.source_common import DEFAULT_ENV as DEFAULT_ENV_CONFIGURATION -from datahub.metadata._urns.urn_defs import CorpGroupUrn, CorpUserUrn from datahub.metadata.schema_classes import ( AssertionKeyClass, AuditStampClass, @@ -51,6 +50,7 @@ UpstreamLineageClass, _Aspect as AspectAbstract, ) +from datahub.metadata.urns import CorpGroupUrn, CorpUserUrn from datahub.utilities.urn_encoder import UrnEncoder from datahub.utilities.urns.data_flow_urn import DataFlowUrn from datahub.utilities.urns.dataset_urn import DatasetUrn From ff2cb2009eff6db5b307018f50182293649f76bc Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Mon, 19 Aug 2024 17:00:01 +0530 Subject: [PATCH 13/16] add queries test using cached audit log --- .../integration/bigquery_v2/audit_log.sqlite | Bin 0 -> 139264 bytes .../bigquery_queries_mcps_golden.json | 5522 +++++++++++++++++ .../bigquery_v2/test_bigquery_queries.py | 42 + 3 files changed, 5564 insertions(+) create mode 100644 metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite create mode 100644 metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json create mode 100644 metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py diff --git a/metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite b/metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite new file mode 100644 index 0000000000000000000000000000000000000000..e37437ccd73717e15b6bdb0b8a5c7b2a18d469d6 GIT binary patch literal 139264 zcmeHw34B~vdGFk-ZDur!9g~eD&W%Z8$+4BYFNvL4w&mEdEjyCD#dgMbwIKW=GHFZJj-PnJry=SI&6V5q)6HX4Y-pzHY{kYMy9cPB&kCI%ER8{0{I;Z9Jz16wM>%h*_8iYVrhO=2GeDEA^)4ang3<}w*2mV zSNDZYx%F`Z_a--zbgN}{9oihm493QkMnQOzajtT{Db*N@?-h?^2Pjc{s8%` z_!eJ*_zJ{VAie_e6^O4udS2>LPMU{JSpQ-G}{Sy42c5ANOglC#6*Qb+TNc?^B(}~n;lb_CiG5P)M8?tWp zzHBL5%pT0{&UPoiks8f*Wmji~?27DF*-NtdY;*G2%zx)Ullf)lh0OOe-^e_h`E2H6 znGa;%o%xge$1?vT^QO$P%&Rj;GJ_c_voF(=*_{7iW^G2vbY!l{T%K8yNoT0m7h8Xt zf2#FIt>13_YU?wtpKSfh)~8$F+4^|vTT+Kx$6G6{_qN{E+TU7e-PO9Sbwlfo`FFJP zt?jK$>&30P)~57t)4xdnYx;Zn$J5WHzm)z=`lIRhr~fSdzte9^KbC$Z|NH6t(_`s@ zw3)s=eOr1{`sTEpUX}mP{G;is)0d?$NTiYxodMva~I{moy(;fa?RP_Wq+0ZN%Ft4Kgxb5`}O?SvtLa9BKw)_$Fd*H zK9zhS`;P46+25Zg!}wu*1>!3ZUxD}v#8=?_z5>k+d77i%k$NoAm~2==bDT$ke}kBS-z1jIvDqvo~_o_sJ;9)VIBn zqn;z9C{thYMvnR-83md8yf<>x-;j})slWC{j`}Myax(P+Z{(<_$#1Pt?`xT>PJbfPMSmjE{r*Ix5BU>;cKiuX+x~>3Eq}t&rZ-VF+VCeTt@{&& zKIl(my5LVF`VN00(g*yBK=1b_JiX7KaP;l|gr)a-6Gfx<_!E`h?N1bXmp_r|o&H3k z`}~PW_xclo?(ru)eVaew=pFuqrMG(%S);rCiArztCknmQpUCtUe`w%G zlRx3KXQRsF4M5a6aiA1mUCn9~bKN0Ag{0UF5@h2R; z+MlrWjow7m=o|crN^AZ^p;dn((~3WlXxX2LwB%0&TJ$G8E%+0T=KTpvbKXSIXx5*o z^!5Hkp*#GEOt11M61~!&i1c;-M4(sr6P|AOCmh}8Pgr`nH{mtn!PwD&69Z3f=6D65Z&HBHiGP0!@1(PgCB=QvZ*Psz&|J8&&GRy-}fl>x~li8*db; z|MW(IdeIws>et@LQvX3lMWcS{jVkpEZ&awCd!t1C%o|1Or`{+~Kk-JM`ZsT6sTas7 zYt+Abqe}gYH!9Tg-Y8K&@U(6AH0rzFs8ave8x`t1-Y8Mu z@L17`YSh=gQKi1-jSBTuZQUy_m2s1JFgO8td5D%1zPQKH`OjUx4byiuV3+#7l7 zecs4Y?_d8a zxO99xyWJSIOGj+$wlTYO&+#`OZ-4OktB$vgH{Inu9+yY$0oyDy1P$Soh6fqlEZsA( z3V4b`-p39OAGQl+d$e4zjE=EU{gD6Y_;`BM9vv+X4;6~m@$T_dxj2Xy>4PK3$CLQ| zZ1T8z`sU;7(=E@pT$s%9EnC>*{i*RJdC9%QLpFXXJoG-nvIojK9&bsIe~%pRIsOp- zKaBt5jbr#bj*qvEm4dUi9v-wy1nf>~x zU9gYXLuF?7rmmhYhNu`coY~yVbno1{b?$24NVW7P=C1bF2B~oA`3rpzpHmze zC}Q1hp|Ian_Ey>&^N)z(?MAPx!BkBXV9n3YNSHP6^aZOA4v#^~s~L9khxG`z;eS0H zap5Ks4Zl3sH;wrBi?2X@1?KMxxXUZc8}r9@@Hm7orv1;IU-ROXP{j8*5iou7SW- zvk=%DLJ`>734wL)x~0uGr_RvEu0ueH{ECRI$v|s5n;A|tH~f1d^`6{SnFm^a+k6Ec z%;}bVVoqZelgseW{J=E*-iJKqz3OMC^+`W0~7D%UqIKDL7+TXJk}1;huN zsMKelkIk)pl@5ejR1K=~Oem-K{Wa$EG~27@XJ=%F)x6WU$nF}`hswqBJ@XW7X`&kc z;B(PXkifIQvZ}I*SkvnHU`;nAa%)E-F0hAgs;p?tzwUks0(&Y9PkGPfSmiZ~XJSbX z=*Ybc4qHX12v?ZoOln>=eQ~bhJ4kv6o^nASEsXZ-C5ya`RL7pC>%2@iOrGV1Wi{VJ z-1pAz4b046hq-AD!znx`%DgPHEGwVx`&Rq!lSGZ3`M#~2w{7lYq{+{!%Ykos_jA$E zzwx!aDT48t}%HkS5#IK2T_4aFE|IWDUtmR6AhUM9}@d2*AW{)j%|R6eDY)xxsj&7b1>rK z&0;ks6Hg{`cVzfF9m75wDKfIqOXWxPk{k0^r$0BlA1aBNm zD{COJ)jUbAFSxC}C6Rd~8S%AW*(KcWSJ9@ zEme8WCuO@3Xemq#F2>XlVRw{o{Q!u!{0F%eOXqE zhd-m-zO3Qk638(m2sLVc+Q-+l_s!?=MdhB02pDL5ttqmKtV20j+14DWqoFyL&8w26 zNUEXBiotP`ggX|Hd`IC;2b2$*qjQdKT8m0I2>O`ITCJuA(V8&>t>`gV8WA?q159Lw~PA976BR=kj9y1F;CGbn2HjEFK!RGrc~L%>Ok_SE^`I6a0_#ac zK!`JGK9|cRKBI+7a!*EmJ#v+G#9-n{ChdGMm|Ccoe~$W+gh|v%Od_77v|^Jez$9v+ zng@N1nP^BlM8l}Ew6fKU6%o&8!NrPFiQESw zAI$HlZ1G}6=S6WiBvyDj8mlt!B*&q7D>*a;;%uA`(S#(Gw4xqOKnd1nVqOcec|F6t zhJ4o?jQFBN1Yi@100=Q8MhX}yT_swGj0Bzmh^guKeFw6+R?Uh@+ z90b9w#i+z8RAQ29di#!O=trQ{;Nk!2D^K|Uf7g6n;}$%OZ}Al%R{&1^PS2^2vmLWI z^=YcP9GOsYcmR<7l|Ew2io^kVCKUW%Wa;A4R73A$5kD#rwCN>58{(tobMK%9zL~!4;`v1wq zTNAnN%-hl*Zh5ik3m*RetuM9vivoO$m@+b{JXr7r_?blR$5D?Y!96gEdmt}J{68f0 z-Kb-IU~C%1*pvyZCN?%}jLlX$bw@*N@LGfS|pm18Q@HJz??Ls^99ZXq5a_5Xe{brp>wY4U)&#aBO#C-n0 z<%#T?)=QK7o8Q^^dGP-&f2qCybME`$-0mmNtwb^_3oiEsMgF6xBkjOGvk%t5QHf+y zUI-=oykOa9(hUvo>5uqEhEn<4J!_Bz3}b7s${I{lIX3c!1y|YYS%W0Fcs_vJ(^Tfq z6A@pMV6)qU7(+=S@y3|VZlZCX2YdF}>`22oZVrXdCwbpIMi-!ZeGxHhLu9i9UFU8D zQ`br)m>Sb{PNQFej?>p9b}uE_P%-D=F*q` zEE*?li0BUA|9^cV`^DC;roP$o{-$>|{2ulCbI$)hAATXXp;E$$)2xt)=g&j&U`?}R zqt1>QE-@v0WI#7<#%s#Uj1CSTwhLt&U4AX2W1u)>6CzZnG(1=^(8ib1N13)!d%!l! z;7=t_PkRS>k>B69{rGr#)E*rz4i6QI*75G~6zLW`TGj_gj*lnN71%#Xs-}~GzYLe$ z^P#A$NLv?GMYF-CtD^U@ zA?TcBaw;crvfwC=Da)Gd7=~!8Hm~p!FUpppo0_1jiflNdBH|4!I`k=|r81|^q82Tc z5vWB@ArZ)5g|a&w3Q~*Hknc&fqDCn?=4Xdk)%5WvqoNA7EhxIo>k7w#1l}}QO*A2B znR7(YX{($g^R|W;2w;SVW+yGe`7+F&iht`kg&JUEO_!zRlaZdiy%J?PzOXNm#aV-!}PNp?BxHlfQCv zcVAb}uFkC>>fW?{XAgNV$JVo9T-SW98Wk;31Vez}6RR({$=5#+l{L=+Ot0evP*_7TSsm>}6xaz8RyyW_ zXtJ^?vv>uEb|R8&S-T|hW!eOxLy&;$?#~6AAfoqz%-SZ z+62gMFmwGgHD9r#v!`!!-{$Sz%(}hIiS1+Z6?J$U%d@S|MMYz5N!K_C1UOHfqFZ~$3j9Rm)CB`KOIahe^dF~0wQb0YiI%$4cRmixi~ z|HstL=YzXFFMp%EzjC;7@oQCuM2WYAvfw^4sq3WEP4K=uHcDLfsy(;nedw4nxdXEt zcHm{BqSvFrX^66Hvzlhx4r^&Tr|P_Ai6)jXmMo_c-djM@85^*dzjw7p3U92tn`oj`LN`!X?WrJsEA+TL~P2Ismr?U zm^xHIu|-amSyMN8%|uYhbQDpt4U-r_5+~_88Y$@3~&`r-qBn1_SmjnvGpCpiE67r2v_(3Rq4$DT6Cl3{J4wBm`o0dv5-#3p5q)c2M zJZ5wNcXbF`6izy*aE!aE!d*c>-pNGc2n*bxfWmhO?n)+6=koz~1);5oy3kMuXd?=+ zLniUzxJ5}oi;@i0`0~g&7@?7N`TT$1NaPl0hSMKSzNvYj@lpC4^Wq!F9}-`Im+lHw zI5$B~F715jA(5q8oMTZpH*T-Ini!KB38%%zWR)?QjZVpLpfTxQRk?1|{ZXfI9gP~3z?8MyN zZF%6sk+&n(rjGBPPF9wAI%6(mhVi9DKyi>$@*njy2C!n2xe@jS;WhKR@) zfG{Fb5)zNxrD%wj= zXA8dEIGe~8-x>{>2Oa^aqrDFjTdgig4p~TOIU98e9j@F96Kt1CB0k>?CFQWiDh-b? zGmq$^!`MfQqvg>vIIQb9ZWHNON9VW|Wkr?@gVO;kbwpj3Bvw!aYgRB;$n*ab4<@qTY<*|yzLxH$tLcBi!?SmDt;#*fpl?!1 zu&Or{qwsHPG6$!Z?({4{-zfCVLw(5d;m2pfV9mDm1m5V(=>g$ zC3<9DGU+!oh&!UOE5TNAA<(~hmOiUu3gQB+n(Ivr_r3DA1Z zCip32NNq2Ea<)OLE(BA7d!hhxVF6X?C>#&iy(Frt zVd~gOHAOKEfdfikaCFDfcw4lBK^X3}mAj!jTUCOmyf75idALT}6RLAUs1r3hI39@@+V(9_tP|^=29Pz$|l1DJBM8aU}`7|b`Q=3MxU2; zOJoHA^$nflbr#4=2Vi_jaDdR4Bomc8z+@`ed*oo}wY_o#n$n|^pyMT>Xv%9Rg+)@} zGdCGOgi!;)F7O>y$JuZhF$+|A)qKvW_LnJLnU`%v&R-UpZ3X;wh9rwDZ=se$7C1pN zB?l;M&K4b6&^TFPMSw0XRQ3pxL=f7#XxgNf2M8a9HG`FWRwW^Q< z>x|T7n*A~_x*}!{KOL2g1#mvy7DY!=HQm-!K>8ikfL0j1Dq182X6nFv36kRotYJB- zsLYugx$}Go38`D3kBXL{R8&wLw29StNwq8kAY)Ur6h$ERh$4vC zT{P5@I=X_*#8fTR3h{!1f-!W+;5mJE|G$a8E0J~5f0z7J({W7WTYLq=y8;hi>9#g5 z8DGt@#GA@C_}?_X0?`w_A9%viy9NrV7aA=N9jcxe^h1YA_94AITsl6U-EN>d^@wfV zHfERZLC2u>2amt%c-wf>UEbq8_aSUqwKc8p=|Vb?kRJ4O?bzD6zKiMWT(`B0Ie5q% zSp{Z-L-?*$d`GR#EKV3gcNfe31(dRlFB zdNoQMvjJZ@W&bctbs1CD@5V4ao!z%|71r%szoo0M`r}qIw{&%P_26R)ojpCBd)rvA zrk32bSAQ_WY~Ml3qk5UPn$Kkz%SNGpafH<3F>9D*8>$`-JIrSPZ(@4L-^5J*O}6E` zo6Dvbdz9?E#)_q?zrNrU2arX@NB8yY>}tPp^4sU`MP?UeelId!q^?S`Vmh{|7$$4- zChD_b{PD7^W8(!Uge5sB$ikM)BSDJ)hGYQt@0bdzzfsV~1)kgwU+1Q=MEW?ExRIBJ zvP2#^=@OZ-MkZxSoXs;?FNaY3>D7RO{G(>Ul(B|Jb<_Ldi8WKL&VlQsF!>X*v}%5_ zxh{Z-U(%~ZK8+PHxkd{1(UD?F_x=V;zh@LnKzM1E3w_{dR7@Dbu@%igNEddCVZeyR zszKGFWIGOmx0-|c8KgAe9l_{DMni+Gg92z6!j{6Sx;Tq()a<4jmmEt2eSzij%TQ8Q zc6Hs|Db~Sgzh1Hk*&m7Gas^U@z#=etaIzFSkS?K)jWbIX9*c@5;DV^&Ffeq(ktBFE zk|>}69zi!vkp$ZYfULIyPxt#hKG6IK>bEx_kip{lHCJGv|ysNplg z$6%_8;=uPZWX%*b)nbF|_#V2*ZGmF0)QB(sP$-J|tFROkR>;8nA;=U+~&r`Z+y8%0Ae2r#gJZs)qIMjH*4f{aBQGl z92uBk&l|1zC}d@79#41h7-oW#Pp<+W=qJDjD;eX&lpF^WkOY1uW1i$`q4t}Ix$j{Y zYQCR5OKvruauu@|qtAHxfbEo-JBN!yGe3XkC(Yy?G*13DIJ8rLGOX_@@Wss6pUH1< z@=O_H&iqvJ%j37G`r0cU^O%oL?pm) z=n_)z@Q``V0R^6ISgI)i*8v+}lyQcNV7k#}NwBVVn-J7KsuBF)b)g)je^jHxbF%P+ z4h!gV#HV~8;NU5&u^F)^p=wPP_<1tKnj*xffpiD#ebaQ11VJ`U22mDg*aA`{wxvKz zfZvcERa3!}Wr#TAI9uUGB-dbu$r=zJfuR+*%WVYOEUgiAqZo?*y#0*zr`8&;PQdB_ ze2{GYNNM;^aIc;liqz2Z_GDBxA_0I7@a2hs{Rk$KvjoZko)%t4n#}@&40Mc!p|B+U z00^0c@&t`D9LHvng_9&J*lu#KavQJ_tF{E0SGghQiZ6v2hzc(+&HJw@ep{OVc zi_=O5{~{}ZG>aTt&==d`MfiKHhRCjla|`hage}N(*o9OXY+NcYiwNri84|eLRQ9_x zp>NR$=J1(N0#lKtZ^_cBhazIn1XeRVbhAql`WB6(vSRv{K>8MyWqGk?HzVfrsp(s0 zzHc7sTTn0i(`^j35kJj^cs}jP&Mwy3Lnyd$?BWB;8{9t6VH;##rlExyV34(Kg zk8~Cp8l0FCV)8f#H4{O59`-s1)7=1Ju>jUP0Q3Qf1RPsZf^qqke)nQXy_@5RCH3`C zq<+ghNxcqPFVA+bjml;N0$SCuQL_usk!pd((1Flxg9n8za*pH=H%jkt{Wsp0+aLUIba+0hVjX`a7Y>diz{uJq1McW+%Xmb;$gh?5@bb5CxD7 zivl`VfVdLrH{i^8Rnk}jM6?y#05%?^^eRfXK~!y+h~tPXaL~Zfn@H6nksE9S_q!MR z>zf!!=h*t5lDU_+t3%eWNWVEUPBHTASg`PIc*8`~2?^A}tO>NK%>%H`>kh99Afq-# z0kwo6Gj&iUBWVKQ?r3mg*nDu0iprht%OL9>jw8U8=R#TEo9DW|X9Sn)koHyCu^my_ z&3RpPz`B44nr3RMs)Ax(vlWXIz}p~6A~t(nG&D&U!4r)XJCN%pXUagM1CgQF+$`Sh zu)74(-j8MtkoLDjk#_gINPDsiO&ub?GP~|rR78$SHi+8ns1IZ#PUoWX-z*g@4d>KYL|M^H-@h;iod=qCa!6?zJ& zGT5jQ*RqvhEKS997emw=IF?`*z8s3E*UeSbQq1BWSyrPN!-$cD3(fXzo-~5fn zqnO6Gm+A^UbjV%Qxa9TNUI|{|Yhk3P_N>QBt_j7>DZ?q&W;iFxj)C}9rZ zWaiL|s^-(@Cy{??%NH(*%Dw|2yMgL6fYvP?nR->Wfow%DDtHBGOoVDQUXxf9NrM2N z6I9j)2p)-iL<@j?M~{PGM4WrT%{MN2Ei4j}$v+#NGv-gv?V~7r7}5vknB5mUYCoVb zHd;Ux3XGO{%Jg59`qk&6qF{!FhDRup&@2^Rrv=QnA=x5=je^8GvW|F@gR)5Uy#at(6L(~(gu1DHs~0Ek3qZA7dkFHk9&*uGFv0P+VFQ7+p+Q8vNzqHNMYTP4^{ zsswN^YtGVrai5!oVu>6}0@j;DQLMiTNwI45;*8NGGxW+UZ6NsI`8gObXn5{mRMd-f z_OU^7sfY-x$fCm;7J9s3{}Kd_pu%82R2t-xi(U`pC~hX{$7BqCOHP(_HH#hv0C-N<@L3^1MkFE#yl^*_ffmM1$k&G`NN-{XWZrkAsk54g8DGRrSHG&X41rR~mM z8`ZlLue<4{s;_B3ah0d($MizgZI5Z6&1O$K)n=+6usYBJa&L?CPp7JY{m1+hL9S^?C?OTj^0 z;{Uf$FC=m;nZ@Zlk{@n&qT&L3JtDQ+2<}rR9fVCl3|QH1XOhh$We_C?1{#C76_SHBxYr1IDrkhdV~bN@l-4Y8Oa<-8#~cxkUCm5eef3uII}2?qv`W zAOa+bxZoN3bBWB)9*FoB3>5Cmal!(HOL9=>7z%d^uQjb~wvIILs_bAyWH!E50y}|j zTOf2tm78g)K;Vkt3}h_@#05zE1HS=Y0z}YN;BqAuxqrYmQDDiEzQjP&2S%MLWA3Gp zdOydJbj%Mz*-QK8n(d#hSx+5;zb@Gy5qoKTEiXC%QUhKKRvaKa3?MVnT1S*r9`$S} zUNTjr;eg0hdCpKQz`HD60e3nA;3!m5)nKkEAOHV!B6nq`n7%vty5=`G{vrJ*iKow1 z_qQ_aE{CG-=Y*FGbXHBIGNa;dMvQ#L!MUqw?iGp4BBP=xz6FW84Gol4-2h3YhPpcR zs$hXfVr56r1=P-glO6Q_*q6Za#EF=La7ZMBfd!elNp3`3M z(||`_0>XzoO)Ji?L+n>1_dgL8v4i`HQ%SlLR6`ml+!SDpd54z`0Zm#=6GdRQrh^Gn zMK=xsJvVek!@jR*k_xsrkP`=+phfo;5IbT!^P{h9Jw8cJSo3G{Ms6*J7CBOV^ zRD^Bn7S^u}Y?q~@K3+okkLkd@U?E9c0sDrm+o0+L4~Jzq3b@h$F^2_efVffC9VM7{ zx-#T0gRpPsc!JLPRw%;WcG|+OUE?Pw0jfjdndYNwqatw>3J8v5!mqSt2v|iGkSf5J z01bkvsuqd_u$Xm0MH!IJIf?_z4n89feSj4JBC_D+&AV426odnTz-FEZMdG~^(Yp29 zcXs!+tvJI_9kY8=S5Fr+25M=K&AFH9-nn&a9df6Vua0~G?|{o-onlqT6c)gOl!0&t z-T3-+%kwQ4CUaEFyCY(*1x9Fz|KCi%B9VPhtDed=f3k5H58_*V1*WcmyUASzz+(;- zgV?#{Pymnb^wRAU2R7i*B`}1y4Gs^1)z%|sI6ux*0W zp`AHY8Xg_>-&mcesiiHfEf@2?AMYC*VT^nHFY%rl29VDM^K_MgDSIE`H+MII6=^F^Q9FW4)9dZPtY+FK6n0QQ z+BT)3r(hDP)w}k(R4TE2QB+h4MkM?L&_<{liA#f)7McR0W!r>{XzP+Gpk@uN_+-Nd z4I7A;foW5DP>Fzs0MVjZnAkSC*ZJEMF)ZWl36NCMc%f(=?;cMPg9Y%?!I9(RNur^i z0aG=dvSr#54bMkKm%`khR`}Z!F?Zwbsa)6gw4A!4Eh;J%=JwR?Z%+hu8E;Q>+@7v& zFe3xj4s&~I^S390evG%L`nIQQn%Z`p_uG??|L3Os|9@`CP~4RB7tf|$fy!=IL_};Q zClV9k)1kyMoYe4(4@G4ypr^ZSh$gs+KvYHePmqED z^&qlQAUGmW15LJ&g+c%uNhtvoSMVZCcp6AOqeF%<3q9R#SMWtkj_QRXS|LKrWMPVy z_eD!i&V?dcF5TRJ-mhAOCppi+76~FOHn8VtV6SR=UotAX6sCdA`UW>?#oJST+tZ5lCn6K^4sCn# z{r|Tma_yO(bTXN0daPkJ^|om~Z~XA=UjbA%-sn{~O2n*PaMg|ZMCQj4+0X;AGwu%e z25iSmIgwb4H-(bSes6U$+nWyH+wI!(d_oNm|JdbDv_{3M9^~ zNBlQrDoj+wFQFrcfF^Vb=)6&Eq4N@&LMs}YwQ!QifgM`qY}Q0b+`vA=nU0Qb=8lDO zEDoS8#|o~#a5uUN_MJ2*5|Ce*oiXbK`FS~!cBVXB)(1{MHR37WZ16ts^SoEp0r9&b zF?cX4n#9YT2EI$fbkMN}=zd2=!v`G%t6;&jCGZ!bh!0g98k(&VEkbcQ4{}496mp6x zuLeW>D!s0ZjpS;u0z;EN5z0owPbkrf)7OAH#Hu#E|7TGVD*%{|ETS?;6ctq$(f&cx zP>gFEx}a+;3+bsEdO;A*cu5C^se^hBG>F$h5P<$(!mMJIe6bRk-9iwnoO%A~^Ioi; z|4(y?+_nspdamWy&7F+~9>%x$3Y`23xU1ZCjZ2R0;Ur=wUKWajGCr}rpK+JQskQ*j z#`)?1F!zHaPbZE=2UVf-tR|u=AE0ai{z2j|0MxJAf?$G5Mn>&ECz19*=*A8{5ffFt zKv|#?&;n|k95I2Ae^B;y;@}WDiCC5~WnWbRS)R(>9TC?fFbG!R-L+m_r$o%X^P#R2 zQtN#&8iqr#)DE~eV^ON`8X>i3LNU@K+nPL2CBGJteLGOV?gsZJ`18%2L|}@1D97}l zc>eOVIN~H3V$qrk0g}=a&U-Zp_jwZ%@;gPerzS{Uwhy5$yQCvpU6?GP@|*v7b=#94 zTJ;O8;p&JB<*9}*JQ*R^5t1e zM3WUmL|ZUZLq!(a11ht`h1%UUF!y$F62U~qjI05StWcKwk3~gU!eeB;-d*j{eM$sn zx!~zO^NH-05sCK&o)dolzc-Ox*Lp>AZ}XnUcQrIqy(f45;%86m3V`%>v$qOlVp}cv zRj??L`NR{^*!luj0l*fUJg|jKtfd79wt$H4i%6+cu!yd6Z-I#VKsE#sy)6`vW$Wy5 zr+u@frfd1Z*0Klch#NIzk3ARl74<52BOJ`5Af<)4UI;~8k59TDZM{3UwXN^$?IIjb z-HgxF&aCU|+uenSEJJ#Ca4geDe#kLhTk%4c>FVCl&aA-mYzN!UT+d9hMxj~gh+Q(l z-cBm3ibKBFk)Ci}s+6E(($_c%^ZSkUC;;_r)D-~TX1cs?Gcv)iJr#xny61AJ)ui7? z9Lfp)FAuj;)83`#RnrOl0W*Pv3d(*U5zvaWADtIZ^sntG6#6>XZSCqU6ly-grwa2O z)S|U!nQoYb_H|j!i^y*j&^6|!H4LZlNVW?C+BC7s>Ar8Z|2_$bqnYoU$8Ixun%W+j zoM&j=X56dX4RC(fauNafA)N2s=Cv2!_yaYfpHSe^@ZANE*r?`*)Ae@R_H%lwy1ceO z#qxT#DLV1Zk4NL28egk|qF9j;6|rT3Lwi|pmFnoJhMtX_44Na5(}1}eWuibhNT!23 z!$uJ)h!C-^WlIj`pLys8cRe<`64E13k|)DZl06sA&qg;Vg;`dsFlV&IAu2^Eg2@iY z*QoL=TVw4OH|>4%y2T~_;;m89UQ=WhpuNDqCEFUHIw)jzEO2m2=<@&qW?4bM4hd|_ z91Dg{M?pC>Ab|kxa1+>_!0eo#|6iZTex)^^+S_t()7u(;M6W+BKR13+dMuCwd2u4&5%p{kLQ>nkkd#7<@dY1}TAaxKQPcyol>=_Km!Xx3?fFb74s>KQ zw2SD>Hx@<$iS?Q^$zxEPt(jWI5!3Kz48YGKr=9B87%8wOe{@H}40>6$Hp+DevWE6}Aw6Wv=8wc5eS z1d$g*HUc0Ui>L^NI75SMRQdluPp@t{3jhCCvHu@C_GfSf;9TzZol9cKE_mnif`x&1 z*yV?8h;h2$LN*Z5h9{z72h8Rk?)2P4B`%wo)^}$y)%@>!A}(Gy8hxIlp%5c2b~J(< z4UvbVA)UFSq0W5YJUSZCz;Z-1FhoZKr9Zu1h(jSj$MYe?v6yOnW_YQ3EYdMvemR|}bbGW(|DSav$mdS#>I{Or4mu3%qckXMpZLBnO!$9!{ zbGS4@+PvOi^{+CDhwd7)OZTkux&y8veH2#lfbLqhqmK=g3zl(wJX?kNB|pra_~EG8 zZx4b~D_Q96>h0aUy&G@zKCRHNkM4G@%qHp;3(*jNb3bxBeQKO87oY_v%iIgxiz&lecv#VLNkJH^%T3V78s6Y>=K!rYP1*~?pROhie* zf+heFEMVT< z?-|!BG1JZmz$y~RCkX}4A_?bBb_!BLw7?mmB665A^_u4|0z3+U#9 z&qiIq``p_hU=fY(A>bIbQH9#LfJ%+kaX6n7>lb0Uy6i=KJx5|L);I!SvX~YQrW+<-gEobKC94Xp&7YNsSVG!yr6+tn6&J+Krt$dJc>^YLFP4(p`)QUi_BjX|Z@o~XyGnHZ2hP?NqKnwEzb|~_`^~~fw2J9OSCvGSl)(fLf-2*jab;vhO)%h zoY@keCht0=+|~*P<^I%ovJA)f-r*tQ9pPb$JhbeAvW~}F66D__$9s-Hg#QoY|9B$_ zMjjtO`%+eU1yrA^!;22@@VcVOVE<%Qkbf$&#oDqcY6f^gSy0DwyoHVetZr+P0D&XK z1*Yj>Dc|WDka7>F5*zfnP^8>-W>TKoyS>@)P~B_#+RUTRMM>I{u3Czzse-IyC7Ysw zQfS9CRYy<(DP|3h#~No9R+lVY*Ho-{lLbRK$ifX?GgT#6+J>upYg#1;z;j|vPf5Gp zlltn^tu-dRoZ4|VL_%UXX@N9^G2@7ZtZ8zv%1M#9V_ImKEeJX*YPO6z@!9t7qFaER zcW^2}9{xipa^6sfoXdkFb#L8F?yGf#D$l;KRZBoybJgMkfiD_-Yt_*Z?}iU^E$j%8MPJ*)C9vB4EniN8H_Smu$vFu)3{GXq@cASlMLs zQHC_6Ye$2+wuvtp)Q`5E_<8%80T$|MYnSV2_<$NE`$1C_($vloW!|cSU?i#AeB}XJjtWi z7;ou30(po91fDCEA@?wZeLJTSm*88W7!jLKv3;DRec&`%6V5hhH2#eA=o}kUFQYyY zspYK;j?Jko8~_pFJPXlTH2g9kY*Aqmt}|p4QC#>AifpQ)fUqu<0mMQG`ADWFsk0!G zhTJ=Si4*tWoR&COm&6(Rv9(cJ))WwsHCPG#q+y98 zs3#yhgAfp_Xc~l!sDO(7U1e=SLn9Df6KvB|R9Q1PT&hK5qa;!`k+d!%->`)mGdg_# zzbTP@S8GpdtmUmuA8z0n1*uLjfAv6DcA@wtL zoXb+~6MLd0T+6U{&gN}^o*W#-f~J_jLm*s&j@g2tilE|!jcuXAN8&8>X(uKdIl)bm zi50bA5+?}vE?>CB23r`y6`McySS(zB<&OBmC9uYYAzZ#?3mXd;!bSFE*w3C23}c)< zDc6-fVN;2xqoh`$Wlx}1L%v!OoMfz4^{LhMEy>TtYBk~i)0vft?ALG?-{LC}UxD}v z#8)7`0`V1yuRweS;wunef%pncT!G3p?h#PZY~oo0;Cpi@0q1om%L|{i#=DJ?s^|%1 zp>dbHV~tCWZAH-s1a)C30N&S}Oi)#1I<*P>jwx~|X5;oArgv{|U)Q!m-{x&y4BG8( z-^h?@Z(rxO9fjVV>)O`u?CI(1?(<&YJ=@m4lG)td*VVJDb1Pc-ZrZ-Hr+w~uhCH2X zid5;AElIj2OQNHiAP6%6(X^1(M`4yEn3iOtIiO%@Iwt}2uNpk3*}!xu7=YJBZ%aNH Hl>7eyiCzJo literal 0 HcmV?d00001 diff --git a/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json b/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json new file mode 100644 index 0000000000000..45b620232cbdd --- /dev/null +++ b/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json @@ -0,0 +1,5522 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.derived_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549585813, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_3.derived_table` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549585813, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549585813, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178195, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549569547, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178196, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.derived_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178196, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498949, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549567376, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:composite_0db44e02f671b69df68565346e9d2b68c7166fccf75bd494f34560bfa16c381b" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_0db44e02f671b69df68565346e9d2b68c7166fccf75bd494f34560bfa16c381b", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create or replace table _smoke_test_db_tmp_tables.tmp_table as (select * from smoke_test_db.base_table);\n\ncreate or replace table smoke_test_db.lineage_from_tmp_table as (select * from _smoke_test_db_tmp_tables.tmp_table)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549567376, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549567376, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064997222, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_external_table` as (select * from `gcp-staging-2.smoke_test_db_4.external_table_us_states`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549602653, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549602653, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178198, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_external_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549602653, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.external_table_us_states,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178200, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.external_table_us_states,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_external_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178199, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549605511, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_0db44e02f671b69df68565346e9d2b68c7166fccf75bd494f34560bfa16c381b", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064997223, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549580688, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549598252, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_0db44e02f671b69df68565346e9d2b68c7166fccf75bd494f34560bfa16c381b", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064997223, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace view smoke_test_db.view_from_view_on_table\n as (select * from smoke_test_db.view_from_table)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549572202, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549572202, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498921, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549598252, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549598252, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178202, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498922, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178203, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178203, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_view_on_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498922, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498952, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549572814, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table smoke_test_db_2.table_from_other_db\n as (select * from smoke_test_db.base_table)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549574990, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549574990, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498946, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table` as (\n SELECT *\n FROM `gcp-staging.smoke_test_db.base_table`\n FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)\n );\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549605511, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549605511, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178206, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549588112, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549588112, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498924, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230201` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549591093, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549591093, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178217, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178207, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178218, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178217, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549588804, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178207, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549600590, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549600590, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178211, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),revenue)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD),revenue)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD),revenue)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue2)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue3)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498924, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178212, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging.smoke_test_db.table_from_another_project` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549583501, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549583501, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178230, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178211, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178231, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_another_project,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178231, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498946, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549600590, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498924, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549578210, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0" + }, + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549578210, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),date_utc)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD),date_utc)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD),revenue)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD),revenue)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498946, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549588112, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549588112, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549588112, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + } + ], + "fineGrainedLineages": [ + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),date_utc)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),date_utc)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),revenue)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD),revenue)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue2)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + { + "upstreamType": "FIELD_SET", + "upstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD),revenue)" + ], + "downstreamType": "FIELD", + "downstreams": [ + "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD),revenue3)" + ], + "confidenceScore": 0.2, + "query": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:005ef53d98fe9ce2d807a16f00695367e6923b11729f2fba0db3e694bd2fe9c9", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n insert into smoke_test_db.usage_test values\n (\"2022-05-01\", \"seven\", 7),\n (\"2022-05-02\", \"ten\", 10),\n (\"2022-06-01\", \"four\", 4)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549557813, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549557813, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498926, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:005ef53d98fe9ce2d807a16f00695367e6923b11729f2fba0db3e694bd2fe9c9", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498927, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_snapshot_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549577574, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.snapshot_from_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498953, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_another_project,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549583501, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:005ef53d98fe9ce2d807a16f00695367e6923b11729f2fba0db3e694bd2fe9c9", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498927, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace view smoke_test_db.view_from_snapshot_on_table\n as (select * from smoke_test_db.snapshot_from_table)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549577574, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549577574, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498924, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549571609, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_view_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549572202, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498925, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.snapshot_from_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_snapshot_on_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498925, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1721549574990, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.snapshot_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 1, + "topSqlQueries": [ + "\n create or replace view smoke_test_db.view_from_snapshot_on_table\n as (select * from smoke_test_db.snapshot_from_table)\n " + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [] + } + }, + "systemMetadata": { + "lastObserved": 1724064997260, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace view smoke_test_db.view_from_table\n as (select * from smoke_test_db.base_table)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549571609, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549571609, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498930, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 3, + "topSqlQueries": [ + "select revenue, date_utc from gcp-staging.smoke_test_db.lineage_from_base", + "select revenue from gcp-staging.smoke_test_db.lineage_from_base", + "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n " + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 2 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 3 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178262, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498930, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 12, + "topSqlQueries": [ + "create or replace table _smoke_test_db_tmp_tables.tmp_table as (select * from smoke_test_db.base_table)", + "\n create or replace table smoke_test_db.table_from_view_and_table\n as (select b.date_utc, v.revenue from smoke_test_db.base_table b, smoke_test_db.view_from_table v)\n ", + "\n create or replace view smoke_test_db.view_from_table\n as (select * from smoke_test_db.base_table)\n ", + "create MATERIALIZED VIEW smoke_test_db.materialized_view_from_table as (select * from smoke_test_db.base_table where revenue>100)", + "select revenue from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", + "select revenue, date_utc from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", + "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table` as (\n SELECT *\n FROM `gcp-staging.smoke_test_db.base_table`\n FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)\n );\n ", + "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", + "select revenue, date_utc from gcp-staging.smoke_test_db.base_table", + "select revenue from gcp-staging.smoke_test_db.base_table", + "\n create or replace table smoke_test_db_2.table_from_other_db\n as (select * from smoke_test_db.base_table)\n ", + "create or replace table smoke_test_db.lineage_from_base as (select * from smoke_test_db.base_table)" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 10 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 2 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 5 + }, + { + "fieldPath": "date_utc", + "count": 4 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 5, + "topSqlQueries": [ + "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`) ;\n ", + "select revenue from gcp-staging.smoke_test_db_2.table_from_other_db", + "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230201` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`);\n ", + "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", + "select revenue, date_utc from gcp-staging.smoke_test_db_2.table_from_other_db" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 4 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 3 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178265, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 5, + "topSqlQueries": [ + "select revenue from gcp-staging.smoke_test_db.view_from_table", + "\n create or replace table smoke_test_db.table_from_view_and_table\n as (select b.date_utc, v.revenue from smoke_test_db.base_table b, smoke_test_db.view_from_table v)\n ", + "\n create or replace table smoke_test_db.table_from_view\n as (select * from smoke_test_db.view_from_table)\n ", + "\n create or replace view smoke_test_db.view_from_view_on_table\n as (select * from smoke_test_db.view_from_table)\n ", + "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_table" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 4 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 3 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178260, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4f5fd82d4808115ef07900a543b7d6e3551899815d11a945870c607d2dbda56e", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "insert into smoke_test_db.partition_test values (\"2022-05-24\", 20), (\"2022-06-24\", 30)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549560560, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549560560, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498928, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498930, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4f5fd82d4808115ef07900a543b7d6e3551899815d11a945870c607d2dbda56e", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498928, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4f5fd82d4808115ef07900a543b7d6e3551899815d11a945870c607d2dbda56e", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498929, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cb345cc231c81ec7b871d6727437a87b5bc18a95ecf37e857f07096254c2d2c1", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create or replace table smoke_test_db.usage_test (date_utc date, key STRING, value INTEGER)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549557257, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549557257, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498927, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cb345cc231c81ec7b871d6727437a87b5bc18a95ecf37e857f07096254c2d2c1", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD),key)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD),value)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498927, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498953, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cb345cc231c81ec7b871d6727437a87b5bc18a95ecf37e857f07096254c2d2c1", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498928, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997269, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "INSERT", + "customProperties": { + "query_urn": "urn:li:query:ebfe552aa0ddb538f3a6c4d444aff757e6df574f16a6dffc3ac146ce587fc491" + }, + "lastUpdatedTimestamp": 1721549563403 + } + }, + "systemMetadata": { + "lastObserved": 1724064997270, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create MATERIALIZED VIEW smoke_test_db.materialized_view_from_table as (select * from smoke_test_db.base_table where revenue>100)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549581208, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549581208, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498938, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498938, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_snapshot_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997268, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:dc49a3d580b4df6c8d24961c39a18b3569d8e58783fe9895324876da32d98d1e" + }, + "lastUpdatedTimestamp": 1721549577574 + } + }, + "systemMetadata": { + "lastObserved": 1724064997270, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997271, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:53d20616e4dd30dfc16ccc5771998f5ed93c9afa9b846104a19d072ba364fb5c" + }, + "lastUpdatedTimestamp": 1721549562792 + } + }, + "systemMetadata": { + "lastObserved": 1724064997273, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997267, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0" + }, + "lastUpdatedTimestamp": 1721549578210 + } + }, + "systemMetadata": { + "lastObserved": 1724064997269, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498938, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4bd08adca1e16a1e10673f736ae2c91e0ee68fd56b187bd507f360e429dbcb8c", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498948, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4bd08adca1e16a1e10673f736ae2c91e0ee68fd56b187bd507f360e429dbcb8c", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table smoke_test_db.partition_test (date_utc date, revenue INTEGER) \n PARTITION BY date_utc\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549559855, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549559855, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498929, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue from gcp-staging.smoke_test_db.lineage_from_tmp_table", + "select revenue, date_utc from gcp-staging.smoke_test_db.lineage_from_tmp_table" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178256, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4bd08adca1e16a1e10673f736ae2c91e0ee68fd56b187bd507f360e429dbcb8c", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498930, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 5, + "topSqlQueries": [ + "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) = (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "select revenue from gcp-staging.smoke_test_db.partition_test", + "select revenue, date_utc from gcp-staging.smoke_test_db.partition_test" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 11 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 4 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 15 + }, + { + "fieldPath": "date_utc", + "count": 6 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178266, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4bd08adca1e16a1e10673f736ae2c91e0ee68fd56b187bd507f360e429dbcb8c", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD),revenue)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498929, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997280, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:b9c1cbdddc1018284bdfb113865f5dc95b5c5a106c8e1dad1297bc9ef70debf1" + }, + "lastUpdatedTimestamp": 1721549574990 + } + }, + "systemMetadata": { + "lastObserved": 1724064997281, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997278, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991" + }, + "lastUpdatedTimestamp": 1721549588112 + } + }, + "systemMetadata": { + "lastObserved": 1724064997280, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_view", + "select revenue from gcp-staging.smoke_test_db.table_from_view" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178259, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue from gcp-staging.smoke_test_db.materialized_view_from_table", + "select revenue, date_utc from gcp-staging.smoke_test_db.materialized_view_from_table" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178271, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 1, + "topSqlQueries": [ + "\n SELECT\n customer_id,\n date1\n FROM\n `gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition`\n WHERE\n customer_id=1\n" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "customer_id", + "count": 1 + }, + { + "fieldPath": "date1", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178264, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2957e74d80b00aef4f3dc7b0b323d1fa863c78fd882d858186579c0737df00e2", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create or replace table smoke_test_db.lineage_from_tmp_table as (select * from _smoke_test_db_tmp_tables.tmp_table)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549567376, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549567376, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498932, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997276, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b" + }, + "lastUpdatedTimestamp": 1721549571609 + } + }, + "systemMetadata": { + "lastObserved": 1724064997278, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2957e74d80b00aef4f3dc7b0b323d1fa863c78fd882d858186579c0737df00e2", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging._smoke_test_db_tmp_tables.tmp_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498933, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2957e74d80b00aef4f3dc7b0b323d1fa863c78fd882d858186579c0737df00e2", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498933, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 1, + "topSqlQueries": [ + "\n select \n transaction_id \n from \n `gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition` \n where \n _PARTITIONDATE = CURRENT_DATE()\n" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "transaction_id", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178275, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997286, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1" + }, + "lastUpdatedTimestamp": 1721549569547 + } + }, + "systemMetadata": { + "lastObserved": 1724064997288, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.external_table_us_states,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 2, + "topSqlQueries": [ + "\n select name, post_abbr from `gcp-staging-2.smoke_test_db_4.external_table_us_states`\n", + "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_external_table` as (select * from `gcp-staging-2.smoke_test_db_4.external_table_us_states`);\n " + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 2 + } + ], + "fieldCounts": [ + { + "fieldPath": "name", + "count": 1 + }, + { + "fieldPath": "post_abbr", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178259, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997288, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:4bd08adca1e16a1e10673f736ae2c91e0ee68fd56b187bd507f360e429dbcb8c" + }, + "lastUpdatedTimestamp": 1721549559855 + } + }, + "systemMetadata": { + "lastObserved": 1724064997290, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 8, + "topSqlQueries": [ + "select revenue from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`", + "select revenue, date_utc from gcp-staging-2.smoke_test_db_4.sharded_table1_20230101", + "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101`);\n ", + "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`);\n ", + "select revenue, date_utc from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`", + "select revenue from gcp-staging-2.smoke_test_db_4.sharded_table1_20230101", + "select revenue, date_utc from gcp-staging-2.smoke_test_db_4.sharded_table1_20230201", + "select revenue from gcp-staging-2.smoke_test_db_4.sharded_table1_20230201" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 5 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 3 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 6 + }, + { + "fieldPath": "date_utc", + "count": 3 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178267, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_another_project,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_another_project", + "select revenue from gcp-staging.smoke_test_db.table_from_another_project" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178269, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997291, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "INSERT", + "customProperties": { + "query_urn": "urn:li:query:4f5fd82d4808115ef07900a543b7d6e3551899815d11a945870c607d2dbda56e" + }, + "lastUpdatedTimestamp": 1721549560560 + } + }, + "systemMetadata": { + "lastObserved": 1724064997293, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 4, + "topSqlQueries": [ + "select revenue from gcp-staging-2.smoke_test_db_3.base_table_2", + "\n create or replace table `gcp-staging.smoke_test_db.table_from_another_project` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n ", + "select revenue, date_utc from gcp-staging-2.smoke_test_db_3.base_table_2", + "\n create or replace table `gcp-staging-2.smoke_test_db_3.derived_table` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n " + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 3 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178263, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 1, + "topSqlQueries": [ + "\n SELECT\n first_name,\n last_name,\n dob,\n addresses[offset(0)].address,\n addresses[offset(0)].city\n FROM \n gcp-staging-2.smoke_test_db_4.table_with_nested_fields\n" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "addresses", + "count": 1 + }, + { + "fieldPath": "dob", + "count": 1 + }, + { + "fieldPath": "first_name", + "count": 1 + }, + { + "fieldPath": "last_name", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178276, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:53d20616e4dd30dfc16ccc5771998f5ed93c9afa9b846104a19d072ba364fb5c", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create or replace table smoke_test_db.base_table (date_utc timestamp, revenue INTEGER)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549562792, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549562792, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498936, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:53d20616e4dd30dfc16ccc5771998f5ed93c9afa9b846104a19d072ba364fb5c", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),revenue)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498936, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997294, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:cb345cc231c81ec7b871d6727437a87b5bc18a95ecf37e857f07096254c2d2c1" + }, + "lastUpdatedTimestamp": 1721549557257 + } + }, + "systemMetadata": { + "lastObserved": 1724064997295, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 4, + "topSqlQueries": [ + "select * from gcp-staging.smoke_test_db.usage_test", + "select key from gcp-staging.smoke_test_db.usage_test", + "select value from gcp-staging.smoke_test_db.usage_test", + "\n select extract(month from date_utc) month, count(*) count, sum(value) total from gcp-staging.smoke_test_db.usage_test\n group by month\n order by total\n " + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 8 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 3 + } + ], + "fieldCounts": [ + { + "fieldPath": "value", + "count": 4 + }, + { + "fieldPath": "key", + "count": 3 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178257, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table smoke_test_db.table_from_view\n as (select * from smoke_test_db.view_from_table)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549572814, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549572814, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498942, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498942, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997296, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "INSERT", + "customProperties": { + "query_urn": "urn:li:query:005ef53d98fe9ce2d807a16f00695367e6923b11729f2fba0db3e694bd2fe9c9" + }, + "lastUpdatedTimestamp": 1721549557813 + } + }, + "systemMetadata": { + "lastObserved": 1724064997298, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:53d20616e4dd30dfc16ccc5771998f5ed93c9afa9b846104a19d072ba364fb5c", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498937, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_view_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue from gcp-staging.smoke_test_db.view_from_view_on_table", + "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_view_on_table" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178272, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498942, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:cb345cc231c81ec7b871d6727437a87b5bc18a95ecf37e857f07096254c2d2c1", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498953, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ebfe552aa0ddb538f3a6c4d444aff757e6df574f16a6dffc3ac146ce587fc491", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498953, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ebfe552aa0ddb538f3a6c4d444aff757e6df574f16a6dffc3ac146ce587fc491", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "insert into smoke_test_db.base_table values (CURRENT_TIMESTAMP(), 100), (TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR), 110)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549563403, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549563403, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498935, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ebfe552aa0ddb538f3a6c4d444aff757e6df574f16a6dffc3ac146ce587fc491", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498936, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_multiple_tables", + "select revenue from gcp-staging.smoke_test_db.view_from_multiple_tables" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178268, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ebfe552aa0ddb538f3a6c4d444aff757e6df574f16a6dffc3ac146ce587fc491", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498935, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.customer_demo.purchase_event,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 1, + "totalSqlQueries": 5, + "topSqlQueries": [ + "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "SELECT\n (SUM(CASE WHEN amount BETWEEN 0 AND 10 THEN 1 ELSE 0 END) * 100.0) / COUNT(*) AS percentage_in_range\nFROM\n gcp-staging.customer_demo.purchase_event;" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 78 + } + ], + "fieldCounts": [ + { + "fieldPath": "amount", + "count": 18 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178270, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498951, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997307, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:2957e74d80b00aef4f3dc7b0b323d1fa863c78fd882d858186579c0737df00e2" + }, + "lastUpdatedTimestamp": 1721549567376 + } + }, + "systemMetadata": { + "lastObserved": 1724064997309, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table smoke_test_db.table_from_view_and_table\n as (select b.date_utc, v.revenue from smoke_test_db.base_table b, smoke_test_db.view_from_table v)\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549578210, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549578210, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498947, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498947, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997309, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "UNKNOWN", + "customProperties": { + "query_urn": "urn:li:query:a5b8ca0f0b97816db6ca440bdd7c6b11acc823fa70250396b902eb1d46835fbc" + }, + "lastUpdatedTimestamp": 1721549580688 + } + }, + "systemMetadata": { + "lastObserved": 1724064997311, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1721520000000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 2, + "topSqlQueries": [ + "select revenue from gcp-staging.smoke_test_db.table_from_view_and_table", + "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_view_and_table" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "count": 1 + }, + { + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "revenue", + "count": 2 + }, + { + "fieldPath": "date_utc", + "count": 1 + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178273, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_view_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997306, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:630f9169072a11dfd8d08a44479f2466acdf2dc2b078b946a739db437b74ad1d" + }, + "lastUpdatedTimestamp": 1721549572202 + } + }, + "systemMetadata": { + "lastObserved": 1724064997308, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997312, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:e1769381f2d261efecb105f3ab6fc8a2fc6717a1509cc65ba125c03841b0923d" + }, + "lastUpdatedTimestamp": 1721549581208 + } + }, + "systemMetadata": { + "lastObserved": 1724064997314, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724064997305, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d" + }, + "lastUpdatedTimestamp": 1721549572814 + } + }, + "systemMetadata": { + "lastObserved": 1724064997307, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a4d62b3996203c5661d02d28c1908d209a56e9966cefc274600a76335bc75de0", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD),date_utc)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD),revenue)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD),revenue)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498947, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997316, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_tmp_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997315, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498950, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178292, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:ccc4a40bb7abb852c9a97a2dc80c0928447bf28b91be0e41a80f691ca8e34e35" + }, + "lastUpdatedTimestamp": 1721549588804 + } + }, + "systemMetadata": { + "lastObserved": 1724065178293, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_external_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178287, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8" + }, + "lastUpdatedTimestamp": 1721549602653 + } + }, + "systemMetadata": { + "lastObserved": 1724065178288, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178288, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f" + }, + "lastUpdatedTimestamp": 1721549582803 + } + }, + "systemMetadata": { + "lastObserved": 1724065178289, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:005ef53d98fe9ce2d807a16f00695367e6923b11729f2fba0db3e694bd2fe9c9", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498947, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178299, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:6bbebdb71c5ef982b74fb6bbef58f7f33162566fd9b14385f2da971503317539" + }, + "lastUpdatedTimestamp": 1721549596824 + } + }, + "systemMetadata": { + "lastObserved": 1724065178300, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178293, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "INSERT", + "customProperties": { + "query_urn": "urn:li:query:e4e43fe8e17490b4b360a704a5604dd8d55763afd40f10c335e340132a9fe178" + }, + "lastUpdatedTimestamp": 1721549593090 + } + }, + "systemMetadata": { + "lastObserved": 1724065178295, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a5b8ca0f0b97816db6ca440bdd7c6b11acc823fa70250396b902eb1d46835fbc", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "drop MATERIALIZED VIEW if exists smoke_test_db.materialized_view_from_table", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549580688, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549580688, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498939, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a5b8ca0f0b97816db6ca440bdd7c6b11acc823fa70250396b902eb1d46835fbc", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.materialized_view_from_table,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498939, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_another_project,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178298, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8" + }, + "lastUpdatedTimestamp": 1721549583501 + } + }, + "systemMetadata": { + "lastObserved": 1724065178299, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178300, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d" + }, + "lastUpdatedTimestamp": 1721549598252 + } + }, + "systemMetadata": { + "lastObserved": 1724065178302, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178307, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:420cad6350235517b34e6e376414d89be0734f87fba6789754be420051d4901c" + }, + "lastUpdatedTimestamp": 1721549596155 + } + }, + "systemMetadata": { + "lastObserved": 1724065178308, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2d4a0fdd933c8f9009a21f00b3b0213b025d97ff5e6a39cd031e9d5e5c31f832", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n CREATE OR REPLACE TABLE `gcp-staging-2.smoke_test_db_4.table_with_nested_fields` (\n id STRING,\n first_name STRING OPTIONS(description = \"First name\"),\n last_name STRING OPTIONS(description = \"Last name\"),\n dob DATE OPTIONS(description = \"Date of birth\"),\n addresses\n ARRAY<\n STRUCT<\n status STRING,\n address STRING OPTIONS(description = \"Full Address\"),\n city STRING,\n state STRING,\n zip STRING,\n numberOfYears STRING>>\n ) OPTIONS (\n description = 'Example name and addresses table');\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549597638, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549597638, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178307, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "create or replace table smoke_test_db.lineage_from_base as (select * from smoke_test_db.base_table)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549569547, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549569547, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724064498937, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2d4a0fdd933c8f9009a21f00b3b0213b025d97ff5e6a39cd031e9d5e5c31f832", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178308, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2d4a0fdd933c8f9009a21f00b3b0213b025d97ff5e6a39cd031e9d5e5c31f832", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD),id)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD),first_name)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD),last_name)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD),dob)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD),addresses)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178307, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178296, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d" + }, + "lastUpdatedTimestamp": 1721549591093 + } + }, + "systemMetadata": { + "lastObserved": 1724065178298, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.derived_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178302, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95" + }, + "lastUpdatedTimestamp": 1721549585813 + } + }, + "systemMetadata": { + "lastObserved": 1724065178303, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178304, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:2d4a0fdd933c8f9009a21f00b3b0213b025d97ff5e6a39cd031e9d5e5c31f832" + }, + "lastUpdatedTimestamp": 1721549597638 + } + }, + "systemMetadata": { + "lastObserved": 1724065178306, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e4e43fe8e17490b4b360a704a5604dd8d55763afd40f10c335e340132a9fe178", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "insert into `gcp-staging-2.smoke_test_db_4.sharded_table1_20230201` values (CURRENT_TIMESTAMP(), 100)", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549593090, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549593090, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178296, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a5b8ca0f0b97816db6ca440bdd7c6b11acc823fa70250396b902eb1d46835fbc", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498939, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e4e43fe8e17490b4b360a704a5604dd8d55763afd40f10c335e340132a9fe178", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178297, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e4e43fe8e17490b4b360a704a5604dd8d55763afd40f10c335e340132a9fe178", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178297, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724064498937, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178284, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930" + }, + "lastUpdatedTimestamp": 1721549600590 + } + }, + "systemMetadata": { + "lastObserved": 1724065178285, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:8f7bb4efb71d494b2bfe115937d6022db0ab9e6ea3d293839a457480e75430e1", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724064498937, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.snapshot_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997317, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997317, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997315, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724065178323, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab" + }, + "lastUpdatedTimestamp": 1721549605511 + } + }, + "systemMetadata": { + "lastObserved": 1724065178324, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.partition_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997316, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_view_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997320, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ccc4a40bb7abb852c9a97a2dc80c0928447bf28b91be0e41a80f691ca8e34e35", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`) ;\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549588804, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549588804, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178294, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.usage_test,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997318, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ccc4a40bb7abb852c9a97a2dc80c0928447bf28b91be0e41a80f691ca8e34e35", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178295, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178326, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ccc4a40bb7abb852c9a97a2dc80c0928447bf28b91be0e41a80f691ca8e34e35", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178294, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2957e74d80b00aef4f3dc7b0b323d1fa863c78fd882d858186579c0737df00e2", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498948, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n create or replace table `gcp-staging-2.smoke_test_db_3.base_table_2` (date_utc timestamp, revenue INTEGER);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549582803, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549582803, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178290, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a5b8ca0f0b97816db6ca440bdd7c6b11acc823fa70250396b902eb1d46835fbc", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498951, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178291, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD),date_utc)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD),revenue)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178290, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:4f5fd82d4808115ef07900a543b7d6e3551899815d11a945870c607d2dbda56e", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498948, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:53d20616e4dd30dfc16ccc5771998f5ed93c9afa9b846104a19d072ba364fb5c", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498949, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997314, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb1e3c0fd1f6a26c2645cf4ba088a22ff346a9e323c6be451459ecfa7329a991", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498952, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:bb3d7f6685e1f71868d0821451e52bfcf1a3bdfeb34c739c0305386256c38f9b", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498952, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178327, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:b66112fa9691aa02354115b5cef8356390b524fa67c6b06e018c362ac8d0b31d", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064498952, + "runId": "bigquery-queries-2024_08_19-16_18_14", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_view_and_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997317, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_multiple_tables,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997318, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997320, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_snapshot_on_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997319, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997319, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:composite_0db44e02f671b69df68565346e9d2b68c7166fccf75bd494f34560bfa16c381b", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724064997332, + "runId": "bigquery-queries-2024_08_19-16_26_33", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6bbebdb71c5ef982b74fb6bbef58f7f33162566fd9b14385f2da971503317539", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD),customer_id)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD),date1)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178301, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6bbebdb71c5ef982b74fb6bbef58f7f33162566fd9b14385f2da971503317539", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n CREATE or REPLACE TABLE `gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition` (customer_id INT64, date1 DATE)\n PARTITION BY\n RANGE_BUCKET(customer_id, GENERATE_ARRAY(0, 100, 10))\n OPTIONS (\n description = \"Description of Integer Range partitioned table\",\n require_partition_filter = TRUE);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549596824, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549596824, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178301, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6bbebdb71c5ef982b74fb6bbef58f7f33162566fd9b14385f2da971503317539", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178339, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178325, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.derived_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178325, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6bbebdb71c5ef982b74fb6bbef58f7f33162566fd9b14385f2da971503317539", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178301, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178328, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178346, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178328, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:5efacf0ddad8fd852ba394b29e7a4654ea454915930fb8dd4882c6f294b95cf8", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178338, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178326, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:420cad6350235517b34e6e376414d89be0734f87fba6789754be420051d4901c", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition,PROD)" + }, + { + "entity": "urn:li:schemaField:(urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition,PROD),transaction_id)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724065178310, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:420cad6350235517b34e6e376414d89be0734f87fba6789754be420051d4901c", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "\n CREATE or REPLACE TABLE\n `gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition` (transaction_id INT64)\n PARTITION BY\n _PARTITIONDATE\n OPTIONS (\n description = \"Description of Ingestion time partitioned table\",\n partition_expiration_days = 3,\n require_partition_filter = TRUE);\n ", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1721549596155, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1721549596155, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724065178309, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:420cad6350235517b34e6e376414d89be0734f87fba6789754be420051d4901c", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178336, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:420cad6350235517b34e6e376414d89be0734f87fba6789754be420051d4901c", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724065178310, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.customer_demo.purchase_event,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178329, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2fa44cc9d306c7523477fad59ff43e2e580081ee770da69b9b9f66e119b4dcab", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178336, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:ccc4a40bb7abb852c9a97a2dc80c0928447bf28b91be0e41a80f691ca8e34e35", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178344, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6684f16158660588d874f7ac46dbd7e56ad42acfb95b8a3d1f01292de8dcb930", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178339, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.external_table_us_states,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178327, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:2d4a0fdd933c8f9009a21f00b3b0213b025d97ff5e6a39cd031e9d5e5c31f832", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178335, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:88c4674d369ef49e881a5ea67ed3485e48f09b9a4924d5282c3ae25004737f95", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178340, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178327, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.derived_table_from_external_table,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178325, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178324, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178337, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:e4e43fe8e17490b4b360a704a5604dd8d55763afd40f10c335e340132a9fe178", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178345, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.table_from_another_project,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178331, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:a049f27174fa88a7a7b1b7d5f60d2c353f3e9dd3d4994a8e35c91adb986eac4d", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178341, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:6e250c6966754a5e6532fbb444172dacf5813b0b7afceefbf7772a29878f48f8", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724065178339, + "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastRunId": "no-run-id-provided" + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py new file mode 100644 index 0000000000000..5874bf60a21f9 --- /dev/null +++ b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py @@ -0,0 +1,42 @@ +from unittest.mock import patch + +from freezegun import freeze_time + +from datahub.ingestion.source.usage.usage_common import BaseUsageConfig +from tests.test_helpers import mce_helpers +from tests.test_helpers.state_helpers import run_and_get_pipeline + +FROZEN_TIME = "2024-08-19 07:00:00" + + +@freeze_time(FROZEN_TIME) +@patch("google.cloud.bigquery.Client") +def test_queries_ingestion(client, pytestconfig, monkeypatch): + + test_resources_dir = pytestconfig.rootpath / "tests/integration/bigquery_v2" + mcp_golden_path = f"{test_resources_dir}/bigquery_queries_mcps_golden.json" + mcp_output_path = "bigquery_queries_mcps.json" + + pipeline_config_dict: dict = { + "source": { + "type": "bigquery-queries", + "config": { + "project_ids": ["gcp-staging", "gcp-staging-2"], + "local_temp_path": test_resources_dir, + }, + }, + "sink": {"type": "file", "config": {"filename": mcp_output_path}}, + } + + # This is hacky to pick all queries instead of any 10. + # Should be easy to remove once top_n_queries is supported in queries config + monkeypatch.setattr(BaseUsageConfig.__fields__["top_n_queries"], "default", 20) + + pipeline = run_and_get_pipeline(pipeline_config_dict) + pipeline.pretty_print_summary() + + mce_helpers.check_golden_file( + pytestconfig, + output_path=mcp_output_path, + golden_path=mcp_golden_path, + ) From ecbd7fe32e6461a323a705da69368497d61b1135 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Tue, 20 Aug 2024 13:56:53 +0530 Subject: [PATCH 14/16] remove unneeded log line --- .../datahub/ingestion/source/bigquery_v2/queries_extractor.py | 1 - 1 file changed, 1 deletion(-) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index d7adc0b161bf8..491a5cbc6ff92 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -246,7 +246,6 @@ def get_workunits_internal( if i > 0 and i % 10000 == 0: logger.info(f"Added {i} query log entries to SQL aggregator") - logger.info(f"{query.query_hash}, {query.timestamp}") self.aggregator.add(query) i += 1 From c90bcb9f2be8f49c1b5a6bb7dae4668d200175dd Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Thu, 22 Aug 2024 18:37:38 +0530 Subject: [PATCH 15/16] more changes - address review comments - support adding extra_info for debugging with queries - fix usage issue, add unit test for sql aggregator usage --- .../src/datahub/emitter/mce_builder.py | 4 +- .../source/bigquery_v2/queries_extractor.py | 76 +- .../ingestion/source/usage/usage_common.py | 44 +- .../sql_parsing/sql_parsing_aggregator.py | 2 + .../integration/bigquery_v2/audit_log.sqlite | Bin 139264 -> 0 bytes .../bigquery_queries_mcps_golden.json | 256 +- .../integration/bigquery_v2/query_log.json | 5404 +++++++++++++++++ .../bigquery_v2/test_bigquery_queries.py | 38 +- .../aggregator_goldens/test_basic_usage.json | 51 + .../unit/sql_parsing/test_sql_aggregator.py | 50 +- 10 files changed, 5832 insertions(+), 93 deletions(-) delete mode 100644 metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite create mode 100644 metadata-ingestion/tests/integration/bigquery_v2/query_log.json create mode 100644 metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_usage.json diff --git a/metadata-ingestion/src/datahub/emitter/mce_builder.py b/metadata-ingestion/src/datahub/emitter/mce_builder.py index 8b41366bd60dc..700bbc2f993f6 100644 --- a/metadata-ingestion/src/datahub/emitter/mce_builder.py +++ b/metadata-ingestion/src/datahub/emitter/mce_builder.py @@ -233,9 +233,9 @@ def make_actor_urn(actor: str) -> Union[CorpUserUrn, CorpGroupUrn]: CorpUserUrn(actor) if not actor.startswith(("urn:li:corpuser:", "urn:li:corpGroup:")) else ( - CorpUserUrn.create_from_string(actor) + CorpUserUrn.from_string(actor) if actor.startswith("urn:li:corpuser:") - else CorpGroupUrn.create_from_string(actor) + else CorpGroupUrn.from_string(actor) ) ) diff --git a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py index 491a5cbc6ff92..8457f4e37b3d2 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py +++ b/metadata-ingestion/src/datahub/ingestion/source/bigquery_v2/queries_extractor.py @@ -52,9 +52,9 @@ class BigQueryTableReference(TypedDict): - projectId: str - datasetId: str - tableId: str + project_id: str + dataset_id: str + table_id: str class DMLJobStatistics(TypedDict): @@ -67,22 +67,16 @@ class BigQueryJob(TypedDict): job_id: str project_id: str creation_time: datetime - start_time: datetime - end_time: datetime - total_slot_ms: int user_email: str - statement_type: str - job_type: Optional[str] query: str - destination_table: Optional[BigQueryTableReference] - # NOTE: This does not capture referenced_view unlike GCP Logging Event - referenced_tables: List[BigQueryTableReference] - total_bytes_billed: int - total_bytes_processed: int - dml_statistics: Optional[DMLJobStatistics] session_id: Optional[str] query_hash: Optional[str] + statement_type: str + destination_table: Optional[BigQueryTableReference] + referenced_tables: List[BigQueryTableReference] + # NOTE: This does not capture referenced_view unlike GCP Logging Event + class BigQueryQueriesExtractorConfig(BigQueryBaseConfig): # TODO: Support stateful ingestion for the time windows. @@ -127,6 +121,16 @@ class BigQueryQueriesExtractorReport(Report): class BigQueryQueriesExtractor: + """ + Extracts query audit log and generates usage/lineage/operation workunits. + + Some notable differences in this wrt older usage extraction method are: + 1. For every lineage/operation workunit, corresponding query id is also present + 2. Operation aspect for a particular query is emitted at max once(last occurence) for a day + 3. "DROP" operation accounts for usage here + + """ + def __init__( self, connection: Client, @@ -342,8 +346,10 @@ def _parse_audit_log_row(self, row: BigQueryJob) -> ObservedQuery: # Also _ at start considers this as temp dataset as per `temp_table_dataset_prefix` config TEMP_TABLE_QUALIFIER = "_SESSION" + query = _extract_query_text(row) + entry = ObservedQuery( - query=row["query"], + query=query, session_id=row["session_id"], timestamp=row["creation_time"], user=( @@ -354,11 +360,41 @@ def _parse_audit_log_row(self, row: BigQueryJob) -> ObservedQuery: default_db=row["project_id"], default_schema=TEMP_TABLE_QUALIFIER, query_hash=row["query_hash"], + extra_info={ + "job_id": row["job_id"], + "statement_type": row["statement_type"], + "destination_table": row["destination_table"], + "referenced_tables": row["referenced_tables"], + }, ) return entry +def _extract_query_text(row: BigQueryJob) -> str: + # We wrap select statements in a CTE to make them parseable as DML statement. + # This is a workaround to support the case where the user runs a query and inserts the result into a table. + # NOTE This will result in showing modified query instead of original query in DataHub UI + # Alternatively, this support needs to be added more natively in aggregator.add_observed_query + if ( + row["statement_type"] == "SELECT" + and row["destination_table"] + and not row["destination_table"]["table_id"].startswith("anon") + ): + table_name = BigqueryTableIdentifier( + row["destination_table"]["project_id"], + row["destination_table"]["dataset_id"], + row["destination_table"]["table_id"], + ).raw_table_name() + query = f"""CREATE TABLE `{table_name}` AS + ( + {row["query"]} + )""" + else: + query = row["query"] + return query + + def _build_enriched_query_log_query( project_id: str, region: str, @@ -395,9 +431,8 @@ def _build_enriched_query_log_query( # NOTE the use of partition column creation_time as timestamp here. # Currently, only required columns are fetched. There are more columns such as - # total_slot_ms, statement_type, job_type, destination_table, referenced_tables, - # total_bytes_billed, dml_statistics(inserted_row_count, etc) that may be fetched - # as required in future. Refer below link for list of all columns + # total_slot_ms, job_type, total_bytes_billed, dml_statistics(inserted_row_count, etc) + # that may be fetched as required in future. Refer below link for list of all columns # https://cloud.google.com/bigquery/docs/information-schema-jobs#schema return f"""\ SELECT @@ -407,7 +442,10 @@ def _build_enriched_query_log_query( user_email, query, session_info.session_id as session_id, - query_info.query_hashes.normalized_literals as query_hash + query_info.query_hashes.normalized_literals as query_hash, + statement_type, + destination_table, + referenced_tables FROM `{project_id}`.`{region}`.INFORMATION_SCHEMA.JOBS WHERE diff --git a/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py b/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py index a2dbef538f73c..2b7aae8330905 100644 --- a/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py +++ b/metadata-ingestion/src/datahub/ingestion/source/usage/usage_common.py @@ -83,9 +83,11 @@ def make_usage_workunit( budget_per_query: int = int(queries_character_limit / top_n_queries) top_sql_queries = [ trim_query( - format_sql_query(query, keyword_case="upper", reindent_aligned=True) - if format_sql_queries - else query, + ( + format_sql_query(query, keyword_case="upper", reindent_aligned=True) + if format_sql_queries + else query + ), budget_per_query=budget_per_query, query_trimmer_string=query_trimmer_string, ) @@ -149,7 +151,7 @@ def add_read_entry( self.userFreq[user_email] += count if query: - self.queryCount += 1 + self.queryCount += count self.queryFreq[query] += count for column in fields: self.columnFreq[column] += count @@ -295,21 +297,25 @@ def convert_usage_aggregation_class( uniqueUserCount=obj.metrics.uniqueUserCount, totalSqlQueries=obj.metrics.totalSqlQueries, topSqlQueries=obj.metrics.topSqlQueries, - userCounts=[ - DatasetUserUsageCountsClass( - user=u.user, count=u.count, userEmail=u.userEmail - ) - for u in obj.metrics.users - if u.user is not None - ] - if obj.metrics.users - else None, - fieldCounts=[ - DatasetFieldUsageCountsClass(fieldPath=f.fieldName, count=f.count) - for f in obj.metrics.fields - ] - if obj.metrics.fields - else None, + userCounts=( + [ + DatasetUserUsageCountsClass( + user=u.user, count=u.count, userEmail=u.userEmail + ) + for u in obj.metrics.users + if u.user is not None + ] + if obj.metrics.users + else None + ), + fieldCounts=( + [ + DatasetFieldUsageCountsClass(fieldPath=f.fieldName, count=f.count) + for f in obj.metrics.fields + ] + if obj.metrics.fields + else None + ), ) return MetadataChangeProposalWrapper(entityUrn=obj.resource, aspect=aspect) else: diff --git a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py index 90195929bb658..764c2b42537bb 100644 --- a/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py +++ b/metadata-ingestion/src/datahub/sql_parsing/sql_parsing_aggregator.py @@ -88,6 +88,8 @@ class LoggedQuery: class ObservedQuery(LoggedQuery): query_hash: Optional[str] = None usage_multiplier: int = 1 + # Use this to store addtitional key-value information about query for debugging + extra_info: Optional[dict] = None @dataclasses.dataclass diff --git a/metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite b/metadata-ingestion/tests/integration/bigquery_v2/audit_log.sqlite deleted file mode 100644 index e37437ccd73717e15b6bdb0b8a5c7b2a18d469d6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 139264 zcmeHw34B~vdGFk-ZDur!9g~eD&W%Z8$+4BYFNvL4w&mEdEjyCD#dgMbwIKW=GHFZJj-PnJry=SI&6V5q)6HX4Y-pzHY{kYMy9cPB&kCI%ER8{0{I;Z9Jz16wM>%h*_8iYVrhO=2GeDEA^)4ang3<}w*2mV zSNDZYx%F`Z_a--zbgN}{9oihm493QkMnQOzajtT{Db*N@?-h?^2Pjc{s8%` z_!eJ*_zJ{VAie_e6^O4udS2>LPMU{JSpQ-G}{Sy42c5ANOglC#6*Qb+TNc?^B(}~n;lb_CiG5P)M8?tWp zzHBL5%pT0{&UPoiks8f*Wmji~?27DF*-NtdY;*G2%zx)Ullf)lh0OOe-^e_h`E2H6 znGa;%o%xge$1?vT^QO$P%&Rj;GJ_c_voF(=*_{7iW^G2vbY!l{T%K8yNoT0m7h8Xt zf2#FIt>13_YU?wtpKSfh)~8$F+4^|vTT+Kx$6G6{_qN{E+TU7e-PO9Sbwlfo`FFJP zt?jK$>&30P)~57t)4xdnYx;Zn$J5WHzm)z=`lIRhr~fSdzte9^KbC$Z|NH6t(_`s@ zw3)s=eOr1{`sTEpUX}mP{G;is)0d?$NTiYxodMva~I{moy(;fa?RP_Wq+0ZN%Ft4Kgxb5`}O?SvtLa9BKw)_$Fd*H zK9zhS`;P46+25Zg!}wu*1>!3ZUxD}v#8=?_z5>k+d77i%k$NoAm~2==bDT$ke}kBS-z1jIvDqvo~_o_sJ;9)VIBn zqn;z9C{thYMvnR-83md8yf<>x-;j})slWC{j`}Myax(P+Z{(<_$#1Pt?`xT>PJbfPMSmjE{r*Ix5BU>;cKiuX+x~>3Eq}t&rZ-VF+VCeTt@{&& zKIl(my5LVF`VN00(g*yBK=1b_JiX7KaP;l|gr)a-6Gfx<_!E`h?N1bXmp_r|o&H3k z`}~PW_xclo?(ru)eVaew=pFuqrMG(%S);rCiArztCknmQpUCtUe`w%G zlRx3KXQRsF4M5a6aiA1mUCn9~bKN0Ag{0UF5@h2R; z+MlrWjow7m=o|crN^AZ^p;dn((~3WlXxX2LwB%0&TJ$G8E%+0T=KTpvbKXSIXx5*o z^!5Hkp*#GEOt11M61~!&i1c;-M4(sr6P|AOCmh}8Pgr`nH{mtn!PwD&69Z3f=6D65Z&HBHiGP0!@1(PgCB=QvZ*Psz&|J8&&GRy-}fl>x~li8*db; z|MW(IdeIws>et@LQvX3lMWcS{jVkpEZ&awCd!t1C%o|1Or`{+~Kk-JM`ZsT6sTas7 zYt+Abqe}gYH!9Tg-Y8K&@U(6AH0rzFs8ave8x`t1-Y8Mu z@L17`YSh=gQKi1-jSBTuZQUy_m2s1JFgO8td5D%1zPQKH`OjUx4byiuV3+#7l7 zecs4Y?_d8a zxO99xyWJSIOGj+$wlTYO&+#`OZ-4OktB$vgH{Inu9+yY$0oyDy1P$Soh6fqlEZsA( z3V4b`-p39OAGQl+d$e4zjE=EU{gD6Y_;`BM9vv+X4;6~m@$T_dxj2Xy>4PK3$CLQ| zZ1T8z`sU;7(=E@pT$s%9EnC>*{i*RJdC9%QLpFXXJoG-nvIojK9&bsIe~%pRIsOp- zKaBt5jbr#bj*qvEm4dUi9v-wy1nf>~x zU9gYXLuF?7rmmhYhNu`coY~yVbno1{b?$24NVW7P=C1bF2B~oA`3rpzpHmze zC}Q1hp|Ian_Ey>&^N)z(?MAPx!BkBXV9n3YNSHP6^aZOA4v#^~s~L9khxG`z;eS0H zap5Ks4Zl3sH;wrBi?2X@1?KMxxXUZc8}r9@@Hm7orv1;IU-ROXP{j8*5iou7SW- zvk=%DLJ`>734wL)x~0uGr_RvEu0ueH{ECRI$v|s5n;A|tH~f1d^`6{SnFm^a+k6Ec z%;}bVVoqZelgseW{J=E*-iJKqz3OMC^+`W0~7D%UqIKDL7+TXJk}1;huN zsMKelkIk)pl@5ejR1K=~Oem-K{Wa$EG~27@XJ=%F)x6WU$nF}`hswqBJ@XW7X`&kc z;B(PXkifIQvZ}I*SkvnHU`;nAa%)E-F0hAgs;p?tzwUks0(&Y9PkGPfSmiZ~XJSbX z=*Ybc4qHX12v?ZoOln>=eQ~bhJ4kv6o^nASEsXZ-C5ya`RL7pC>%2@iOrGV1Wi{VJ z-1pAz4b046hq-AD!znx`%DgPHEGwVx`&Rq!lSGZ3`M#~2w{7lYq{+{!%Ykos_jA$E zzwx!aDT48t}%HkS5#IK2T_4aFE|IWDUtmR6AhUM9}@d2*AW{)j%|R6eDY)xxsj&7b1>rK z&0;ks6Hg{`cVzfF9m75wDKfIqOXWxPk{k0^r$0BlA1aBNm zD{COJ)jUbAFSxC}C6Rd~8S%AW*(KcWSJ9@ zEme8WCuO@3Xemq#F2>XlVRw{o{Q!u!{0F%eOXqE zhd-m-zO3Qk638(m2sLVc+Q-+l_s!?=MdhB02pDL5ttqmKtV20j+14DWqoFyL&8w26 zNUEXBiotP`ggX|Hd`IC;2b2$*qjQdKT8m0I2>O`ITCJuA(V8&>t>`gV8WA?q159Lw~PA976BR=kj9y1F;CGbn2HjEFK!RGrc~L%>Ok_SE^`I6a0_#ac zK!`JGK9|cRKBI+7a!*EmJ#v+G#9-n{ChdGMm|Ccoe~$W+gh|v%Od_77v|^Jez$9v+ zng@N1nP^BlM8l}Ew6fKU6%o&8!NrPFiQESw zAI$HlZ1G}6=S6WiBvyDj8mlt!B*&q7D>*a;;%uA`(S#(Gw4xqOKnd1nVqOcec|F6t zhJ4o?jQFBN1Yi@100=Q8MhX}yT_swGj0Bzmh^guKeFw6+R?Uh@+ z90b9w#i+z8RAQ29di#!O=trQ{;Nk!2D^K|Uf7g6n;}$%OZ}Al%R{&1^PS2^2vmLWI z^=YcP9GOsYcmR<7l|Ew2io^kVCKUW%Wa;A4R73A$5kD#rwCN>58{(tobMK%9zL~!4;`v1wq zTNAnN%-hl*Zh5ik3m*RetuM9vivoO$m@+b{JXr7r_?blR$5D?Y!96gEdmt}J{68f0 z-Kb-IU~C%1*pvyZCN?%}jLlX$bw@*N@LGfS|pm18Q@HJz??Ls^99ZXq5a_5Xe{brp>wY4U)&#aBO#C-n0 z<%#T?)=QK7o8Q^^dGP-&f2qCybME`$-0mmNtwb^_3oiEsMgF6xBkjOGvk%t5QHf+y zUI-=oykOa9(hUvo>5uqEhEn<4J!_Bz3}b7s${I{lIX3c!1y|YYS%W0Fcs_vJ(^Tfq z6A@pMV6)qU7(+=S@y3|VZlZCX2YdF}>`22oZVrXdCwbpIMi-!ZeGxHhLu9i9UFU8D zQ`br)m>Sb{PNQFej?>p9b}uE_P%-D=F*q` zEE*?li0BUA|9^cV`^DC;roP$o{-$>|{2ulCbI$)hAATXXp;E$$)2xt)=g&j&U`?}R zqt1>QE-@v0WI#7<#%s#Uj1CSTwhLt&U4AX2W1u)>6CzZnG(1=^(8ib1N13)!d%!l! z;7=t_PkRS>k>B69{rGr#)E*rz4i6QI*75G~6zLW`TGj_gj*lnN71%#Xs-}~GzYLe$ z^P#A$NLv?GMYF-CtD^U@ zA?TcBaw;crvfwC=Da)Gd7=~!8Hm~p!FUpppo0_1jiflNdBH|4!I`k=|r81|^q82Tc z5vWB@ArZ)5g|a&w3Q~*Hknc&fqDCn?=4Xdk)%5WvqoNA7EhxIo>k7w#1l}}QO*A2B znR7(YX{($g^R|W;2w;SVW+yGe`7+F&iht`kg&JUEO_!zRlaZdiy%J?PzOXNm#aV-!}PNp?BxHlfQCv zcVAb}uFkC>>fW?{XAgNV$JVo9T-SW98Wk;31Vez}6RR({$=5#+l{L=+Ot0evP*_7TSsm>}6xaz8RyyW_ zXtJ^?vv>uEb|R8&S-T|hW!eOxLy&;$?#~6AAfoqz%-SZ z+62gMFmwGgHD9r#v!`!!-{$Sz%(}hIiS1+Z6?J$U%d@S|MMYz5N!K_C1UOHfqFZ~$3j9Rm)CB`KOIahe^dF~0wQb0YiI%$4cRmixi~ z|HstL=YzXFFMp%EzjC;7@oQCuM2WYAvfw^4sq3WEP4K=uHcDLfsy(;nedw4nxdXEt zcHm{BqSvFrX^66Hvzlhx4r^&Tr|P_Ai6)jXmMo_c-djM@85^*dzjw7p3U92tn`oj`LN`!X?WrJsEA+TL~P2Ismr?U zm^xHIu|-amSyMN8%|uYhbQDpt4U-r_5+~_88Y$@3~&`r-qBn1_SmjnvGpCpiE67r2v_(3Rq4$DT6Cl3{J4wBm`o0dv5-#3p5q)c2M zJZ5wNcXbF`6izy*aE!aE!d*c>-pNGc2n*bxfWmhO?n)+6=koz~1);5oy3kMuXd?=+ zLniUzxJ5}oi;@i0`0~g&7@?7N`TT$1NaPl0hSMKSzNvYj@lpC4^Wq!F9}-`Im+lHw zI5$B~F715jA(5q8oMTZpH*T-Ini!KB38%%zWR)?QjZVpLpfTxQRk?1|{ZXfI9gP~3z?8MyN zZF%6sk+&n(rjGBPPF9wAI%6(mhVi9DKyi>$@*njy2C!n2xe@jS;WhKR@) zfG{Fb5)zNxrD%wj= zXA8dEIGe~8-x>{>2Oa^aqrDFjTdgig4p~TOIU98e9j@F96Kt1CB0k>?CFQWiDh-b? zGmq$^!`MfQqvg>vIIQb9ZWHNON9VW|Wkr?@gVO;kbwpj3Bvw!aYgRB;$n*ab4<@qTY<*|yzLxH$tLcBi!?SmDt;#*fpl?!1 zu&Or{qwsHPG6$!Z?({4{-zfCVLw(5d;m2pfV9mDm1m5V(=>g$ zC3<9DGU+!oh&!UOE5TNAA<(~hmOiUu3gQB+n(Ivr_r3DA1Z zCip32NNq2Ea<)OLE(BA7d!hhxVF6X?C>#&iy(Frt zVd~gOHAOKEfdfikaCFDfcw4lBK^X3}mAj!jTUCOmyf75idALT}6RLAUs1r3hI39@@+V(9_tP|^=29Pz$|l1DJBM8aU}`7|b`Q=3MxU2; zOJoHA^$nflbr#4=2Vi_jaDdR4Bomc8z+@`ed*oo}wY_o#n$n|^pyMT>Xv%9Rg+)@} zGdCGOgi!;)F7O>y$JuZhF$+|A)qKvW_LnJLnU`%v&R-UpZ3X;wh9rwDZ=se$7C1pN zB?l;M&K4b6&^TFPMSw0XRQ3pxL=f7#XxgNf2M8a9HG`FWRwW^Q< z>x|T7n*A~_x*}!{KOL2g1#mvy7DY!=HQm-!K>8ikfL0j1Dq182X6nFv36kRotYJB- zsLYugx$}Go38`D3kBXL{R8&wLw29StNwq8kAY)Ur6h$ERh$4vC zT{P5@I=X_*#8fTR3h{!1f-!W+;5mJE|G$a8E0J~5f0z7J({W7WTYLq=y8;hi>9#g5 z8DGt@#GA@C_}?_X0?`w_A9%viy9NrV7aA=N9jcxe^h1YA_94AITsl6U-EN>d^@wfV zHfERZLC2u>2amt%c-wf>UEbq8_aSUqwKc8p=|Vb?kRJ4O?bzD6zKiMWT(`B0Ie5q% zSp{Z-L-?*$d`GR#EKV3gcNfe31(dRlFB zdNoQMvjJZ@W&bctbs1CD@5V4ao!z%|71r%szoo0M`r}qIw{&%P_26R)ojpCBd)rvA zrk32bSAQ_WY~Ml3qk5UPn$Kkz%SNGpafH<3F>9D*8>$`-JIrSPZ(@4L-^5J*O}6E` zo6Dvbdz9?E#)_q?zrNrU2arX@NB8yY>}tPp^4sU`MP?UeelId!q^?S`Vmh{|7$$4- zChD_b{PD7^W8(!Uge5sB$ikM)BSDJ)hGYQt@0bdzzfsV~1)kgwU+1Q=MEW?ExRIBJ zvP2#^=@OZ-MkZxSoXs;?FNaY3>D7RO{G(>Ul(B|Jb<_Ldi8WKL&VlQsF!>X*v}%5_ zxh{Z-U(%~ZK8+PHxkd{1(UD?F_x=V;zh@LnKzM1E3w_{dR7@Dbu@%igNEddCVZeyR zszKGFWIGOmx0-|c8KgAe9l_{DMni+Gg92z6!j{6Sx;Tq()a<4jmmEt2eSzij%TQ8Q zc6Hs|Db~Sgzh1Hk*&m7Gas^U@z#=etaIzFSkS?K)jWbIX9*c@5;DV^&Ffeq(ktBFE zk|>}69zi!vkp$ZYfULIyPxt#hKG6IK>bEx_kip{lHCJGv|ysNplg z$6%_8;=uPZWX%*b)nbF|_#V2*ZGmF0)QB(sP$-J|tFROkR>;8nA;=U+~&r`Z+y8%0Ae2r#gJZs)qIMjH*4f{aBQGl z92uBk&l|1zC}d@79#41h7-oW#Pp<+W=qJDjD;eX&lpF^WkOY1uW1i$`q4t}Ix$j{Y zYQCR5OKvruauu@|qtAHxfbEo-JBN!yGe3XkC(Yy?G*13DIJ8rLGOX_@@Wss6pUH1< z@=O_H&iqvJ%j37G`r0cU^O%oL?pm) z=n_)z@Q``V0R^6ISgI)i*8v+}lyQcNV7k#}NwBVVn-J7KsuBF)b)g)je^jHxbF%P+ z4h!gV#HV~8;NU5&u^F)^p=wPP_<1tKnj*xffpiD#ebaQ11VJ`U22mDg*aA`{wxvKz zfZvcERa3!}Wr#TAI9uUGB-dbu$r=zJfuR+*%WVYOEUgiAqZo?*y#0*zr`8&;PQdB_ ze2{GYNNM;^aIc;liqz2Z_GDBxA_0I7@a2hs{Rk$KvjoZko)%t4n#}@&40Mc!p|B+U z00^0c@&t`D9LHvng_9&J*lu#KavQJ_tF{E0SGghQiZ6v2hzc(+&HJw@ep{OVc zi_=O5{~{}ZG>aTt&==d`MfiKHhRCjla|`hage}N(*o9OXY+NcYiwNri84|eLRQ9_x zp>NR$=J1(N0#lKtZ^_cBhazIn1XeRVbhAql`WB6(vSRv{K>8MyWqGk?HzVfrsp(s0 zzHc7sTTn0i(`^j35kJj^cs}jP&Mwy3Lnyd$?BWB;8{9t6VH;##rlExyV34(Kg zk8~Cp8l0FCV)8f#H4{O59`-s1)7=1Ju>jUP0Q3Qf1RPsZf^qqke)nQXy_@5RCH3`C zq<+ghNxcqPFVA+bjml;N0$SCuQL_usk!pd((1Flxg9n8za*pH=H%jkt{Wsp0+aLUIba+0hVjX`a7Y>diz{uJq1McW+%Xmb;$gh?5@bb5CxD7 zivl`VfVdLrH{i^8Rnk}jM6?y#05%?^^eRfXK~!y+h~tPXaL~Zfn@H6nksE9S_q!MR z>zf!!=h*t5lDU_+t3%eWNWVEUPBHTASg`PIc*8`~2?^A}tO>NK%>%H`>kh99Afq-# z0kwo6Gj&iUBWVKQ?r3mg*nDu0iprht%OL9>jw8U8=R#TEo9DW|X9Sn)koHyCu^my_ z&3RpPz`B44nr3RMs)Ax(vlWXIz}p~6A~t(nG&D&U!4r)XJCN%pXUagM1CgQF+$`Sh zu)74(-j8MtkoLDjk#_gINPDsiO&ub?GP~|rR78$SHi+8ns1IZ#PUoWX-z*g@4d>KYL|M^H-@h;iod=qCa!6?zJ& zGT5jQ*RqvhEKS997emw=IF?`*z8s3E*UeSbQq1BWSyrPN!-$cD3(fXzo-~5fn zqnO6Gm+A^UbjV%Qxa9TNUI|{|Yhk3P_N>QBt_j7>DZ?q&W;iFxj)C}9rZ zWaiL|s^-(@Cy{??%NH(*%Dw|2yMgL6fYvP?nR->Wfow%DDtHBGOoVDQUXxf9NrM2N z6I9j)2p)-iL<@j?M~{PGM4WrT%{MN2Ei4j}$v+#NGv-gv?V~7r7}5vknB5mUYCoVb zHd;Ux3XGO{%Jg59`qk&6qF{!FhDRup&@2^Rrv=QnA=x5=je^8GvW|F@gR)5Uy#at(6L(~(gu1DHs~0Ek3qZA7dkFHk9&*uGFv0P+VFQ7+p+Q8vNzqHNMYTP4^{ zsswN^YtGVrai5!oVu>6}0@j;DQLMiTNwI45;*8NGGxW+UZ6NsI`8gObXn5{mRMd-f z_OU^7sfY-x$fCm;7J9s3{}Kd_pu%82R2t-xi(U`pC~hX{$7BqCOHP(_HH#hv0C-N<@L3^1MkFE#yl^*_ffmM1$k&G`NN-{XWZrkAsk54g8DGRrSHG&X41rR~mM z8`ZlLue<4{s;_B3ah0d($MizgZI5Z6&1O$K)n=+6usYBJa&L?CPp7JY{m1+hL9S^?C?OTj^0 z;{Uf$FC=m;nZ@Zlk{@n&qT&L3JtDQ+2<}rR9fVCl3|QH1XOhh$We_C?1{#C76_SHBxYr1IDrkhdV~bN@l-4Y8Oa<-8#~cxkUCm5eef3uII}2?qv`W zAOa+bxZoN3bBWB)9*FoB3>5Cmal!(HOL9=>7z%d^uQjb~wvIILs_bAyWH!E50y}|j zTOf2tm78g)K;Vkt3}h_@#05zE1HS=Y0z}YN;BqAuxqrYmQDDiEzQjP&2S%MLWA3Gp zdOydJbj%Mz*-QK8n(d#hSx+5;zb@Gy5qoKTEiXC%QUhKKRvaKa3?MVnT1S*r9`$S} zUNTjr;eg0hdCpKQz`HD60e3nA;3!m5)nKkEAOHV!B6nq`n7%vty5=`G{vrJ*iKow1 z_qQ_aE{CG-=Y*FGbXHBIGNa;dMvQ#L!MUqw?iGp4BBP=xz6FW84Gol4-2h3YhPpcR zs$hXfVr56r1=P-glO6Q_*q6Za#EF=La7ZMBfd!elNp3`3M z(||`_0>XzoO)Ji?L+n>1_dgL8v4i`HQ%SlLR6`ml+!SDpd54z`0Zm#=6GdRQrh^Gn zMK=xsJvVek!@jR*k_xsrkP`=+phfo;5IbT!^P{h9Jw8cJSo3G{Ms6*J7CBOV^ zRD^Bn7S^u}Y?q~@K3+okkLkd@U?E9c0sDrm+o0+L4~Jzq3b@h$F^2_efVffC9VM7{ zx-#T0gRpPsc!JLPRw%;WcG|+OUE?Pw0jfjdndYNwqatw>3J8v5!mqSt2v|iGkSf5J z01bkvsuqd_u$Xm0MH!IJIf?_z4n89feSj4JBC_D+&AV426odnTz-FEZMdG~^(Yp29 zcXs!+tvJI_9kY8=S5Fr+25M=K&AFH9-nn&a9df6Vua0~G?|{o-onlqT6c)gOl!0&t z-T3-+%kwQ4CUaEFyCY(*1x9Fz|KCi%B9VPhtDed=f3k5H58_*V1*WcmyUASzz+(;- zgV?#{Pymnb^wRAU2R7i*B`}1y4Gs^1)z%|sI6ux*0W zp`AHY8Xg_>-&mcesiiHfEf@2?AMYC*VT^nHFY%rl29VDM^K_MgDSIE`H+MII6=^F^Q9FW4)9dZPtY+FK6n0QQ z+BT)3r(hDP)w}k(R4TE2QB+h4MkM?L&_<{liA#f)7McR0W!r>{XzP+Gpk@uN_+-Nd z4I7A;foW5DP>Fzs0MVjZnAkSC*ZJEMF)ZWl36NCMc%f(=?;cMPg9Y%?!I9(RNur^i z0aG=dvSr#54bMkKm%`khR`}Z!F?Zwbsa)6gw4A!4Eh;J%=JwR?Z%+hu8E;Q>+@7v& zFe3xj4s&~I^S390evG%L`nIQQn%Z`p_uG??|L3Os|9@`CP~4RB7tf|$fy!=IL_};Q zClV9k)1kyMoYe4(4@G4ypr^ZSh$gs+KvYHePmqED z^&qlQAUGmW15LJ&g+c%uNhtvoSMVZCcp6AOqeF%<3q9R#SMWtkj_QRXS|LKrWMPVy z_eD!i&V?dcF5TRJ-mhAOCppi+76~FOHn8VtV6SR=UotAX6sCdA`UW>?#oJST+tZ5lCn6K^4sCn# z{r|Tma_yO(bTXN0daPkJ^|om~Z~XA=UjbA%-sn{~O2n*PaMg|ZMCQj4+0X;AGwu%e z25iSmIgwb4H-(bSes6U$+nWyH+wI!(d_oNm|JdbDv_{3M9^~ zNBlQrDoj+wFQFrcfF^Vb=)6&Eq4N@&LMs}YwQ!QifgM`qY}Q0b+`vA=nU0Qb=8lDO zEDoS8#|o~#a5uUN_MJ2*5|Ce*oiXbK`FS~!cBVXB)(1{MHR37WZ16ts^SoEp0r9&b zF?cX4n#9YT2EI$fbkMN}=zd2=!v`G%t6;&jCGZ!bh!0g98k(&VEkbcQ4{}496mp6x zuLeW>D!s0ZjpS;u0z;EN5z0owPbkrf)7OAH#Hu#E|7TGVD*%{|ETS?;6ctq$(f&cx zP>gFEx}a+;3+bsEdO;A*cu5C^se^hBG>F$h5P<$(!mMJIe6bRk-9iwnoO%A~^Ioi; z|4(y?+_nspdamWy&7F+~9>%x$3Y`23xU1ZCjZ2R0;Ur=wUKWajGCr}rpK+JQskQ*j z#`)?1F!zHaPbZE=2UVf-tR|u=AE0ai{z2j|0MxJAf?$G5Mn>&ECz19*=*A8{5ffFt zKv|#?&;n|k95I2Ae^B;y;@}WDiCC5~WnWbRS)R(>9TC?fFbG!R-L+m_r$o%X^P#R2 zQtN#&8iqr#)DE~eV^ON`8X>i3LNU@K+nPL2CBGJteLGOV?gsZJ`18%2L|}@1D97}l zc>eOVIN~H3V$qrk0g}=a&U-Zp_jwZ%@;gPerzS{Uwhy5$yQCvpU6?GP@|*v7b=#94 zTJ;O8;p&JB<*9}*JQ*R^5t1e zM3WUmL|ZUZLq!(a11ht`h1%UUF!y$F62U~qjI05StWcKwk3~gU!eeB;-d*j{eM$sn zx!~zO^NH-05sCK&o)dolzc-Ox*Lp>AZ}XnUcQrIqy(f45;%86m3V`%>v$qOlVp}cv zRj??L`NR{^*!luj0l*fUJg|jKtfd79wt$H4i%6+cu!yd6Z-I#VKsE#sy)6`vW$Wy5 zr+u@frfd1Z*0Klch#NIzk3ARl74<52BOJ`5Af<)4UI;~8k59TDZM{3UwXN^$?IIjb z-HgxF&aCU|+uenSEJJ#Ca4geDe#kLhTk%4c>FVCl&aA-mYzN!UT+d9hMxj~gh+Q(l z-cBm3ibKBFk)Ci}s+6E(($_c%^ZSkUC;;_r)D-~TX1cs?Gcv)iJr#xny61AJ)ui7? z9Lfp)FAuj;)83`#RnrOl0W*Pv3d(*U5zvaWADtIZ^sntG6#6>XZSCqU6ly-grwa2O z)S|U!nQoYb_H|j!i^y*j&^6|!H4LZlNVW?C+BC7s>Ar8Z|2_$bqnYoU$8Ixun%W+j zoM&j=X56dX4RC(fauNafA)N2s=Cv2!_yaYfpHSe^@ZANE*r?`*)Ae@R_H%lwy1ceO z#qxT#DLV1Zk4NL28egk|qF9j;6|rT3Lwi|pmFnoJhMtX_44Na5(}1}eWuibhNT!23 z!$uJ)h!C-^WlIj`pLys8cRe<`64E13k|)DZl06sA&qg;Vg;`dsFlV&IAu2^Eg2@iY z*QoL=TVw4OH|>4%y2T~_;;m89UQ=WhpuNDqCEFUHIw)jzEO2m2=<@&qW?4bM4hd|_ z91Dg{M?pC>Ab|kxa1+>_!0eo#|6iZTex)^^+S_t()7u(;M6W+BKR13+dMuCwd2u4&5%p{kLQ>nkkd#7<@dY1}TAaxKQPcyol>=_Km!Xx3?fFb74s>KQ zw2SD>Hx@<$iS?Q^$zxEPt(jWI5!3Kz48YGKr=9B87%8wOe{@H}40>6$Hp+DevWE6}Aw6Wv=8wc5eS z1d$g*HUc0Ui>L^NI75SMRQdluPp@t{3jhCCvHu@C_GfSf;9TzZol9cKE_mnif`x&1 z*yV?8h;h2$LN*Z5h9{z72h8Rk?)2P4B`%wo)^}$y)%@>!A}(Gy8hxIlp%5c2b~J(< z4UvbVA)UFSq0W5YJUSZCz;Z-1FhoZKr9Zu1h(jSj$MYe?v6yOnW_YQ3EYdMvemR|}bbGW(|DSav$mdS#>I{Or4mu3%qckXMpZLBnO!$9!{ zbGS4@+PvOi^{+CDhwd7)OZTkux&y8veH2#lfbLqhqmK=g3zl(wJX?kNB|pra_~EG8 zZx4b~D_Q96>h0aUy&G@zKCRHNkM4G@%qHp;3(*jNb3bxBeQKO87oY_v%iIgxiz&lecv#VLNkJH^%T3V78s6Y>=K!rYP1*~?pROhie* zf+heFEMVT< z?-|!BG1JZmz$y~RCkX}4A_?bBb_!BLw7?mmB665A^_u4|0z3+U#9 z&qiIq``p_hU=fY(A>bIbQH9#LfJ%+kaX6n7>lb0Uy6i=KJx5|L);I!SvX~YQrW+<-gEobKC94Xp&7YNsSVG!yr6+tn6&J+Krt$dJc>^YLFP4(p`)QUi_BjX|Z@o~XyGnHZ2hP?NqKnwEzb|~_`^~~fw2J9OSCvGSl)(fLf-2*jab;vhO)%h zoY@keCht0=+|~*P<^I%ovJA)f-r*tQ9pPb$JhbeAvW~}F66D__$9s-Hg#QoY|9B$_ zMjjtO`%+eU1yrA^!;22@@VcVOVE<%Qkbf$&#oDqcY6f^gSy0DwyoHVetZr+P0D&XK z1*Yj>Dc|WDka7>F5*zfnP^8>-W>TKoyS>@)P~B_#+RUTRMM>I{u3Czzse-IyC7Ysw zQfS9CRYy<(DP|3h#~No9R+lVY*Ho-{lLbRK$ifX?GgT#6+J>upYg#1;z;j|vPf5Gp zlltn^tu-dRoZ4|VL_%UXX@N9^G2@7ZtZ8zv%1M#9V_ImKEeJX*YPO6z@!9t7qFaER zcW^2}9{xipa^6sfoXdkFb#L8F?yGf#D$l;KRZBoybJgMkfiD_-Yt_*Z?}iU^E$j%8MPJ*)C9vB4EniN8H_Smu$vFu)3{GXq@cASlMLs zQHC_6Ye$2+wuvtp)Q`5E_<8%80T$|MYnSV2_<$NE`$1C_($vloW!|cSU?i#AeB}XJjtWi z7;ou30(po91fDCEA@?wZeLJTSm*88W7!jLKv3;DRec&`%6V5hhH2#eA=o}kUFQYyY zspYK;j?Jko8~_pFJPXlTH2g9kY*Aqmt}|p4QC#>AifpQ)fUqu<0mMQG`ADWFsk0!G zhTJ=Si4*tWoR&COm&6(Rv9(cJ))WwsHCPG#q+y98 zs3#yhgAfp_Xc~l!sDO(7U1e=SLn9Df6KvB|R9Q1PT&hK5qa;!`k+d!%->`)mGdg_# zzbTP@S8GpdtmUmuA8z0n1*uLjfAv6DcA@wtL zoXb+~6MLd0T+6U{&gN}^o*W#-f~J_jLm*s&j@g2tilE|!jcuXAN8&8>X(uKdIl)bm zi50bA5+?}vE?>CB23r`y6`McySS(zB<&OBmC9uYYAzZ#?3mXd;!bSFE*w3C23}c)< zDc6-fVN;2xqoh`$Wlx}1L%v!OoMfz4^{LhMEy>TtYBk~i)0vft?ALG?-{LC}UxD}v z#8)7`0`V1yuRweS;wunef%pncT!G3p?h#PZY~oo0;Cpi@0q1om%L|{i#=DJ?s^|%1 zp>dbHV~tCWZAH-s1a)C30N&S}Oi)#1I<*P>jwx~|X5;oArgv{|U)Q!m-{x&y4BG8( z-^h?@Z(rxO9fjVV>)O`u?CI(1?(<&YJ=@m4lG)td*VVJDb1Pc-ZrZ-Hr+w~uhCH2X zid5;AElIj2OQNHiAP6%6(X^1(M`4yEn3iOtIiO%@Iwt}2uNpk3*}!xu7=YJBZ%aNH Hl>7eyiCzJo diff --git a/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json b/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json index 45b620232cbdd..2f12848ddfc69 100644 --- a/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json +++ b/metadata-ingestion/tests/integration/bigquery_v2/bigquery_queries_mcps_golden.json @@ -13,7 +13,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549585813, + "time": 1724322481569, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", @@ -71,7 +71,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549569547, + "time": 1724322460257, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -156,7 +156,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549567376, + "time": 1724322457731, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -242,7 +242,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549602653, + "time": 1724322505477, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.external_table_us_states,PROD)", @@ -311,7 +311,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549605511, + "time": 1724322508214, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -364,7 +364,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549580688, + "time": 1724322475572, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -394,7 +394,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549598252, + "time": 1724322500148, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", @@ -498,6 +498,103 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.destination_table_of_select_query,PROD)", + "changeType": "UPSERT", + "aspectName": "upstreamLineage", + "aspect": { + "json": { + "upstreams": [ + { + "auditStamp": { + "time": 1724050800000, + "actor": "urn:li:corpuser:_ingestion" + }, + "created": { + "time": 1724322510656, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", + "type": "TRANSFORMED", + "query": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4", + "changeType": "UPSERT", + "aspectName": "queryProperties", + "aspect": { + "json": { + "statement": { + "value": "CREATE TABLE `gcp-staging-2.smoke_test_db_4.destination_table_of_select_query` AS\n (\n SELECT * FROM `gcp-staging.smoke_test_db.base_table`\n )", + "language": "SQL" + }, + "source": "SYSTEM", + "created": { + "time": 1724322510656, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + }, + "lastModified": { + "time": 1724322510656, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test" + } + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4", + "changeType": "UPSERT", + "aspectName": "querySubjects", + "aspect": { + "json": { + "subjects": [ + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)" + }, + { + "entity": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.destination_table_of_select_query,PROD)" + } + ] + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, +{ + "entityType": "query", + "entityUrn": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4", + "changeType": "UPSERT", + "aspectName": "dataPlatformInstance", + "aspect": { + "json": { + "platform": "urn:li:dataPlatform:bigquery" + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "query", "entityUrn": "urn:li:query:49b267d0cd050c6a45b4d26bcdc6d9ddceb51aa7ed29399c52ef967e8da2b58d", @@ -590,7 +687,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549572814, + "time": 1724322465459, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", @@ -787,7 +884,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549588804, + "time": 1724322485618, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", @@ -1043,7 +1140,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549600590, + "time": 1724322502689, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.sharded_table1,PROD)", @@ -1089,7 +1186,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549578210, + "time": 1724322472836, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -1102,7 +1199,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549578210, + "time": 1724322472836, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", @@ -1181,7 +1278,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549588112, + "time": 1724322484293, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -1194,7 +1291,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549588112, + "time": 1724322484293, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.lineage_from_base,PROD)", @@ -1207,7 +1304,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549588112, + "time": 1724322484293, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db_2.table_from_other_db,PROD)", @@ -1335,7 +1432,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549577574, + "time": 1724322471500, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.snapshot_from_table,PROD)", @@ -1381,7 +1478,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549583501, + "time": 1724322478955, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_3.base_table_2,PROD)", @@ -1455,7 +1552,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549571609, + "time": 1724322462741, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -1485,7 +1582,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549572202, + "time": 1724322464098, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.view_from_table,PROD)", @@ -1554,7 +1651,7 @@ "actor": "urn:li:corpuser:_ingestion" }, "created": { - "time": 1721549574990, + "time": 1724322467835, "actor": "urn:li:corpuser:dh-bigquery-smoke-test" }, "dataset": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging.smoke_test_db.base_table,PROD)", @@ -1715,7 +1812,7 @@ "aspectName": "datasetUsageStatistics", "aspect": { "json": { - "timestampMillis": 1721520000000, + "timestampMillis": 1724284800000, "eventGranularity": { "unit": "DAY", "multiple": 1 @@ -1725,25 +1822,26 @@ "type": "FULL_TABLE" }, "uniqueUserCount": 2, - "totalSqlQueries": 12, + "totalSqlQueries": 13, "topSqlQueries": [ - "create or replace table _smoke_test_db_tmp_tables.tmp_table as (select * from smoke_test_db.base_table)", + "create or replace table smoke_test_db.lineage_from_base as (select * from smoke_test_db.base_table)", + "\n create or replace table smoke_test_db_2.table_from_other_db\n as (select * from smoke_test_db.base_table)\n ", "\n create or replace table smoke_test_db.table_from_view_and_table\n as (select b.date_utc, v.revenue from smoke_test_db.base_table b, smoke_test_db.view_from_table v)\n ", - "\n create or replace view smoke_test_db.view_from_table\n as (select * from smoke_test_db.base_table)\n ", - "create MATERIALIZED VIEW smoke_test_db.materialized_view_from_table as (select * from smoke_test_db.base_table where revenue>100)", - "select revenue from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", - "select revenue, date_utc from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table` as (\n SELECT *\n FROM `gcp-staging.smoke_test_db.base_table`\n FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)\n );\n ", - "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", "select revenue, date_utc from gcp-staging.smoke_test_db.base_table", + "\n create or replace view smoke_test_db.view_from_table\n as (select * from smoke_test_db.base_table)\n ", + "create or replace table _smoke_test_db_tmp_tables.tmp_table as (select * from smoke_test_db.base_table)", + "select revenue from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", "select revenue from gcp-staging.smoke_test_db.base_table", - "\n create or replace table smoke_test_db_2.table_from_other_db\n as (select * from smoke_test_db.base_table)\n ", - "create or replace table smoke_test_db.lineage_from_base as (select * from smoke_test_db.base_table)" + "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", + "create MATERIALIZED VIEW smoke_test_db.materialized_view_from_table as (select * from smoke_test_db.base_table where revenue>100)", + "CREATE TABLE `gcp-staging-2.smoke_test_db_4.destination_table_of_select_query` AS\n (\n SELECT * FROM `gcp-staging.smoke_test_db.base_table`\n )", + "select revenue, date_utc from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)" ], "userCounts": [ { "user": "urn:li:corpuser:dh-bigquery-smoke-test", - "count": 10 + "count": 11 }, { "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", @@ -2331,7 +2429,7 @@ "aspectName": "datasetUsageStatistics", "aspect": { "json": { - "timestampMillis": 1721520000000, + "timestampMillis": 1724284800000, "eventGranularity": { "unit": "DAY", "multiple": 1 @@ -2341,13 +2439,13 @@ "type": "FULL_TABLE" }, "uniqueUserCount": 2, - "totalSqlQueries": 5, + "totalSqlQueries": 15, "topSqlQueries": [ "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) = (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", - "select revenue from gcp-staging.smoke_test_db.partition_test", - "select revenue, date_utc from gcp-staging.smoke_test_db.partition_test" + "select revenue, date_utc from gcp-staging.smoke_test_db.partition_test", + "select revenue from gcp-staging.smoke_test_db.partition_test" ], "userCounts": [ { @@ -2372,8 +2470,8 @@ } }, "systemMetadata": { - "lastObserved": 1724065178266, - "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", "lastRunId": "no-run-id-provided" } }, @@ -3016,6 +3114,32 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.destination_table_of_select_query,PROD)", + "changeType": "UPSERT", + "aspectName": "operation", + "aspect": { + "json": { + "timestampMillis": 1724050800000, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "actor": "urn:li:corpuser:dh-bigquery-smoke-test", + "operationType": "CREATE", + "customProperties": { + "query_urn": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4" + }, + "lastUpdatedTimestamp": 1724322510656 + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "dataset", "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.table_with_nested_fields,PROD)", @@ -3156,7 +3280,7 @@ "aspectName": "datasetUsageStatistics", "aspect": { "json": { - "timestampMillis": 1721520000000, + "timestampMillis": 1724284800000, "eventGranularity": { "unit": "DAY", "multiple": 1 @@ -3166,11 +3290,11 @@ "type": "FULL_TABLE" }, "uniqueUserCount": 2, - "totalSqlQueries": 4, + "totalSqlQueries": 11, "topSqlQueries": [ "select * from gcp-staging.smoke_test_db.usage_test", - "select key from gcp-staging.smoke_test_db.usage_test", "select value from gcp-staging.smoke_test_db.usage_test", + "select key from gcp-staging.smoke_test_db.usage_test", "\n select extract(month from date_utc) month, count(*) count, sum(value) total from gcp-staging.smoke_test_db.usage_test\n group by month\n order by total\n " ], "userCounts": [ @@ -3200,8 +3324,8 @@ } }, "systemMetadata": { - "lastObserved": 1724065178257, - "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", "lastRunId": "no-run-id-provided" } }, @@ -3517,7 +3641,7 @@ "aspectName": "datasetUsageStatistics", "aspect": { "json": { - "timestampMillis": 1721520000000, + "timestampMillis": 1724284800000, "eventGranularity": { "unit": "DAY", "multiple": 1 @@ -3527,31 +3651,31 @@ "type": "FULL_TABLE" }, "uniqueUserCount": 1, - "totalSqlQueries": 5, + "totalSqlQueries": 79, "topSqlQueries": [ - "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", - "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", "SELECT\n (SUM(CASE WHEN amount BETWEEN 0 AND 10 THEN 1 ELSE 0 END) * 100.0) / COUNT(*) AS percentage_in_range\nFROM\n gcp-staging.customer_demo.purchase_event;" ], "userCounts": [ { "user": "urn:li:corpuser:dh-bigquery-smoke-test", - "count": 78 + "count": 79 } ], "fieldCounts": [ { "fieldPath": "amount", - "count": 18 + "count": 24 } ] } }, "systemMetadata": { - "lastObserved": 1724065178270, - "runId": "bigquery-queries-2024_08_19-16_29_19", + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", "lastRunId": "no-run-id-provided" } }, @@ -4704,6 +4828,22 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:bigquery,gcp-staging-2.smoke_test_db_4.destination_table_of_select_query,PROD)", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "query", "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", @@ -5132,6 +5272,22 @@ "lastRunId": "no-run-id-provided" } }, +{ + "entityType": "query", + "entityUrn": "urn:li:query:44c19e1fccfa56779f6958f62d3476819b48af701bbee43ccecf5c9e04d63fc4", + "changeType": "UPSERT", + "aspectName": "status", + "aspect": { + "json": { + "removed": false + } + }, + "systemMetadata": { + "lastObserved": 1724050800000, + "runId": "bigquery-queries-2024_08_19-07_00_00", + "lastRunId": "no-run-id-provided" + } +}, { "entityType": "query", "entityUrn": "urn:li:query:f53151fa3d689ec865c53fa535479e3fc3cc5794be5daaa9a5a4f7f40a7c660f", diff --git a/metadata-ingestion/tests/integration/bigquery_v2/query_log.json b/metadata-ingestion/tests/integration/bigquery_v2/query_log.json new file mode 100644 index 0000000000000..23513e46eb5fe --- /dev/null +++ b/metadata-ingestion/tests/integration/bigquery_v2/query_log.json @@ -0,0 +1,5404 @@ +[ + { + "query": "create or replace table smoke_test_db.usage_test (date_utc date, key STRING, value INTEGER)", + "session_id": null, + "timestamp": "2024-08-22 10:27:23.887000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "714133dca6aefca9787e41860cac29c12fcf745ebc2022515aafa4a1ac275562", + "usage_multiplier": 1, + "extra_info": { + "job_id": "15e12d10-ecd2-49df-9265-15e1d13063f2", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + }, + "referenced_tables": [] + } + }, + { + "query": "\n insert into smoke_test_db.usage_test values\n (\"2022-05-01\", \"seven\", 7),\n (\"2022-05-02\", \"ten\", 10),\n (\"2022-06-01\", \"four\", 4)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:25.357000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9a0a288a8fb978ca88c0bccb466e099eef0d9a18a2dd4c8b184319820047dd17", + "usage_multiplier": 1, + "extra_info": { + "job_id": "74e32b90-4099-463d-94c8-d7dee843cfa4", + "statement_type": "INSERT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "\n create or replace table smoke_test_db.partition_test (date_utc date, revenue INTEGER) \n PARTITION BY date_utc\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:27.620000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "e5a917a8c68b133b30f7acf5c1a4d843cf00132d3785bf0320fba2ad579c519e", + "usage_multiplier": 1, + "extra_info": { + "job_id": "194dff44-329f-49e6-82f0-865c06c223ba", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + }, + "referenced_tables": [] + } + }, + { + "query": "insert into smoke_test_db.partition_test values (\"2022-05-24\", 20), (\"2022-06-24\", 30)", + "session_id": null, + "timestamp": "2024-08-22 10:27:28.864000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "1442aa8170b7c0a8ae76695175a8a34c06c602aa116656ddd29746f348478c95", + "usage_multiplier": 1, + "extra_info": { + "job_id": "56e06ba7-a054-4100-a0e3-d7d5c7a32d7e", + "statement_type": "INSERT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "create or replace table smoke_test_db.base_table (date_utc timestamp, revenue INTEGER)", + "session_id": null, + "timestamp": "2024-08-22 10:27:31.326000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "48c41f9be14b4816933e27350f75d291a87194168b202c835d3fdb0e82e8cc1a", + "usage_multiplier": 1, + "extra_info": { + "job_id": "d7ef294f-8776-4d12-adc8-dd9de5acda16", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + "referenced_tables": [] + } + }, + { + "query": "insert into smoke_test_db.base_table values (CURRENT_TIMESTAMP(), 100), (TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR), 110)", + "session_id": null, + "timestamp": "2024-08-22 10:27:32.660000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "73b3a547e874aa113a5623257fd0b174217521b34d1d8189e52bbb67bf38b494", + "usage_multiplier": 1, + "extra_info": { + "job_id": "027cb80d-d2f8-4ef7-931b-c2ba70d0acb1", + "statement_type": "INSERT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "create or replace table _smoke_test_db_tmp_tables.tmp_table as (select * from smoke_test_db.base_table)", + "session_id": null, + "timestamp": "2024-08-22 10:27:35.205000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "ee37a62a711cf6cb094c22761f49fa81f62e9a623d6425274a549be1425e3d59", + "usage_multiplier": 1, + "extra_info": { + "job_id": "47449db3-409f-4518-95fb-bbe986ff82cc", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_smoke_test_db_tmp_tables", + "table_id": "tmp_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "create or replace table smoke_test_db.lineage_from_tmp_table as (select * from _smoke_test_db_tmp_tables.tmp_table)", + "session_id": null, + "timestamp": "2024-08-22 10:27:37.731000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a15c1815163f7fc6696fbb4e8e2725246d7ac93a876bf4743a10b22775d3188", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b583ade6-98f9-4323-909e-4b16678f7aaf", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_tmp_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "_smoke_test_db_tmp_tables", + "table_id": "tmp_table" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236058040\n AND last_modified_time <= 1724322458040\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:27:38.597000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "953e6ea0-99b4-43dc-b27a-5eefa8cfb1ca", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon77d4fd91_539c_4f4e_9fee_b78ad8c277ea" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:39.259000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "92088ca0-f8a7-416b-a6c9-dbfc734d59a6", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:27:39.633000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "ef29f16d-9db8-47fc-bc3a-fdc11fdbd138", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:27:40.178000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "c572ae0c-c77d-4f72-9284-be1b1fa8d401", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:40.256000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "0d12200c-aefe-42cf-92b9-c3e728cc5e2c", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "create or replace table smoke_test_db.lineage_from_base as (select * from smoke_test_db.base_table)", + "session_id": null, + "timestamp": "2024-08-22 10:27:40.257000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "ea4879ec408f01bb2e055212f9e42e239a2ab9414230d4e471a41684924f9408", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9ac780d2-ce9a-487e-b5b0-fa50c534600f", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236060301\n AND last_modified_time <= 1724322460301\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:27:40.824000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "60d848ac-1fcd-4def-a81b-c08b0166b5c5", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon5af5d332_c8eb_4525_a078_3c31668e4a6b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:27:41.442000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "9eb7efff-67d8-48e4-a4dd-73e7a72e6ce5", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:27:41.535000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "78284ace-bd28-4a53-a476-cfc86db51919", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonbafa243d_7625_4485_a27d_6ca94e711867" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:27:42.038000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "29cb5356-e155-4f88-b79b-d9d7bb2ff47f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:27:42.191000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "bf6998d0-0984-4d09-80c2-d6167f0bd5be", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:27:42.281000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "4274ea3c-97f4-46eb-95f7-57f801b9643f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anona67fc478_bd01_4230_9fbd_ec017a318330" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n create or replace view smoke_test_db.view_from_table\n as (select * from smoke_test_db.base_table)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:42.741000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "bf437f8369258ddb78cc9d77370a07344d0910bfa7316c8d5d3237d8134021aa", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3dee2d9b-8a0a-4f85-a953-19bb77d6e5bf", + "statement_type": "CREATE_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "view_from_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:27:43.087000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "ba08fe4a-a50c-4485-a00a-37b6eba7e98a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:43.178000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "3bba8392-5644-4565-a283-01b7ef199138", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32e6bd0458339a10581928d131db191eaea3e29936a3454e7946ed8a12f17e38" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:27:43.300000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "b88614c8-d56f-42fd-8d33-0bbdf082e396", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724322061794\n AND last_modified_time <= 1724322463023\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:27:43.523000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cebf6f8d-bb13-41ef-816d-9292eac62cc5", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anondf074cd5_822f_45ff_aa5a_cb4d4d69b71b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n create or replace view smoke_test_db.view_from_view_on_table\n as (select * from smoke_test_db.view_from_table)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:44.098000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "a3ad3488cc0978b7fad948192fdde83bf669a9fd49ab3c296181e115cbdba713", + "usage_multiplier": 1, + "extra_info": { + "job_id": "d398a69a-f4ba-4e81-b591-871152fb9a03", + "statement_type": "CREATE_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "view_from_view_on_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:27:44.456000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "5bbb3cfd-a62c-4a36-b15f-a207d89e2c62", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:44.791000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "55dabfc7-a89e-47a6-a065-965c6dafb47e", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n create or replace table smoke_test_db.table_from_view\n as (select * from smoke_test_db.view_from_table)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:45.459000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "047ae44f589ae98693f8b355b284d052dcaac235ff30bdf8df5b89316ab1efd0", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0b71e40e-fe16-4f70-8173-db68f89787e3", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:27:45.555000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "dcf62ed3-098b-4e8f-afd0-62297db76b05", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:27:46.821000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "56ee02b5-9e75-4acd-be96-19aee59af431", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon14ea5eb6_003f_40a7_ad6f_44dd1bad7026" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n create or replace table smoke_test_db_2.table_from_other_db\n as (select * from smoke_test_db.base_table)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:47.835000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c4b56402d62a63135c5f0e81e4f6391670485bd46d355fdb0a4ce31884a170c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "ea311635-bb0a-4216-a68d-81bc87ec6e89", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "\n create snapshot table if not exists smoke_test_db.snapshot_from_table\n clone smoke_test_db.base_table\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:50.147000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "27ac78089e13c8ae1c71126ad4036cbba12a0656fe1453f23d55c49202a70a06", + "usage_multiplier": 1, + "extra_info": { + "job_id": "bd00bc76-81b9-4a1c-8bc9-f28e333cdae4", + "statement_type": "CREATE_SNAPSHOT_TABLE", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "snapshot_from_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "\n create or replace view smoke_test_db.view_from_snapshot_on_table\n as (select * from smoke_test_db.snapshot_from_table)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:51.500000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "d61201ebb9413c74be7a18d33ef712d7a5488bca765977cb311513faff92e4d7", + "usage_multiplier": 1, + "extra_info": { + "job_id": "12ed5ed0-38c6-428c-99d0-d230b140dd1f", + "statement_type": "CREATE_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "view_from_snapshot_on_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "snapshot_from_table" + } + ] + } + }, + { + "query": "\n create or replace table smoke_test_db.table_from_view_and_table\n as (select b.date_utc, v.revenue from smoke_test_db.base_table b, smoke_test_db.view_from_table v)\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:52.836000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "dd07587f59ed3d65b092bfb66fe8079129f780a0d9c8b530d1ea8ae868ef7c3e", + "usage_multiplier": 1, + "extra_info": { + "job_id": "955a444b-e9ef-44e4-9cc6-2a93e16a8eb0", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view_and_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "drop MATERIALIZED VIEW if exists smoke_test_db.materialized_view_from_table", + "session_id": null, + "timestamp": "2024-08-22 10:27:55.572000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "246765bb1a63ff4a6550373a2dc1f31070f1247be8db662f42a6fcf27073e9da", + "usage_multiplier": 1, + "extra_info": { + "job_id": "script_job_7ca68f4489c9b94df86a90933cfa7886_0", + "statement_type": "DROP_MATERIALIZED_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + "referenced_tables": [] + } + }, + { + "query": "create MATERIALIZED VIEW smoke_test_db.materialized_view_from_table as (select * from smoke_test_db.base_table where revenue>100)", + "session_id": null, + "timestamp": "2024-08-22 10:27:56.091000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2184be1599060a6740d129b60172096d22107b42fc1634bf052db04b4d4f259b", + "usage_multiplier": 1, + "extra_info": { + "job_id": "script_job_034cd994c0ae9fb720f91dabfde3149b_1", + "statement_type": "CREATE_MATERIALIZED_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_3.base_table_2` (date_utc timestamp, revenue INTEGER);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:57.705000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "0e636e8703fba1cb8b4bef2404e090c291e848a02324043c086a918264620640", + "usage_multiplier": 1, + "extra_info": { + "job_id": "dd349313-36b1-4607-914c-94df425773a2", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2" + }, + "referenced_tables": [] + } + }, + { + "query": "\n create or replace table `gcp-staging.smoke_test_db.table_from_another_project` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:27:58.955000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "30362bb4bca3c47a23d588a26faddc8b276bb70e9f38685b4d91bffe0ad955d4", + "usage_multiplier": 1, + "extra_info": { + "job_id": "f4971ee8-f969-4d7c-8744-7b4ce66c29b4", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_another_project" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_3.derived_table` as (select * from `gcp-staging-2.smoke_test_db_3.base_table_2`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:01.569000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "fd4278ccffdd4f56c411be3c09ed78d09776f8985d2b450d1e724dca39869d33", + "usage_multiplier": 1, + "extra_info": { + "job_id": "5a335e2b-0a28-4ac6-9138-c103ad3193bc", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "derived_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2" + } + ] + } + }, + { + "query": "\n create or replace view smoke_test_db.view_from_multiple_tables\n as \n (\n select a.date_utc, a.revenue, b.revenue as revenue2, c.revenue as revenue3 \n from \n smoke_test_db.base_table a\n left join \n smoke_test_db.lineage_from_base b \n on a.date_utc = b.date_utc \n left join \n smoke_test_db_2.table_from_other_db c \n on b.date_utc = c.date_utc \n );\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:04.293000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "fe631cb81007f8ae1cb9cc51e80a57cba7d21f876ebd8c3791708468d0497496", + "usage_multiplier": 1, + "extra_info": { + "job_id": "8fa83df5-e0ca-449e-a11e-4f5789e67e8e", + "statement_type": "CREATE_VIEW", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "view_from_multiple_tables" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`) ;\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:05.618000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "04834ed24134c3a61000b5df49c6b4a99db596b9f7535894922a06d050d4d084", + "usage_multiplier": 1, + "extra_info": { + "job_id": "def72ad2-7f1e-4d39-a2d9-e2e49984dd9a", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230101" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.sharded_table1_20230201` OPTIONS(description=\"Description of sharded table ending with _yyyyMMdd\") as (select * from `gcp-staging.smoke_test_db_2.table_from_other_db`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:08.551000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "23d48cd9268ba14ba5274cc16188f5504efa64b022cc83c877f734b69c398d02", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1647e16c-5a81-4b25-852f-ae08e25c18b4", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230201" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + } + ] + } + }, + { + "query": "insert into `gcp-staging-2.smoke_test_db_4.sharded_table1_20230201` values (CURRENT_TIMESTAMP(), 100)", + "session_id": null, + "timestamp": "2024-08-22 10:28:11.425000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "0238bbbbabf55ab3543b48a129753ee1726ccce34ab30e79d1eb1278f75723dc", + "usage_multiplier": 1, + "extra_info": { + "job_id": "ede91fe9-615f-4877-86b6-65211b1c16d7", + "statement_type": "INSERT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230201" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230201" + } + ] + } + }, + { + "query": "\n CREATE or REPLACE TABLE\n `gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition` (transaction_id INT64)\n PARTITION BY\n _PARTITIONDATE\n OPTIONS (\n description = \"Description of Ingestion time partitioned table\",\n partition_expiration_days = 3,\n require_partition_filter = TRUE);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:15.660000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3fe79bd659d7bb81ebc0f005eff68e9f3dc037c2383b3109369871192d708a40", + "usage_multiplier": 1, + "extra_info": { + "job_id": "07f61a59-999e-4cad-b5d2-e36f3efc3e1c", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_ingestion_time_partition" + }, + "referenced_tables": [] + } + }, + { + "query": "\n CREATE or REPLACE TABLE `gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition` (customer_id INT64, date1 DATE)\n PARTITION BY\n RANGE_BUCKET(customer_id, GENERATE_ARRAY(0, 100, 10))\n OPTIONS (\n description = \"Description of Integer Range partitioned table\",\n require_partition_filter = TRUE);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:17.080000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "a8567cfec7bc0c2c17aae4266a7d8f01d5f2d3fa720a620ab5bafafc79fa2971", + "usage_multiplier": 1, + "extra_info": { + "job_id": "35c66a87-c24d-4a84-aaa7-02ad67c7924e", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_integer_range_partition" + }, + "referenced_tables": [] + } + }, + { + "query": "\n CREATE OR REPLACE TABLE `gcp-staging-2.smoke_test_db_4.table_with_nested_fields` (\n id STRING,\n first_name STRING OPTIONS(description = \"First name\"),\n last_name STRING OPTIONS(description = \"Last name\"),\n dob DATE OPTIONS(description = \"Date of birth\"),\n addresses\n ARRAY<\n STRUCT<\n status STRING,\n address STRING OPTIONS(description = \"Full Address\"),\n city STRING,\n state STRING,\n zip STRING,\n numberOfYears STRING>>\n ) OPTIONS (\n description = 'Example name and addresses table');\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:18.418000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "590b097235695f628515f838f96c71e86b2d763bf480ab07a67700023885b250", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6ce6ff3a-80e3-4766-84cf-d285a86ea704", + "statement_type": "CREATE_TABLE", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_nested_fields" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236098639\n AND last_modified_time <= 1724322498639\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:28:19.276000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "24336324-7714-4ab2-95f4-243f5e54950f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonf4cce75c_4c8c_4ef2_802e_0f6700b6f877" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_sharded_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_20230101`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:20.148000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "466ee2217419c64f1bdcf7d693317f73a6cbef15abb6424711b990a78d5e3273", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0d6a54d2-102a-418f-a8fe-25cfac92abbe", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "derived_table_from_sharded_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230101" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:28:21.530000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "89b073c2-bf2b-49b1-a983-9589de5ec524", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_wildcard_table` as (select * from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:22.689000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "bb9db7b8cca0e55c5fcc8a5ced32d780886c6eb052b022731b33b42f852cc0c9", + "usage_multiplier": 1, + "extra_info": { + "job_id": "2b1929cc-ad64-44f6-9069-e9afbaa3ef2a", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "derived_table_from_wildcard_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_*" + } + ] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:28:23.898000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0a7899cf-c0ac-4e67-8733-ed6374e7f920", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2f4ead4a_d35d_445d_8819_ed92bc8748cf" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:28:24.028000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "4f04684e-202e-4afe-b4df-f813d0925999", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:28:25.181000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "e7014fff-ef9e-4008-88c2-41b5b15c96af", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:28:25.260000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6266127d-3d3b-4393-97f6-0725079c9f0a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonf31236fd_fb12_4e4d_9bdf_47983e7013f6" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_external_table` as (select * from `gcp-staging-2.smoke_test_db_4.external_table_us_states`);\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:25.477000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "bda343fa9d8839d698b5e4c92352f6a2e4f9788c6302a972e1177fe50f8be872", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3f8a74d1-a74b-4757-a1ae-7ee12580be0f", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "derived_table_from_external_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "external_table_us_states" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724322464301\n AND last_modified_time <= 1724322504931\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:28:25.508000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b5fa72b4-4bf8-4ff2-86fc-0e6f762160a7", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon1d72692a_44df_4f8c_9b1c_6aae8a38a887" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:26.413000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "74775f70-cbaa-43b5-8136-a9185f5cec66", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32e6bd0458339a10581928d131db191eaea3e29936a3454e7946ed8a12f17e38" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:28:26.413000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "dcd0881c-4027-4585-ae64-95e2efc891a1", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:28:26.850000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "64aadd1d-2596-4600-9452-f67af261c0ea", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:28:27.362000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "8dfe3671-cfac-4bbf-bf17-325bd9a13aa0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236106863\n AND last_modified_time <= 1724322506863\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:28:27.443000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9b8500cf-1c04-4c1e-8a13-e15577e67641", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonc9084373_367a_4390_af54_16f619797096" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:28:28.030000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "1c014139-8f13-46b9-b440-27259f3b7701", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:28.141000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "a8a83a53-e200-4205-8e30-73d11c149e44", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n create or replace table `gcp-staging-2.smoke_test_db_4.derived_table_from_timetravelled_table` as (\n SELECT *\n FROM `gcp-staging.smoke_test_db.base_table`\n FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)\n );\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:28.214000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "d8fba9ffc7dad8c86eded34dcaf16829629501451411380ec17084a1e2d38432", + "usage_multiplier": 1, + "extra_info": { + "job_id": "e4aa2fb4-56b5-4515-a50d-75ec31794ff2", + "statement_type": "CREATE_TABLE_AS_SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "derived_table_from_timetravelled_table" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table@1724318908218" + } + ] + } + }, + { + "query": "CREATE TABLE `gcp-staging-2.smoke_test_db_4.destination_table_of_select_query` AS\n (\n SELECT * FROM `gcp-staging.smoke_test_db.base_table`\n )", + "session_id": null, + "timestamp": "2024-08-22 10:28:30.656000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "e9226e326a20c90425b04cd42358663c8446312d0d8dadf78b177ab708d92343", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6486765d-2753-477f-9e11-cfcfb6407c8c", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "destination_table_of_select_query" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:31.305000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "d02b0ba9-0c04-47ce-82ae-dd2686458739", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32e6bd0458339a10581928d131db191eaea3e29936a3454e7946ed8a12f17e38" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:28:32.614000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "a71ddace-2595-4b3f-a677-258b46409f65", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone859c132_30c8_476f_8be3_8f7882e8c4d7" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:28:33.038000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "f78b5ce5-713b-4898-b13f-91b6304c2cce", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id=\"purchase_event\"\n AND last_modified_time >= 1692786512601\n AND last_modified_time <= 1724322512601\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:28:33.155000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "c4076a26606e9f6d6b9fde2855758ba67b1155b110a65f72cf1999fa1facdfa4", + "usage_multiplier": 1, + "extra_info": { + "job_id": "87f27258-3890-47d3-97da-75151c67d647", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon6599dcd6_73ba_42e7_b0a3_d9a5985a4ddd" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.base_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:34.004000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "376701d9b1677febe3189c534c475310e79dd893ac7759e42f59433240d3a83b", + "usage_multiplier": 1, + "extra_info": { + "job_id": "139c0fa9-cc67-4ec4-85cf-1b9a231f4179", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonac9eceea_3ec8_4740_905e_eeab3bc9b91d" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236114344\n AND last_modified_time <= 1724322514344\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:28:34.896000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9f864017-5753-4a2e-b2cc-53ac789dd9ea", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonf6b6eab6_7da6_451b_923e_cd310a5df0eb" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:28:36.387000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "a915c600-d661-432c-9236-df37a7c65c1e", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id='purchase_event';", + "session_id": null, + "timestamp": "2024-08-22 10:28:36.511000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "8aed7604ce6319236cca0b0f7d16614774821c2666f5cf6f5be2cec702026d64", + "usage_multiplier": 1, + "extra_info": { + "job_id": "8d362768-c6e9-42ee-bbaf-3b114a32d827", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon3c7a5401_2635_4e79_b452_21ede12dacf2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:28:37.586000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "450d30ee-719e-43c2-b9f0-4ef252080b54", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT\n (SUM(CASE WHEN amount BETWEEN 0 AND 10 THEN 1 ELSE 0 END) * 100.0) / COUNT(*) AS percentage_in_range\nFROM\n gcp-staging.customer_demo.purchase_event;", + "session_id": null, + "timestamp": "2024-08-22 10:28:38.119000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "43486ae9-5d58-47a0-98e0-7a064aa634b5", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anondaaf8a98bb16f9b4fd4e7f6ada08bac86581b8e2a25e55399914c155fd8566b7" + }, + "referenced_tables": [] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.partition_test", + "session_id": null, + "timestamp": "2024-08-22 10:28:38.956000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "43b7b24e1b461dd151b2fa4f3e9ff9d4f03f831dfff20b1d7016686d9ac29e7c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "12049165-b270-4f94-9531-03542fc9f680", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anond6b394e5_9ab5_4d08_a48c_9826e78e05ef" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.lineage_from_tmp_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:42.802000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "a5460497edcdd62d40807fd1810005e02b13b930fe560570e16a6a652854390b", + "usage_multiplier": 1, + "extra_info": { + "job_id": "477cc9dd-ffca-4352-a8ed-ebd1e7c54967", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon00c98d0b_6cbc_4752_8a0b_d0f5bff232fb" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_tmp_table" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.lineage_from_base", + "session_id": null, + "timestamp": "2024-08-22 10:28:45.871000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2a4f4d44119c98882739e7d1360a5544c0a4b95a45198f789ac1c6efdf3d7e10", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6c29d65a-fb33-4805-bd32-4a9e00ceee68", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon37cb5eae_6fb4_4f80_a2b5_dcdf18293e5b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:48.915000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "c039e0e092bcd3150010cddee2a172af283951c43db68caf513593e4f552be28", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6f142a0e-d71c-40f7-9b76-70080d55638f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon0026b3be_d07b_4302_b7ca_fe6c2b3ce150" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_view_on_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:52.005000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3dd24342ec0b26c692285908192e7eb6ad5e38f25816bce80405c57acc109c0f", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6b9755dd-e5e4-4c06-b850-72b02a0c583d", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon264bee95_5a2e_411d_971d_3b15c65475c0" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_view", + "session_id": null, + "timestamp": "2024-08-22 10:28:55.333000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "4c08bebf0f438dcb4fda1eb2cb02e41ff216ee613eb9ffcc8a9e01dffd1f5d85", + "usage_multiplier": 1, + "extra_info": { + "job_id": "978ee15e-fc5f-4464-a288-4a588d10a72b", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon57c90059_bdfe_4f67_a482_8423f135abd5" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_view_and_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:58.350000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "fc92290f2da155cba009370357576469c7262cff1fd4b521399e8b555fb9457e", + "usage_multiplier": 1, + "extra_info": { + "job_id": "762a642a-9d71-4ffd-a0af-f1a6f0843643", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32470674_7f5e_4d81_875c_08afdc7ee994" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view_and_table" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:29:00.852000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "0ce60fad-0590-4b9d-a333-c0651033c162", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:29:01.249000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9e9dcf86-6712-4981-a7a8-7abff832a52b", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4765d310_ebab_49c6_8edb_2a8b5610bc41" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.materialized_view_from_table", + "session_id": null, + "timestamp": "2024-08-22 10:29:01.401000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "24ffb91f53691b2b9082fc4582d5836ec8501613821b7de7dda75b0b7b918881", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cd5b5b95-41ad-4451-b9c8-f7eedc134b22", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2173da01_0669_4d20_9814_155ccaf243a7" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:29:01.448000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "ed85e89b-173a-4acc-8e38-56ac85b501a9", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.view_from_multiple_tables", + "session_id": null, + "timestamp": "2024-08-22 10:29:04.870000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "58cdaed9da9d3cd837048b70d7d48fa485483cb5e85755803feb28c30aefc800", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cb16a84f-23e3-48f1-9cbd-cd6c1ae2ccdb", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone6b9ea5f_d7b2_4d88_bce8_67813e1297be" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db_2.table_from_other_db", + "session_id": null, + "timestamp": "2024-08-22 10:29:08.392000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "8073f5c704e675d8c4383a750c2c88da65437a38e28a1f7f06f1162531d544a9", + "usage_multiplier": 1, + "extra_info": { + "job_id": "13b0be99-04e9-40e9-80fb-3ca54815bcc0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon02c064af_8ff6_41f5_9a59_ad6b45254529" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging-2.smoke_test_db_3.base_table_2", + "session_id": null, + "timestamp": "2024-08-22 10:29:11.240000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "cad2086113da8705258cfb5e0531e75d26aeccc85ddbf774c18121bb8716af7a", + "usage_multiplier": 1, + "extra_info": { + "job_id": "ab261d82-f178-44bc-8f16-135a2b7698f0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon0bf48c7f_6413_4559_9fdd_dc4f0be48c47" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:29:11.898000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "107b0124-7754-4d72-8e50-3ed562afdeb7", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:29:13.829000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "be2fc0f0-13be-44ad-a77c-2d81391f206c", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.table_from_another_project", + "session_id": null, + "timestamp": "2024-08-22 10:29:14.152000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "22c18f1bdf759bab7dcd2f26b3b66cc62ce9a7aa861237a5a790437958bf5316", + "usage_multiplier": 1, + "extra_info": { + "job_id": "573cbacd-aee3-4655-a44c-1d721bcb49ed", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonc7614baf_c661_4d17_aba2_f7b43763a7e2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_another_project" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236154390\n AND last_modified_time <= 1724322554390\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:29:14.876000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "7a627727-a4da-45fa-b504-147ea8f7fcf8", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonf45b3a81_0bc1_413b_b39f_04a62fdf41b7" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging-2.smoke_test_db_4.sharded_table1_20230101", + "session_id": null, + "timestamp": "2024-08-22 10:29:17.063000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "691a01378badc5f66308c9cd67305ff487ab5b1208c9cef4fd0e7b0d12401e72", + "usage_multiplier": 1, + "extra_info": { + "job_id": "a2089528-786c-4cde-b87b-61a88a17d7fa", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone3b99b14_5cf1_4a38_97a5_b435d004fcc8" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230101" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:29:17.577000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "9f167fab-f7db-4cce-82d7-8e97c890d9ec", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:29:18.636000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "3098ddd6-b6a3-45a2-9905-8521bea14486", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32e6bd0458339a10581928d131db191eaea3e29936a3454e7946ed8a12f17e38" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236157888\n AND last_modified_time <= 1724322557888\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:29:18.676000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "91631d91-6208-4f60-9ba7-8cfe67cb4185", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anoncaea99c4_c2ca_465a_984a_07a359657b86" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:29:18.692000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "f93e3148-9c20-4ccb-b7a5-f5608a39a877", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "select revenue, date_utc from gcp-staging-2.smoke_test_db_4.sharded_table1_20230201", + "session_id": null, + "timestamp": "2024-08-22 10:29:20.058000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "f1d54aed9c1d03837b735a53385cb93c1eabff91aa33fa99dfa8d9e7e5ec1ff7", + "usage_multiplier": 1, + "extra_info": { + "job_id": "8e1d105a-52f3-46f9-a4b5-4baa374e4daf", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon01dc59e0_2426_4a26_aa38_abe2a1c214ec" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230201" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236161244\n AND last_modified_time <= 1724322561244\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:29:21.866000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6177dad2-5bb9-4512-84da-5a606a8ecde0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anona85c4226_4415_4833_bd70_f3509f3b647d" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:29:22.992000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "f7eac38f-48ec-4f63-8879-1a0ce02e9474", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "select revenue, date_utc from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`", + "session_id": null, + "timestamp": "2024-08-22 10:29:23.212000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "a394d7e82168ab8dd90e6ff402fa06f3a3043be775803f3831477534a444e421", + "usage_multiplier": 1, + "extra_info": { + "job_id": "a5707bee-8e81-46a3-a714-9c65347f2258", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonf0916716_a817_4feb_9357_09b7219c0890" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_*" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:26.682000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a774cee689b0b7d4d3d1268ec5a7ea5c32f5ae16be5b452786b82b528599e74", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0b0ac8b9-c9a8-461a-99b1-4408c2468e3d", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon128ef087_3e96_433e_a49c_7c3dbeda89f9" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:28.369000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9b6d48846141019d32c4c9f2660f49c8ea9659ef1c530fa14e3e1b831553d100", + "usage_multiplier": 1, + "extra_info": { + "job_id": "e7455bfd-9fea-409c-82a0-259088e942a4", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon441d7e56_e71e_4184_b282_50122f28ab8f" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:29.856000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a774cee689b0b7d4d3d1268ec5a7ea5c32f5ae16be5b452786b82b528599e74", + "usage_multiplier": 1, + "extra_info": { + "job_id": "032ad33c-8d21-4fe0-8013-e7f4bfef7b91", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anona0ecfb8a_d416_41c5_92c7_ddeb71cc3872" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:31.415000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9b6d48846141019d32c4c9f2660f49c8ea9659ef1c530fa14e3e1b831553d100", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b7eab270-0938-40f2-b2c7-9608feb71219", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon1f7efd21_bc80_4a18_b307_46415df75599" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:33.082000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a774cee689b0b7d4d3d1268ec5a7ea5c32f5ae16be5b452786b82b528599e74", + "usage_multiplier": 1, + "extra_info": { + "job_id": "68f63699-3ed3-4b36-b956-a38f9f72d721", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2a405ce1_4a34_45c2_b741_eb862f9bd688" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:34.811000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9b6d48846141019d32c4c9f2660f49c8ea9659ef1c530fa14e3e1b831553d100", + "usage_multiplier": 1, + "extra_info": { + "job_id": "83e0e55f-be23-422c-a741-88253937e178", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anona86a38fe_7ba8_4625_bd80_08b83836dc74" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:36.374000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a774cee689b0b7d4d3d1268ec5a7ea5c32f5ae16be5b452786b82b528599e74", + "usage_multiplier": 1, + "extra_info": { + "job_id": "98c3b073-7686-4c4e-9f12-fb18d9d084f5", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonced0b673_cde3_4a7e_85fb_b14a3f15729d" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:37.890000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9b6d48846141019d32c4c9f2660f49c8ea9659ef1c530fa14e3e1b831553d100", + "usage_multiplier": 1, + "extra_info": { + "job_id": "aa89ab39-11e7-45ce-b9fd-181e8f16843d", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon551db81a_c63c_40b3_b8d5_4ecceabe1c68" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) < (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:39.451000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "3a774cee689b0b7d4d3d1268ec5a7ea5c32f5ae16be5b452786b82b528599e74", + "usage_multiplier": 1, + "extra_info": { + "job_id": "c955f68d-7d2e-4c8d-978b-1b51496390d2", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone845e07d_3c57_4dad_930c_c9c2eb4e27ad" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(revenue) gross from gcp-staging.smoke_test_db.partition_test\n group by month\n order by gross\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:41.032000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "9b6d48846141019d32c4c9f2660f49c8ea9659ef1c530fa14e3e1b831553d100", + "usage_multiplier": 1, + "extra_info": { + "job_id": "4d279f45-7b97-45f8-a263-88e2d8f31bf0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon6b4e215f_b542_4941_9274_3ba107aa4955" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "select value from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:47.575000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "18deb4c7e128343b7ecc2cf59b9c9a06f96d2ce4e66b33bcd2b5c26899d6b9b5", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1bea24f3-644e-4dd4-98d4-5f74d3d3e6d3", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon91770d44_60b5_4727_8f2c_1a4f7ab92866" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select value from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:50.777000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "18deb4c7e128343b7ecc2cf59b9c9a06f96d2ce4e66b33bcd2b5c26899d6b9b5", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1d670ea7-1e10-4952-8beb-5fe77bb8f22e", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb0bc3965_5bee_4284_92f8_32e838db3f2a" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select value from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:54.074000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "18deb4c7e128343b7ecc2cf59b9c9a06f96d2ce4e66b33bcd2b5c26899d6b9b5", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3eb125df-feca-44c0-9896-326b6c34a3bf", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone59747e9_b4ff_4ccc_b8ba_f3f92eb85f44" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select * from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:57.066000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "553f645d24478a4dd699b8deba3a900f3a8907f53105a104f543d08aaef22930", + "usage_multiplier": 1, + "extra_info": { + "job_id": "13e496a2-3160-4bb2-b5d6-b81d61303d8a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon411f6440_ee8b_4fbe_83f1_e4c99b930da7" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:29:58.963000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "19af6f98-b709-47dc-8358-14e2dcc91894", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "select * from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:30:00.076000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "553f645d24478a4dd699b8deba3a900f3a8907f53105a104f543d08aaef22930", + "usage_multiplier": 1, + "extra_info": { + "job_id": "aa6811f2-bfea-4ec3-a7b5-9d70f0475706", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon42e59724_9567_47eb_aa95_c1ec7418902b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:30:01.034000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "152ed309-796b-4177-b082-82010a4f7e08", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:30:02.137000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "a1a22c15-873b-453f-80d0-b82b6391d360", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonc5692ab6_b9b1_4294_adaf_9bc23e242801" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:30:02.168000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "4e603b0a-2c6d-4c32-acde-0646f680a84f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:30:02.202000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "8c807377-da03-4a71-bb46-a191792ff0bf", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724322506419\n AND last_modified_time <= 1724322600205\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:30:02.282000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0502a1a5-bb81-4926-96ca-c6dbf496f3d7", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon02406632_13ec_4bb5_8272_9241dc8485cf" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n select extract(month from date_utc) month, count(*) count, sum(value) total from gcp-staging.smoke_test_db.usage_test\n group by month\n order by total\n ", + "session_id": null, + "timestamp": "2024-08-22 10:30:02.823000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2619b42bcfd850c2f6df3a72e3b4e53a635a7169d09facdfa4269366f3a48280", + "usage_multiplier": 1, + "extra_info": { + "job_id": "2f1c758f-62a5-45ba-9d2e-769bed74a333", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon8af21009_09ce_4944_a5ad_6a57c73352f1" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "\n select name, post_abbr from `gcp-staging-2.smoke_test_db_4.external_table_us_states`\n", + "session_id": null, + "timestamp": "2024-08-22 10:30:05.248000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "508134047ff2cc6a25ed279ebf1541b1681e0cb7eb22e1cfacd9fd5a51c8efd6", + "usage_multiplier": 1, + "extra_info": { + "job_id": "2e204e78-a2a1-4b5f-90f2-a0a6662cd4b1", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon28991a9f_ae0c_4016_b1b7_7d9346528fb8" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "external_table_us_states" + } + ] + } + }, + { + "query": "\n SELECT\n first_name,\n last_name,\n dob,\n addresses[offset(0)].address,\n addresses[offset(0)].city\n FROM \n gcp-staging-2.smoke_test_db_4.table_with_nested_fields\n", + "session_id": null, + "timestamp": "2024-08-22 10:30:06.806000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "71b4e0082e4005e45c1e32199147cc5d02491a45c225065c8bbdc6576f4724ea", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3305dfa5-708c-4955-b240-0657bcddaa04", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonfcea9c21_50a4_41c2_82bc_31e8d23874bd" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_nested_fields" + } + ] + } + }, + { + "query": "\n select \n transaction_id \n from \n `gcp-staging-2.smoke_test_db_4.table_with_ingestion_time_partition` \n where \n _PARTITIONDATE = CURRENT_DATE()\n", + "session_id": null, + "timestamp": "2024-08-22 10:30:08.186000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "6711368fa67682497ba5ff5dbbce9302950b990da8951d52f83a59385e22e9ca", + "usage_multiplier": 1, + "extra_info": { + "job_id": "878f73b2-9ff7-4d40-b8c4-81c72de46922", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon385fe8ae_2321_4782_ac22_9be1ec64a4a2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_ingestion_time_partition" + } + ] + } + }, + { + "query": "\n SELECT\n customer_id,\n date1\n FROM\n `gcp-staging-2.smoke_test_db_4.table_with_integer_range_partition`\n WHERE\n customer_id=1\n", + "session_id": null, + "timestamp": "2024-08-22 10:30:09.673000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "6690a922f76f680ab004c39b375979b610546bb4fd20d498bb4a83c94ce2d476", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cf03003b-4142-461f-b199-2edb45cda9bf", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anondcc3a621_6750_4415_95c6_7af5662e95cb" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "table_with_integer_range_partition" + } + ] + } + }, + { + "query": "select revenue, date_utc from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", + "session_id": null, + "timestamp": "2024-08-22 10:30:11.320000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "1a20947e51269d384a3e3e8e33c755633f158543f56881c7906d27a3df5a99b8", + "usage_multiplier": 1, + "extra_info": { + "job_id": "771f3da2-a218-456e-a0a1-f22e15410442", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonc39cfdd0_cf93_4e71_98a8_1ecdad5bdad9" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table@1724319011327" + } + ] + } + }, + { + "query": "select revenue, date_utc from [gcp-staging-2.smoke_test_db_3.base_table_2@0]", + "session_id": null, + "timestamp": "2024-08-22 10:30:14.269000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "c24065c5-8f8e-4aad-a2fa-c856f517a906", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon12a7e0e3_25d3_4121_82ba_8700fa152525" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2@0" + } + ] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:31:01.120000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "66e0e6f8-a780-4755-9f65-c3b51107ef96", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon471d3729_9b83_47b1_80f9_a5a41453441e" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:31:01.179000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "279851a7-44ce-477a-8200-3c52dc5d9b1a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:31:01.192000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "98febd5c-417d-4d5b-91aa-58ebbccfa2c0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:32:01.120000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "ecaf762e-2b22-4abb-9e67-ff6aee05c046", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:32:01.234000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9c99e087-c5fb-4a93-bb72-852573578dbe", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anond4824d79_f127_4f96_b17f_17f67aa2859b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:32:01.252000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "ad3a2684-c680-4509-b849-3bcc4f9a7231", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:33:00.571000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "df8f9856-3809-4b55-afc1-76087191c772", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:33:01.712000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b8f9703c-bb4e-43f1-b701-638bacf79f50", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon3bd064cd_a083_47dc_8af1_5794e160f59a" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236667305\n AND last_modified_time <= 1724323067305\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:37:48.146000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "d04d3fb8-bb99-4028-b596-c951ede50ca6", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anone16d436b_d361_45f5_8b54_7f859d609acd" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:37:48.300000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "96f20c34-3956-4c47-a13b-c669fc4c7f63", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id='purchase_event';", + "session_id": null, + "timestamp": "2024-08-22 10:37:48.793000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "8aed7604ce6319236cca0b0f7d16614774821c2666f5cf6f5be2cec702026d64", + "usage_multiplier": 1, + "extra_info": { + "job_id": "6b73759c-3f08-455b-8df2-dd1bc7dc0902", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon08572e99_180f_4e41_9323_7186634593fe" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:37:49.520000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "01a6c13e-cb18-45f4-b40a-38315a2c6430", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT\n (SUM(CASE WHEN amount BETWEEN 0 AND 10 THEN 1 ELSE 0 END) * 100.0) / COUNT(*) AS percentage_in_range\nFROM\n gcp-staging.customer_demo.purchase_event;", + "session_id": null, + "timestamp": "2024-08-22 10:37:49.785000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "4fad02be-37dd-4252-89f3-b4d98b674f29", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anondaaf8a98bb16f9b4fd4e7f6ada08bac86581b8e2a25e55399914c155fd8566b7" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724322603618\n AND last_modified_time <= 1724323070196\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:37:50.757000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "55c62d69-99af-4040-bff8-e97fadd3cc90", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon10c6bd04_a9f1_4c8d_bcb1_a1b1f2f80a58" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:37:51.009000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "14b1ce0b-61c7-4fe9-8d9b-839ef1c3ee06", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:37:51.359000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "2e5fc864-fc67-416c-9244-448da4e82a78", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:37:51.944000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "68babe3d-54f6-45ff-8b0b-9da7911dae59", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anoncb8a32f3_1aa6_4da8_9d78_eeb08eb16661" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:37:52.220000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "b6c7bb75-35cd-4715-8773-a6abc2f300dc", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:37:53.407000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "fa4425f7-1d80-42fc-8f90-97a00df580e8", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:37:53.503000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "b23c1455-ddfa-4064-8323-d39aafb2694e", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:37:53.512000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "02810b1a-a070-4ab3-b49a-fe0be8bc99ba", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:37:53.605000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "dfa85a7f-0ff1-4e28-87a1-800574a32c61", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4f90bad8_7754_4a8e_a9f6_4b468ac7fb4d" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:37:54.799000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "7229d1dc-e6e9-4cf4-81b8-8df0152379d8", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id=\"purchase_event\"\n AND last_modified_time >= 1692787100665\n AND last_modified_time <= 1724323100665\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:38:21.339000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "c4076a26606e9f6d6b9fde2855758ba67b1155b110a65f72cf1999fa1facdfa4", + "usage_multiplier": 1, + "extra_info": { + "job_id": "08be753d-4377-43a9-86ec-b6dd147b18a9", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb802196a_56ec_43df_9453_dc27b67fc1ee" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236704226\n AND last_modified_time <= 1724323104226\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:38:24.871000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "361a3095-dfdc-4d2d-af0b-44852398caba", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4cfceddf_4d07_43d7_8e73_61dc558b48b6" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:38:26.351000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "33f0ba19-1925-4063-bb6c-fea0d299f76b", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:38:27.315000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "f489b9ab-aa90-4ba3-8de8-2e6e2482a15a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236707152\n AND last_modified_time <= 1724323107152\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:38:27.719000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b5571b2d-602d-4d66-a6cd-77772f72656f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb6690e00_3f56_44ac_8ab9_03b7c3c34bbe" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:38:28.175000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "c768ca60-e0cb-41dc-8bab-8117d2a8dd54", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon8cc6e9e7_16fd_490b_86e4_dc8c4b60cc22" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE quantity < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:38:28.678000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "327849f9-eea0-4428-89ef-7188bafafbc0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon32e6bd0458339a10581928d131db191eaea3e29936a3454e7946ed8a12f17e38" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724323071874\n AND last_modified_time <= 1724323108983\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:38:29.577000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "19e77df4-43aa-4ab5-85d2-e3be1621832b", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon12e0faf9_b2b4_4b8e_acd5_5bda7dc52803" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:38:31.050000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "4f21cc35-0ad1-451f-b087-e11e8c469a9c", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:38:34.897000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "886c3952-b0b6-452e-b7f1-c45ff2c24e67", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonfe55e7f9_741d_441e_a27b_47696d24ccaf" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:38:35.783000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "15855ee2-041f-4ab8-9b19-03b080fc884d", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:38:37.358000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "c88d7780-92b6-4596-8a40-cfa3db95f058", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:38:37.485000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "3d6d5760-9b9e-4e73-a638-17779d88233b", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:38:38.698000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "28199aa8-c5cb-445f-939c-73bb7f49fed7", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:38:38.831000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3733cf41-b202-4a72-822d-04298d9fff57", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon8da3cafa_0dc9_4a37_a70a_d066a541b880" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:38:41.103000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "9ad989d5-81f0-4a2d-baaf-97064f36dde8", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:38:43.435000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "68aa5358-09cc-4d2f-a6a5-488c729fbbd3", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236723957\n AND last_modified_time <= 1724323123957\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:38:44.473000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cae21ce5-5c2d-4151-9d35-bcdb799c9cdd", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2fb5c10b_43b3_40e8_a854_03b63c1b3502" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:39:00.857000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "e728c245-07a0-4b19-809b-e44d173b5135", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724323110374\n AND last_modified_time <= 1724323140076\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:39:00.970000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cd4e6023-5a4c-4445-99b9-6f7aef393ff1", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anond5cc46f2_7658_4061_8aeb_b2a2dd6688a8" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:39:01.969000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "0787f7d7-a643-439b-9442-baa914362c87", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:39:02.760000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b31aacd0-b9d7-4ef6-979a-e825fb106c96", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonaaf15c74_96c6_4bb6_91dc_e85b02790c73" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:39:02.862000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "9c823051-2be3-4c62-b393-4876ef5af734", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:39:03.200000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "03321ba7-908e-4acf-b858-648654da4249", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236746653\n AND last_modified_time <= 1724323146653\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:39:07.316000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "b7a3bcf4-e5da-4c8b-b61c-b7324e37823c", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon11845a49_abe5_4720_a257_1af0548c3a71" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:39:09.494000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "7ef7f8e0-f8c2-4a47-a6b5-e8ebe3fc69b2", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:39:15.149000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "cd265738-ae52-4e75-a241-682e601b7408", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236758284\n AND last_modified_time <= 1724323158284\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:39:18.899000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "709d978b-c616-4ccf-b0bc-46015c7ea201", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon5e97dfab_c9b9_4fb8_bf50_acb06ce77edc" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT\n (SUM(CASE WHEN amount BETWEEN 0 AND 10 THEN 1 ELSE 0 END) * 100.0) / COUNT(*) AS percentage_in_range\nFROM\n gcp-staging.customer_demo.purchase_event;", + "session_id": null, + "timestamp": "2024-08-22 10:39:27.170000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "5ef872bb-d3c3-4e6e-87b5-9e1838f38563", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anondaaf8a98bb16f9b4fd4e7f6ada08bac86581b8e2a25e55399914c155fd8566b7" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT COUNT(*)\n FROM gcp-staging.customer_demo.purchase_event\n WHERE amount < 0\n ", + "session_id": null, + "timestamp": "2024-08-22 10:39:28.388000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "fb84efb3-c71d-4b80-b717-863b22a0bceb", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon60e011141b8e3c85fb386e55d995a8290fc3a0896c26ac6d5717f14f5f163947" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) FROM gcp-staging.customer_demo.purchase_event WHERE user_id IS NULL", + "session_id": null, + "timestamp": "2024-08-22 10:39:30.390000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "de5f59b1-0249-421d-90eb-5103a43b49fd", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb22d0d582afd566e08144b196aec36622576fdf25b6e74a35e044a5b720ce191" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id='purchase_event';", + "session_id": null, + "timestamp": "2024-08-22 10:39:31.842000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "8aed7604ce6319236cca0b0f7d16614774821c2666f5cf6f5be2cec702026d64", + "usage_multiplier": 1, + "extra_info": { + "job_id": "438c4b01-ec53-421a-bc5f-d61ed1edaf7f", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2c5c1f70_9bc9_41c4_a096_93b8bda0a0e9" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:40:01.278000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "6ee9b8d0-7767-447f-b333-55d5265d4388", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:40:01.332000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "b2ffdb5b-52cd-44b9-a2d9-38067e9cd9a4", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:40:01.488000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "bd6599fe-13b0-4fe8-8981-c307ccd4448d", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2072d2c5_e133_4b5a_8945_5254588340a3" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc2023\"\n AND last_modified_time >= 1724236803602\n AND last_modified_time <= 1724323203602\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:40:04.214000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "9c6f7727-ca89-4ad3-b29f-f4cc21b195e2", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anonb0ef55ef_0e63_4138_b8c0_49b113c4ea78" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging.customer_demo.__TABLES__\n WHERE table_id=\"purchase_event\"\n AND last_modified_time >= 1692787205076\n AND last_modified_time <= 1724323205076\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:40:05.684000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "c4076a26606e9f6d6b9fde2855758ba67b1155b110a65f72cf1999fa1facdfa4", + "usage_multiplier": 1, + "extra_info": { + "job_id": "12ffdab3-f7ce-4260-9437-80aacfb42e08", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2ba3f5b8_331b_4fed_9985_0559a69e5bf5" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "customer_demo", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:41:00.943000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "31c7e5fd-c5d3-477b-89b8-74c57c1b0fb9", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:41:00.951000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cfb45092-68bb-4be3-8111-dc9533c9bbe2", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon08499120_ac59_47f0_9d7a_5d3b9e2d95c2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:41:02.520000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "e6882b25-dc5e-4d49-a916-ee09d2e4bcc6", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT AVG(amount) FROM gcp-staging.customer_demo.purchase_event", + "session_id": null, + "timestamp": "2024-08-22 10:42:00.672000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "d916df78-aca9-4365-aa7e-32cc56929771", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4db3da963edf09d1d3e6b8dddc9f22960bb56016490ef27620acc390fcf18a27" + }, + "referenced_tables": [] + } + }, + { + "query": "SELECT COUNT(*) \nFROM gcp-staging.customer_demo.purchase_event \nWHERE amount < 0", + "session_id": null, + "timestamp": "2024-08-22 10:42:01.938000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "08737edb-28ad-4dde-887e-c6c4054d94ef", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon84323ce6f50f99cfc5d2e8ad640f3923743cb41a6a759f18a7fce13c7a358d28" + }, + "referenced_tables": [] + } + }, + { + "query": "\n SELECT last_modified_time\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id=\"abc\"\n AND last_modified_time >= 1724323141731\n AND last_modified_time <= 1724323322589\n LIMIT 5\n ;", + "session_id": null, + "timestamp": "2024-08-22 10:42:03.231000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "2c44fb03f02e7c6777499bbd1a0e15b075d39b85c2bb544223f01a9e5ce4656d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "914aafd8-5543-42d5-b21b-92df47c28268", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon4cd2e7fe_9f6f_4a6c_9c92_ff521c7caa99" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "\n SELECT row_count\n FROM gcp-staging-2.dataset_as_sharded_table.__TABLES__\n WHERE table_id='abc';", + "session_id": null, + "timestamp": "2024-08-22 10:42:04.926000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test", + "default_db": "gcp-staging", + "default_schema": "_SESSION", + "query_hash": "81bcba1019e933b2de506faf0f6549b8602519495edc855c0a9267b8d5bb976c", + "usage_multiplier": 1, + "extra_info": { + "job_id": "177a3d11-f04f-4819-9b67-555b3190af6a", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging", + "dataset_id": "_283d0dc3d7613bc3d88b6692661164402822056f", + "table_id": "anon2f5817f8_15ba_4df7_83c9_412d70c62fce" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "dataset_as_sharded_table", + "table_id": "__TABLES__" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.base_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:37.386000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "39323692dfca0074ed7ac63c27830a64ce68742109e09033015b854d46be7dde", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1b9741fb-0693-4371-acd0-e7c77a775aa0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anona358818e_bfc6_4373_b610_778f49467cef" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.partition_test", + "session_id": null, + "timestamp": "2024-08-22 10:28:41.153000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "6e1d1e89de700713d87b8de79d43da4d437ca0d98a5589385104ee7b4142f922", + "usage_multiplier": 1, + "extra_info": { + "job_id": "c2cec833-2d60-42b0-994b-604ec92a88ae", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anonbcd279fa_10aa_422d_a999_2383b3e8c932" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.lineage_from_tmp_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:44.291000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "8d3f968d2fbae03772fedb7e2dd43c2ba66ae90aa35e4824d40fd66997d51681", + "usage_multiplier": 1, + "extra_info": { + "job_id": "ac0b567c-b711-489e-93ea-1a4696055163", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anonc046b2fa_61f1_48da_9df1_5e517b6b1901" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_tmp_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.lineage_from_base", + "session_id": null, + "timestamp": "2024-08-22 10:28:47.479000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "1f52fe6a0e09ab08bb65356947420bc85f0cc1bb38557fcdc7e33a049e6b113f", + "usage_multiplier": 1, + "extra_info": { + "job_id": "8ea9e3a5-4ef2-4665-8ad2-c5aeaf4ce063", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anone04f0dcf_9a17_4be8_92c4_92b31811d167" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.view_from_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:50.411000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "d0726cb4d89262a037a6d3208dfa36439952b1fa95bcb6ea35fa6c46e46d5b5a", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1bf06f7f-1d21-413e-b26a-ca4569586cb4", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon59fa6fb5_e019_4328_8f23_01f38f7b5e3a" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.view_from_view_on_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:53.604000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "5a8d7c9836a318c4be63fcc8f38e460b12c2b070a5daa98dc2c004ff53b29c87", + "usage_multiplier": 1, + "extra_info": { + "job_id": "3e577d7c-7ccc-4be8-a19e-e1399fabee69", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon6d70c7aa_a77d_402a_96e0_414de3d1c7fb" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.table_from_view", + "session_id": null, + "timestamp": "2024-08-22 10:28:56.837000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "827d388f24f2f2a7961260884a76d0e6449ba200efd12d2500ae95350c63c53f", + "usage_multiplier": 1, + "extra_info": { + "job_id": "35d712b5-ce79-43db-98bb-52b6363a5ef3", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anonb6ff5e62_e4b7_4f9a_8fa7_7e86d3a6a7e4" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.table_from_view_and_table", + "session_id": null, + "timestamp": "2024-08-22 10:28:59.969000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "e0f5980f7d9741829dc2b3183b1ea065e9f5f6e8cb7650c10eeb3398c10d4d19", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cb805273-1313-45d0-af6d-fa6d9b3d8b56", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon50fed9a7_008c_4405_b22f_f93efccda484" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_view_and_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.materialized_view_from_table", + "session_id": null, + "timestamp": "2024-08-22 10:29:03.069000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "59fa2d6828221cc2962cc609935bc4c94a6f3a846e94439f7ff3374bcc439b4e", + "usage_multiplier": 1, + "extra_info": { + "job_id": "32825ee8-28b7-4670-9296-1b67d8978bf6", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anona7c9b810_373e_4102_8b6c_74622dbddb92" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.view_from_multiple_tables", + "session_id": null, + "timestamp": "2024-08-22 10:29:06.639000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "d6239d9f32f5ddd664255bb3629824c5a099e9a0213a484174fa6af613a3bc16", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1d8c6501-d09c-4540-b62c-0960c6ce4461", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anone4cf1322_c0a3_4178_85fe_ed3d25a86cf5" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "lineage_from_base" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + }, + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "materialized_view_from_table" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db_2.table_from_other_db", + "session_id": null, + "timestamp": "2024-08-22 10:29:09.851000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "a5c10c32c0b05d5248e6d337d9fe607998473778b0680e392f7a93ecc7869b1b", + "usage_multiplier": 1, + "extra_info": { + "job_id": "95c21bb0-9a17-4dde-8198-bde46c52d98e", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anone376867c_5467_4841_803a_4a547f3ba08b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db_2", + "table_id": "table_from_other_db" + } + ] + } + }, + { + "query": "select revenue from gcp-staging-2.smoke_test_db_3.base_table_2", + "session_id": null, + "timestamp": "2024-08-22 10:29:12.655000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "180575b59ab828f1bb56a49c25da862144fdf1a484773533a5363df180f81919", + "usage_multiplier": 1, + "extra_info": { + "job_id": "cf9d8407-934c-4b98-8fcf-be51bd5fc4b2", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anone0788f93_2e16_4300_b94c_f86eee9758c2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.table_from_another_project", + "session_id": null, + "timestamp": "2024-08-22 10:29:15.601000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "69c8f6b58948d2bafceffb46708bb041e1b46830a052530567fc054c752d43b1", + "usage_multiplier": 1, + "extra_info": { + "job_id": "72249ae6-3035-4698-ad2f-54f0f6f58976", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon2608c050_7916_471e_8ba3_c9e004e22daa" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "table_from_another_project" + } + ] + } + }, + { + "query": "select revenue from gcp-staging-2.smoke_test_db_4.sharded_table1_20230101", + "session_id": null, + "timestamp": "2024-08-22 10:29:18.514000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "cd3ceb2068c7b0691a03fb47183df1d55b07bd102781ede467de375a68736a51", + "usage_multiplier": 1, + "extra_info": { + "job_id": "7ad97142-5030-4723-ae8c-e48f0dc74d34", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon118ffa8c_5d72_4f8d_b022_b7d7d5e679e2" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230101" + } + ] + } + }, + { + "query": "select revenue from gcp-staging-2.smoke_test_db_4.sharded_table1_20230201", + "session_id": null, + "timestamp": "2024-08-22 10:29:21.765000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "a3fe1801fa4d68d8413b4b90c4704bcb6c629d8976c84355b7875b10655c9582", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1a903e95-526f-4f2a-b7bb-cf85559557cd", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon5e1130d1_930c_4a12_9c21_7c00388ee11f" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_20230201" + } + ] + } + }, + { + "query": "select revenue from `gcp-staging-2.smoke_test_db_4.sharded_table1_*`", + "session_id": null, + "timestamp": "2024-08-22 10:29:25.044000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "8e8d1f83b2d56f70fff7ea7879290c0b26663d185a80a872902ebf2da23bc3b1", + "usage_multiplier": 1, + "extra_info": { + "job_id": "1489ced3-a8db-42f0-b6c1-0b23c2e042e7", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon89622e5a_c0f9_469b_9a9a_2379cac465b6" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_4", + "table_id": "sharded_table1_*" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) = (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:42.590000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "9776768f462eef58c2dbc091f9548e020cf7addc1628cf83a2ea506aa81ece86", + "usage_multiplier": 1, + "extra_info": { + "job_id": "d6479b3a-694b-4271-aa3c-3be5e8b8f7c6", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon9717e692_a817_40f3_8f8c_d2af76a4eed9" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) = (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:44.320000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "9776768f462eef58c2dbc091f9548e020cf7addc1628cf83a2ea506aa81ece86", + "usage_multiplier": 1, + "extra_info": { + "job_id": "0e5a166c-4018-4585-a341-a40585343818", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon3453f1d1_c634_4273_b472_c72fac89f718" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "\n select revenue from gcp-staging.smoke_test_db.partition_test t\n where cast(t.date_utc as DATE) = (select max(date_utc) from gcp-staging.smoke_test_db.partition_test)\n", + "session_id": null, + "timestamp": "2024-08-22 10:29:45.961000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "9776768f462eef58c2dbc091f9548e020cf7addc1628cf83a2ea506aa81ece86", + "usage_multiplier": 1, + "extra_info": { + "job_id": "a36c8702-c796-4173-a19a-6b072c5064aa", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anond9168686_8f61_47f7_9546_d97b9daf573b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "partition_test" + } + ] + } + }, + { + "query": "select key from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:49.238000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "dbd21e2e4c5f9fe397c76585acdf3b84f72525d0d951d79e8057b05dc8d7c49d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "2360b8ad-4e69-45ac-9294-a528634219b3", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon0bff271e_f2e5_4dd8_ba72_3996ab9a4fbf" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select key from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:52.280000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "dbd21e2e4c5f9fe397c76585acdf3b84f72525d0d951d79e8057b05dc8d7c49d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "68464689-cf31-4cee-87cc-9cf96db96285", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anonced1fae9_3dcb_40ba_a69c_ae6d75133e97" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select key from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:55.586000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "dbd21e2e4c5f9fe397c76585acdf3b84f72525d0d951d79e8057b05dc8d7c49d", + "usage_multiplier": 1, + "extra_info": { + "job_id": "277b376a-186b-40aa-a64b-34940ed9d5e1", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon27185a25_2daf_481f_8c43_20fbac91241b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select * from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:29:58.466000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "553f645d24478a4dd699b8deba3a900f3a8907f53105a104f543d08aaef22930", + "usage_multiplier": 1, + "extra_info": { + "job_id": "f7010dd4-4fd8-42eb-a973-844a161d73ac", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon1fd96c89_d1e8_44f4_b714_bdf425f3920a" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select * from gcp-staging.smoke_test_db.usage_test", + "session_id": null, + "timestamp": "2024-08-22 10:30:01.431000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "553f645d24478a4dd699b8deba3a900f3a8907f53105a104f543d08aaef22930", + "usage_multiplier": 1, + "extra_info": { + "job_id": "8bd62522-3803-431d-8970-caf0c9f64726", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon026ba4a8_61fe_40e0_bbac_c2745e7e989b" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "usage_test" + } + ] + } + }, + { + "query": "select revenue from gcp-staging.smoke_test_db.base_table FOR SYSTEM_TIME AS OF TIMESTAMP_SUB(CURRENT_TIMESTAMP(), INTERVAL 1 HOUR)", + "session_id": null, + "timestamp": "2024-08-22 10:30:12.787000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": "e55a9654f8c04ebea7d9b153b53cd5e0763b9a15fd68b219eb07719ee0214726", + "usage_multiplier": 1, + "extra_info": { + "job_id": "51df8223-866e-4322-9c1c-a91c808a37b0", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anoncbdc907d_def4_4170_a5e2_537bc2cadcfc" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging", + "dataset_id": "smoke_test_db", + "table_id": "base_table@1724319012792" + } + ] + } + }, + { + "query": "select revenue from [gcp-staging-2.smoke_test_db_3.base_table_2@0]", + "session_id": null, + "timestamp": "2024-08-22 10:30:15.755000+00:00", + "user": "urn:li:corpuser:dh-bigquery-smoke-test-2", + "default_db": "gcp-staging-2", + "default_schema": "_SESSION", + "query_hash": null, + "usage_multiplier": 1, + "extra_info": { + "job_id": "58a71429-22f4-4251-a4a0-f5370f1caadd", + "statement_type": "SELECT", + "destination_table": { + "project_id": "gcp-staging-2", + "dataset_id": "_9b92e769ee331eccc38faf408e659980e15ac5cb", + "table_id": "anon5be4a9ee_7236_4a08_8e69_62eee0161075" + }, + "referenced_tables": [ + { + "project_id": "gcp-staging-2", + "dataset_id": "smoke_test_db_3", + "table_id": "base_table_2@0" + } + ] + } + } +] \ No newline at end of file diff --git a/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py index 5874bf60a21f9..88ac30e79843a 100644 --- a/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py +++ b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py @@ -1,28 +1,62 @@ +import json +from datetime import datetime +from pathlib import Path from unittest.mock import patch +import pytest from freezegun import freeze_time from datahub.ingestion.source.usage.usage_common import BaseUsageConfig +from datahub.sql_parsing.sql_parsing_aggregator import ObservedQuery +from datahub.utilities.file_backed_collections import ConnectionWrapper, FileBackedList from tests.test_helpers import mce_helpers from tests.test_helpers.state_helpers import run_and_get_pipeline FROZEN_TIME = "2024-08-19 07:00:00" +def _generate_queries_cached_file(tmp_path: Path, queries_json_path: Path) -> None: + # We choose to generate Cached audit log (FileBackedList backed by sqlite) at runtime + # instead of using pre-existing sqlite file here as default serializer for FileBackedList + # uses pickle which may not work well across python versions. + + query_cache: FileBackedList[ObservedQuery] = FileBackedList( + ConnectionWrapper(tmp_path / "audit_log.sqlite") + ) + with open(queries_json_path, "r") as f: + queries = json.load(f) + assert isinstance(queries, list) + for query in queries: + query["timestamp"] = datetime.fromisoformat(query["timestamp"]) + query_cache.append(ObservedQuery(**query)) + + query_cache.flush() + + @freeze_time(FROZEN_TIME) @patch("google.cloud.bigquery.Client") -def test_queries_ingestion(client, pytestconfig, monkeypatch): +def test_queries_ingestion(client, pytestconfig, monkeypatch, tmp_path): test_resources_dir = pytestconfig.rootpath / "tests/integration/bigquery_v2" mcp_golden_path = f"{test_resources_dir}/bigquery_queries_mcps_golden.json" mcp_output_path = "bigquery_queries_mcps.json" + try: + # query_log.json is originally created by using queries dump generated by + # acryl bigquery connector smoke test and using `datahub check extract-sql-agg-log` + # command with tablename="data" to convert cached audit log to queries json followed by + # a simple `acryl-staging`->`gcp-staging` replacement. + + _generate_queries_cached_file(tmp_path, test_resources_dir / "query_log.json") + except Exception as e: + pytest.fail(f"Failed to generate queries sqlite cache: {e}") + pipeline_config_dict: dict = { "source": { "type": "bigquery-queries", "config": { "project_ids": ["gcp-staging", "gcp-staging-2"], - "local_temp_path": test_resources_dir, + "local_temp_path": tmp_path, }, }, "sink": {"type": "file", "config": {"filename": mcp_output_path}}, diff --git a/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_usage.json b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_usage.json new file mode 100644 index 0000000000000..6ea29520dc688 --- /dev/null +++ b/metadata-ingestion/tests/unit/sql_parsing/aggregator_goldens/test_basic_usage.json @@ -0,0 +1,51 @@ +[ +{ + "entityType": "dataset", + "entityUrn": "urn:li:dataset:(urn:li:dataPlatform:redshift,dev.public.foo,PROD)", + "changeType": "UPSERT", + "aspectName": "datasetUsageStatistics", + "aspect": { + "json": { + "timestampMillis": 1707177600000, + "eventGranularity": { + "unit": "DAY", + "multiple": 1 + }, + "partitionSpec": { + "partition": "FULL_TABLE_SNAPSHOT", + "type": "FULL_TABLE" + }, + "uniqueUserCount": 2, + "totalSqlQueries": 6, + "topSqlQueries": [ + "SELECT\n *\nFROM foo", + "CREATE TABLE bar AS\nSELECT\n b + c AS c\nFROM foo" + ], + "userCounts": [ + { + "user": "urn:li:corpuser:user1", + "count": 5 + }, + { + "user": "urn:li:corpuser:user2", + "count": 1 + } + ], + "fieldCounts": [ + { + "fieldPath": "b", + "count": 6 + }, + { + "fieldPath": "c", + "count": 6 + }, + { + "fieldPath": "a", + "count": 5 + } + ] + } + } +} +] \ No newline at end of file diff --git a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py index 2b8bc68ee6dc5..2e15dabb10d11 100644 --- a/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py +++ b/metadata-ingestion/tests/unit/sql_parsing/test_sql_aggregator.py @@ -4,6 +4,9 @@ import pytest from freezegun import freeze_time +from datahub.configuration.datetimes import parse_user_datetime +from datahub.configuration.time_window_config import BucketDuration, get_time_bucket +from datahub.ingestion.source.usage.usage_common import BaseUsageConfig from datahub.metadata.urns import CorpUserUrn, DatasetUrn from datahub.sql_parsing.sql_parsing_aggregator import ( KnownQueryLineageInfo, @@ -20,7 +23,7 @@ from tests.test_helpers.click_helpers import run_datahub_cmd RESOURCE_DIR = pathlib.Path(__file__).parent / "aggregator_goldens" -FROZEN_TIME = "2024-02-06 01:23:45" +FROZEN_TIME = "2024-02-06T01:23:45Z" def _ts(ts: int) -> datetime: @@ -553,3 +556,48 @@ def test_lineage_via_temp_table_disordered_add(pytestconfig: pytest.Config) -> N outputs=mcps, golden_path=RESOURCE_DIR / "test_lineage_via_temp_table_disordered_add.json", ) + + +@freeze_time(FROZEN_TIME) +def test_basic_usage(pytestconfig: pytest.Config) -> None: + + frozen_timestamp = parse_user_datetime(FROZEN_TIME) + aggregator = SqlParsingAggregator( + platform="redshift", + generate_lineage=False, + generate_usage_statistics=True, + generate_operations=False, + usage_config=BaseUsageConfig( + start_time=get_time_bucket(frozen_timestamp, BucketDuration.DAY), + end_time=frozen_timestamp, + ), + ) + + aggregator._schema_resolver.add_raw_schema_info( + DatasetUrn("redshift", "dev.public.foo").urn(), + {"a": "int", "b": "int", "c": "int"}, + ) + + aggregator.add_observed_query( + query="select * from foo", + default_db="dev", + default_schema="public", + usage_multiplier=5, + query_timestamp=frozen_timestamp, + user=CorpUserUrn("user1"), + ) + aggregator.add_observed_query( + query="create table bar as select b+c as c from foo", + default_db="dev", + default_schema="public", + query_timestamp=frozen_timestamp, + user=CorpUserUrn("user2"), + ) + + mcps = list(aggregator.gen_metadata()) + + mce_helpers.check_goldens_stream( + pytestconfig, + outputs=mcps, + golden_path=RESOURCE_DIR / "test_basic_usage.json", + ) From e8fcdcbcbd552df936cb7e3898ba37604f172843 Mon Sep 17 00:00:00 2001 From: Mayuri N Date: Mon, 26 Aug 2024 10:44:24 +0530 Subject: [PATCH 16/16] fix test --- .../integration/bigquery_v2/test_bigquery_queries.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py index 88ac30e79843a..fb51aac9fa246 100644 --- a/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py +++ b/metadata-ingestion/tests/integration/bigquery_v2/test_bigquery_queries.py @@ -20,9 +20,8 @@ def _generate_queries_cached_file(tmp_path: Path, queries_json_path: Path) -> No # instead of using pre-existing sqlite file here as default serializer for FileBackedList # uses pickle which may not work well across python versions. - query_cache: FileBackedList[ObservedQuery] = FileBackedList( - ConnectionWrapper(tmp_path / "audit_log.sqlite") - ) + shared_connection = ConnectionWrapper(tmp_path / "audit_log.sqlite") + query_cache: FileBackedList[ObservedQuery] = FileBackedList(shared_connection) with open(queries_json_path, "r") as f: queries = json.load(f) assert isinstance(queries, list) @@ -30,12 +29,14 @@ def _generate_queries_cached_file(tmp_path: Path, queries_json_path: Path) -> No query["timestamp"] = datetime.fromisoformat(query["timestamp"]) query_cache.append(ObservedQuery(**query)) - query_cache.flush() + query_cache.close() + shared_connection.close() @freeze_time(FROZEN_TIME) @patch("google.cloud.bigquery.Client") -def test_queries_ingestion(client, pytestconfig, monkeypatch, tmp_path): +@patch("google.cloud.resourcemanager_v3.ProjectsClient") +def test_queries_ingestion(project_client, client, pytestconfig, monkeypatch, tmp_path): test_resources_dir = pytestconfig.rootpath / "tests/integration/bigquery_v2" mcp_golden_path = f"{test_resources_dir}/bigquery_queries_mcps_golden.json"