diff --git a/pyproject.toml b/pyproject.toml index 01ca7ae20..7830d603f 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -16,7 +16,7 @@ dependencies = [ "databricks-sdk~=0.29.0", "sqlglot==25.8.1", "databricks-labs-blueprint[yaml]>=0.2.3", - "databricks-labs-lsql>=0.4.3", + "databricks-labs-lsql>=0.7.5", "cryptography>=41.0.3", ] diff --git a/src/databricks/labs/remorph/deployment/dashboard.py b/src/databricks/labs/remorph/deployment/dashboard.py index 98cdd46d6..5b76f09b8 100644 --- a/src/databricks/labs/remorph/deployment/dashboard.py +++ b/src/databricks/labs/remorph/deployment/dashboard.py @@ -1,83 +1,140 @@ -import json import logging from datetime import timedelta -from importlib.abc import Traversable -from typing import Any +from pathlib import Path from databricks.labs.blueprint.installation import Installation from databricks.labs.blueprint.installer import InstallState +from databricks.labs.lsql.dashboards import DashboardMetadata, Dashboards from databricks.sdk import WorkspaceClient -from databricks.sdk.errors import DatabricksError -from databricks.sdk.errors import InvalidParameterValue +from databricks.sdk.errors import ( + InvalidParameterValue, + NotFound, + DeadlineExceeded, + InternalError, + ResourceAlreadyExists, +) from databricks.sdk.retries import retried -from databricks.sdk.service.dashboards import Dashboard +from databricks.sdk.service.dashboards import LifecycleState, Dashboard + +from databricks.labs.remorph.config import ReconcileConfig, ReconcileMetadataConfig logger = logging.getLogger(__name__) class DashboardDeployment: - _UPLOAD_TIMEOUT = timedelta(seconds=30) - def __init__(self, ws: WorkspaceClient, installation: Installation, install_state: InstallState): + def __init__( + self, + ws: WorkspaceClient, + installation: Installation, + install_state: InstallState, + ): self._ws = ws self._installation = installation self._install_state = install_state - def deploy(self, name: str, dashboard_file: Traversable, parameters: dict[str, Any] | None = None): - logger.debug(f"Deploying dashboard {name} from {dashboard_file.name}") - dashboard_data = self._substitute_params(dashboard_file, parameters or {}) - dashboard = self._update_or_create_dashboard(name, dashboard_data, dashboard_file) - logger.info(f"Dashboard deployed with dashboard_id {dashboard.dashboard_id}") - logger.info(f"Dashboard URL: {self._ws.config.host}/sql/dashboardsv3/{dashboard.dashboard_id}") - self._install_state.save() - - @retried(on=[DatabricksError], timeout=_UPLOAD_TIMEOUT) - def _update_or_create_dashboard(self, name: str, dashboard_data, dashboard_file) -> Dashboard: - if name in self._install_state.dashboards: + def deploy( + self, + folder: Path, + config: ReconcileConfig, + ): + """ + Create dashboards from Dashboard metadata files. + The given folder is expected to contain subfolders each containing metadata for individual dashboards. + + :param folder: Path to the base folder. + :param config: Configuration for reconciliation. + """ + logger.info(f"Deploying dashboards from base folder {folder}") + parent_path = f"{self._installation.install_folder()}/dashboards" + try: + self._ws.workspace.mkdirs(parent_path) + except ResourceAlreadyExists: + logger.info(f"Dashboard parent path already exists: {parent_path}") + + valid_dashboard_refs = set() + for dashboard_folder in folder.iterdir(): + if not dashboard_folder.is_dir(): + continue + valid_dashboard_refs.add(self._dashboard_reference(dashboard_folder)) + dashboard = self._update_or_create_dashboard(dashboard_folder, parent_path, config.metadata_config) + logger.info( + f"Dashboard deployed with URL: {self._ws.config.host}/sql/dashboardsv3/{dashboard.dashboard_id}" + ) + self._install_state.save() + + self._remove_deprecated_dashboards(valid_dashboard_refs) + + def _dashboard_reference(self, folder: Path) -> str: + return f"{folder.stem}".lower() + + # InternalError and DeadlineExceeded are retried because of Lakeview internal issues + # These issues have been reported to and are resolved by the Lakeview team + # Keeping the retry for resilience + @retried(on=[InternalError, DeadlineExceeded], timeout=timedelta(minutes=3)) + def _update_or_create_dashboard( + self, + folder: Path, + ws_parent_path: str, + config: ReconcileMetadataConfig, + ) -> Dashboard: + logging.info(f"Reading dashboard folder {folder}") + metadata = DashboardMetadata.from_path(folder).replace_database( + catalog=config.catalog, + catalog_to_replace="remorph", + database=config.schema, + database_to_replace="reconcile", + ) + + metadata.display_name = self._name_with_prefix(metadata.display_name) + reference = self._dashboard_reference(folder) + dashboard_id = self._install_state.dashboards.get(reference) + if dashboard_id is not None: try: - dashboard_id = self._install_state.dashboards[name] - logger.info(f"Updating dashboard with id={dashboard_id}") - updated_dashboard = self._ws.lakeview.update( - dashboard_id, - display_name=self._name_with_prefix(name), - serialized_dashboard=dashboard_data, - ) - return updated_dashboard - except InvalidParameterValue: - del self._install_state.dashboards[name] - logger.warning(f"Dashboard {name} does not exist anymore for some reason.") - return self._update_or_create_dashboard(name, dashboard_data, dashboard_file) - logger.info(f"Creating new dashboard {name}") - new_dashboard = self._ws.lakeview.create( - display_name=self._name_with_prefix(name), - parent_path=self._install_state.install_folder(), - serialized_dashboard=dashboard_data, + dashboard_id = self._handle_existing_dashboard(dashboard_id, metadata.display_name) + except (NotFound, InvalidParameterValue): + logger.info(f"Recovering invalid dashboard: {metadata.display_name} ({dashboard_id})") + try: + dashboard_path = f"{ws_parent_path}/{metadata.display_name}.lvdash.json" + self._ws.workspace.delete(dashboard_path) # Cannot recreate dashboard if file still exists + logger.debug( + f"Deleted dangling dashboard {metadata.display_name} ({dashboard_id}): {dashboard_path}" + ) + except NotFound: + pass + dashboard_id = None # Recreate the dashboard if it's reference is corrupted (manually) + + dashboard = Dashboards(self._ws).create_dashboard( + metadata, + dashboard_id=dashboard_id, + parent_path=ws_parent_path, + warehouse_id=self._ws.config.warehouse_id, + publish=True, ) - assert new_dashboard.dashboard_id is not None - self._install_state.dashboards[name] = new_dashboard.dashboard_id - return new_dashboard - - def _substitute_params(self, dashboard_file: Traversable, parameters: dict[str, Any]) -> str: - if not parameters: - return dashboard_file.read_text() - - with dashboard_file.open() as f: - dashboard_data = json.load(f) - - for dataset in dashboard_data.get("datasets", []): - for param in dataset.get("parameters", []): - if param["keyword"] in parameters: - param["defaultSelection"] = { - "values": { - "dataType": "STRING", - "values": [ - {"value": parameters[param["keyword"]]}, - ], - }, - } - - return json.dumps(dashboard_data) + assert dashboard.dashboard_id is not None + self._install_state.dashboards[reference] = dashboard.dashboard_id + return dashboard def _name_with_prefix(self, name: str) -> str: prefix = self._installation.product() return f"[{prefix.upper()}] {name}" + + def _handle_existing_dashboard(self, dashboard_id: str, display_name: str) -> str | None: + dashboard = self._ws.lakeview.get(dashboard_id) + if dashboard.lifecycle_state is None: + raise NotFound(f"Dashboard life cycle state: {display_name} ({dashboard_id})") + if dashboard.lifecycle_state == LifecycleState.TRASHED: + logger.info(f"Recreating trashed dashboard: {display_name} ({dashboard_id})") + return None # Recreate the dashboard if it is trashed (manually) + return dashboard_id # Update the existing dashboard + + def _remove_deprecated_dashboards(self, valid_dashboard_refs: set[str]): + for ref, dashboard_id in self._install_state.dashboards.items(): + if ref not in valid_dashboard_refs: + try: + logger.info(f"Removing dashboard_id={dashboard_id}, as it is no longer needed.") + del self._install_state.dashboards[ref] + self._ws.lakeview.trash(dashboard_id) + except (InvalidParameterValue, NotFound): + logger.warning(f"Dashboard `{dashboard_id}` doesn't exist anymore for some reason.") + continue diff --git a/src/databricks/labs/remorph/deployment/installation.py b/src/databricks/labs/remorph/deployment/installation.py index ae6cd3d84..35aa6b200 100644 --- a/src/databricks/labs/remorph/deployment/installation.py +++ b/src/databricks/labs/remorph/deployment/installation.py @@ -2,16 +2,14 @@ from databricks.labs.blueprint.installation import Installation from databricks.labs.blueprint.tui import Prompts +from databricks.labs.blueprint.upgrades import Upgrades +from databricks.labs.blueprint.wheels import WheelsV2 from databricks.sdk import WorkspaceClient from databricks.sdk.errors import NotFound +from databricks.sdk.errors.platform import InvalidParameterValue from databricks.labs.remorph.config import RemorphConfigs from databricks.labs.remorph.deployment.recon import ReconDeployment -from databricks.labs.blueprint.wheels import WheelsV2 - -from databricks.sdk.errors.platform import InvalidParameterValue -from databricks.labs.blueprint.upgrades import Upgrades - logger = logging.getLogger("databricks.labs.remorph.install") @@ -48,6 +46,7 @@ def _upload_wheel(self): def install(self, config: RemorphConfigs): wheel_paths: list[str] = self._upload_wheel() if config.reconcile: + logger.info("Installing Remorph reconcile Metadata components.") self._recon_deployment.install(config.reconcile, wheel_paths) self._apply_upgrades() diff --git a/src/databricks/labs/remorph/deployment/recon.py b/src/databricks/labs/remorph/deployment/recon.py index 3bf93fa18..a9db3b569 100644 --- a/src/databricks/labs/remorph/deployment/recon.py +++ b/src/databricks/labs/remorph/deployment/recon.py @@ -4,6 +4,7 @@ from databricks.labs.blueprint.installation import Installation from databricks.labs.blueprint.installer import InstallState from databricks.labs.blueprint.wheels import ProductInfo +from databricks.labs.blueprint.wheels import find_project_root from databricks.sdk import WorkspaceClient from databricks.sdk.errors import InvalidParameterValue, NotFound @@ -17,7 +18,6 @@ _RECON_PREFIX = "Reconciliation" RECON_JOB_NAME = f"{_RECON_PREFIX} Runner" -RECON_METRICS_DASHBOARD_NAME = f"{_RECON_PREFIX} Metrics" class ReconDeployment: @@ -41,6 +41,7 @@ def __init__( def install(self, recon_config: ReconcileConfig | None, wheel_paths: list[str]): if not recon_config: + logger.warning("Recon Config is empty.") return logger.info("Installing reconcile components.") self._deploy_tables(recon_config) @@ -87,50 +88,21 @@ def _deploy_tables(self, recon_config: ReconcileConfig): def _deploy_dashboards(self, recon_config: ReconcileConfig): logger.info("Deploying reconciliation dashboards.") - self._deploy_recon_metrics_dashboard(RECON_METRICS_DASHBOARD_NAME, recon_config) - for dashboard_name, dashboard_id in self._get_deprecated_dashboards(): - try: - logger.info(f"Removing dashboard_id={dashboard_id}, as it is no longer needed.") - del self._install_state.dashboards[dashboard_name] - self._ws.lakeview.trash(dashboard_id) - except (InvalidParameterValue, NotFound): - logger.warning(f"Dashboard `{dashboard_name}` doesn't exist anymore for some reason.") - continue - - def _deploy_recon_metrics_dashboard(self, name: str, recon_config: ReconcileConfig): - dashboard_params = { - "catalog": recon_config.metadata_config.catalog, - "schema": recon_config.metadata_config.schema, - } - - reconcile_dashboard_path = "reconcile/dashboards/Remorph-Reconciliation.lvdash.json" - dashboard_file = files(databricks.labs.remorph.resources).joinpath(reconcile_dashboard_path) - logger.info(f"Creating Reconciliation Dashboard `{name}`") - self._dashboard_deployer.deploy(name, dashboard_file, parameters=dashboard_params) + dashboard_base_dir = find_project_root(__file__) / "src/databricks/labs/remorph/resources/reconcile/dashboards" + self._dashboard_deployer.deploy(dashboard_base_dir, recon_config) def _get_dashboards(self) -> list[tuple[str, str]]: - return [ - (dashboard_name, dashboard_id) - for dashboard_name, dashboard_id in self._install_state.dashboards.items() - if dashboard_name.startswith(_RECON_PREFIX) - ] - - def _get_deprecated_dashboards(self) -> list[tuple[str, str]]: - return [ - (dashboard_name, dashboard_id) - for dashboard_name, dashboard_id in self._install_state.dashboards.items() - if dashboard_name.startswith(_RECON_PREFIX) and dashboard_name != RECON_METRICS_DASHBOARD_NAME - ] + return list(self._install_state.dashboards.items()) def _remove_dashboards(self): logger.info("Removing reconciliation dashboards.") - for dashboard_name, dashboard_id in self._get_dashboards(): + for dashboard_ref, dashboard_id in self._get_dashboards(): try: - logger.info(f"Removing dashboard {dashboard_name} with dashboard_id={dashboard_id}.") - del self._install_state.dashboards[dashboard_name] + logger.info(f"Removing dashboard with id={dashboard_id}.") + del self._install_state.dashboards[dashboard_ref] self._ws.lakeview.trash(dashboard_id) except (InvalidParameterValue, NotFound): - logger.warning(f"Dashboard `{dashboard_name}` doesn't exist anymore for some reason.") + logger.warning(f"Dashboard with id={dashboard_id} doesn't exist anymore for some reason.") continue def _deploy_jobs(self, recon_config: ReconcileConfig, remorph_wheel_path: str): diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/Remorph-Reconciliation.lvdash.json b/src/databricks/labs/remorph/resources/reconcile/dashboards/Remorph-Reconciliation.lvdash.json deleted file mode 100644 index 3d529583c..000000000 --- a/src/databricks/labs/remorph/resources/reconcile/dashboards/Remorph-Reconciliation.lvdash.json +++ /dev/null @@ -1 +0,0 @@ -{"datasets":[{"name":"98d4d60d","displayName":"recon_main","query":"select \nmain.recon_id,\nmain.source_type,\nmain.report_type,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table, \nmetrics.run_metrics.status as status,\nmetrics.run_metrics.exception_message as exception,\nmetrics.recon_metrics.row_comparison.missing_in_source as missing_in_source,\nmetrics.recon_metrics.row_comparison.missing_in_target as missing_in_target,\nmetrics.recon_metrics.column_comparison.absolute_mismatch as absolute_mismatch,\nmetrics.recon_metrics.column_comparison.threshold_mismatch as threshold_mismatch,\nmetrics.recon_metrics.column_comparison.mismatch_columns as mismatch_columns,\nmetrics.recon_metrics.schema_comparison as schema_comparison,\nmetrics.run_metrics.run_by_user as executed_by,\nmain.start_ts as start_ts,\nmain.end_ts as end_ts\nfrom IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \nIDENTIFIER(:catalog || '.' || :schema || '.metrics' ) metrics\non main.recon_table_id = metrics.recon_table_id\norder by metrics.inserted_ts desc, main.recon_id, main.target_table.table_name","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"26df7b0f","displayName":"recon_details_schema","query":"with tmp as (select \nrecon_table_id,\ninserted_ts,\nexplode(data) as schema_data\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type='schema'\n) \nselect \nmain.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nschema_data['source_column'] as source_column,\nschema_data['source_datatype'] as source_datatype,\nschema_data['databricks_column'] as databricks_column,\nschema_data['databricks_datatype'] as databricks_datatype,\nschema_data['is_valid'] as is_valid\nfrom \nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \ntmp\non main.recon_table_id = tmp.recon_table_id\norder by tmp.inserted_ts desc, main.recon_id, main.target_table\n","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"f34c6502","displayName":"recon_details_pivot","query":"with tmp as (select recon_table_id, inserted_ts ,recon_type, explode(data) as data, \nrow_number() over(partition by recon_table_id,recon_type order by recon_table_id) as rn\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type != 'schema')\nselect main.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nrecon_type, key, value, rn\nfrom tmp\ninner join\nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\non main.recon_table_id = tmp.recon_table_id\nlateral view explode(data) exploded_data AS key, value\n","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"82e8fe4b","displayName":"failed_table","query":"select \nmain.recon_id,\nmain.source_type,\nmain.report_type,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table, \nmetrics.run_metrics.status as status,\nmetrics.run_metrics.run_by_user as executed_by,\nmain.start_ts as start_ts,\nmain.end_ts as end_ts,\ndate(main.start_ts) as start_date\nfrom IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \nIDENTIFIER(:catalog || '.' || :schema || '.metrics' ) metrics\non main.recon_table_id = metrics.recon_table_id\nwhere metrics.run_metrics.status = false\norder by metrics.inserted_ts desc, main.recon_id, main.target_table.table_name","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"6aae9526","displayName":"success_table","query":"select \nmain.recon_id,\nmain.source_type,\nmain.report_type,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table, \nmetrics.run_metrics.status as status,\nmetrics.run_metrics.run_by_user as executed_by,\nmain.start_ts as start_ts,\nmain.end_ts as end_ts,\ndate(main.start_ts) as start_date\nfrom IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \nIDENTIFIER(:catalog || '.' || :schema || '.metrics' ) metrics\non main.recon_table_id = metrics.recon_table_id\nwhere metrics.run_metrics.status = true\norder by metrics.inserted_ts desc, main.recon_id, main.target_table.table_name","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"fe962300","displayName":"aggregate_details","query":"with details_view AS (\n select\n recon_table_id,\n rule_id,\n recon_type,\n explode(data) as agg_details\n from\n IDENTIFIER(:catalog || '.' || :schema || '.aggregate_details' ) \n),\nmetrics_view AS (\n select\n recon_table_id,\n rule_id,\n recon_metrics,\n run_metrics\n from\n IDENTIFIER(:catalog || '.' || :schema || '.aggregate_metrics' ) \n)\nselect\n recon_id,\n source_table,\n target_table,\n recon_type,\n aggregate_type,\n rule as aggregate_column,\n source_value,\n target_value,\n zip_with(rule_group_by_columns, group_by_column_values, (groupby, value) -> concat_ws(':', trim(groupby), value)) as group_by_columns,\n coalesce(status, \"false\") as status\nFROM\n (\n select\n main.recon_id,\n main.source_table.`catalog` as source_catalog,\n main.source_table.`schema` as source_schema,\n main.source_table.table_name as source_table_name,\n IF(\n ISNULL(source_catalog),\n CONCAT_WS('.', source_schema, source_table_name),\n CONCAT_WS(\n '.',\n source_catalog,\n source_schema,\n source_table_name\n )\n ) AS source_table,\n main.target_table.`catalog` as target_catalog,\n main.target_table.`schema` as target_schema,\n main.target_table.table_name as target_table_name,\n CONCAT_WS(\n '.',\n target_catalog,\n target_schema,\n target_table_name\n ) AS target_table,\n dtl.recon_type,\n rul.rule_info.agg_type as aggregate_type,\n upper(rul.rule_info.agg_type) || CONCAT('(', rul.rule_info.agg_column, ')') as rule,\n concat_ws(\n '_',\n 'source',\n rul.rule_info.agg_type,\n rul.rule_info.agg_column\n ) as source_agg_column,\n dtl.agg_details [source_agg_column] as source_value,\n concat_ws(\n '_',\n 'target',\n rul.rule_info.agg_type,\n rul.rule_info.agg_column\n ) as target_agg_column,\n dtl.agg_details [target_agg_column] as target_value,\n split(rul.rule_info.group_by_columns, ',') as rule_group_by_columns,\n transform(rule_group_by_columns, colm -> \n coalesce(dtl.agg_details[concat('source_group_by_', trim(colm))],\n dtl.agg_details[concat('target_group_by_', trim(colm))])) as group_by_column_values,\n concat_ws(\n '_',\n 'match',\n rul.rule_info.agg_type,\n rul.rule_info.agg_column\n ) as status_column,\n dtl.agg_details [status_column] as status\n from\n metrics_view mtc\n inner join IDENTIFIER(:catalog || '.' || :schema || '.main') main ON main.recon_table_id = mtc.recon_table_id\n inner join details_view dtl ON mtc.recon_table_id = dtl.recon_table_id\n inner join IDENTIFIER(:catalog || '.' || :schema || '.aggregate_rules' ) rul ON mtc.rule_id = dtl.rule_id\n AND dtl.rule_id = rul.rule_id\n )\norder by\n recon_id;","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]},{"name":"b09ed583","displayName":"aggregate_summary","query":"select\n main.recon_id,\n main.source_type,\n main.source_table.`catalog` as source_catalog,\n main.source_table.`schema` as source_schema,\n main.source_table.table_name as source_table_name,\n IF(\n ISNULL(source_catalog),\n CONCAT_WS('.', source_schema, source_table_name),\n CONCAT_WS(\n '.',\n source_catalog,\n source_schema,\n source_table_name\n )\n ) AS source_table,\n main.target_table.`catalog` as target_catalog,\n main.target_table.`schema` as target_schema,\n main.target_table.table_name as target_table_name,\n CONCAT_WS(\n '.',\n target_catalog,\n target_schema,\n target_table_name\n ) AS target_table,\n upper(rules.rule_info.agg_type) || CONCAT('(', rules.rule_info.agg_column, ')') as aggregate_column,\n rules.rule_info.group_by_columns,\n metrics.run_metrics.status as status,\n metrics.run_metrics.exception_message as exception,\n metrics.recon_metrics.missing_in_source as missing_in_source,\n metrics.recon_metrics.missing_in_target as missing_in_target,\n metrics.recon_metrics.mismatch as mismatch,\n metrics.run_metrics.run_by_user as executed_by,\n main.start_ts as start_ts,\n main.end_ts as end_ts\nfrom\n IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\n inner join IDENTIFIER(:catalog || '.' || :schema || '.aggregate_metrics' ) metrics\n inner join IDENTIFIER(:catalog || '.' || :schema || '.aggregate_rules' ) rules\non main.recon_table_id = metrics.recon_table_id\nand rules.rule_id = metrics.rule_id\norder by\n metrics.inserted_ts desc,\n main.recon_id,\n main.target_table.table_name;","parameters":[{"displayName":"catalog","keyword":"catalog","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"remorph_pvn"}]}}},{"displayName":"schema","keyword":"schema","dataType":"STRING","defaultSelection":{"values":{"dataType":"STRING","values":[{"value":"reconcile"}]}}}]}],"pages":[{"name":"761f7442","displayName":"New Page","layout":[{"widget":{"name":"13443c6e","queries":[{"name":"main_query","query":{"datasetName":"98d4d60d","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"source_type","expression":"`source_type`"},{"name":"report_type","expression":"`report_type`"},{"name":"source_table","expression":"`source_table`"},{"name":"target_table","expression":"`target_table`"},{"name":"status","expression":"`status`"},{"name":"exception","expression":"`exception`"},{"name":"missing_in_source","expression":"`missing_in_source`"},{"name":"missing_in_target","expression":"`missing_in_target`"},{"name":"absolute_mismatch","expression":"`absolute_mismatch`"},{"name":"threshold_mismatch","expression":"`threshold_mismatch`"},{"name":"mismatch_columns","expression":"`mismatch_columns`"},{"name":"schema_comparison","expression":"`schema_comparison`"},{"name":"executed_by","expression":"`executed_by`"},{"name":"start_ts","expression":"`start_ts`"},{"name":"end_ts","expression":"`end_ts`"}],"disaggregated":true}}],"spec":{"version":1,"widgetType":"table","encodings":{"columns":[{"fieldName":"recon_id","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":0,"title":"recon_id","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"cellFormat":{"default":{"foregroundColor":null},"rules":[{"if":{"column":"status","fn":"=","literal":"true"},"value":{"foregroundColor":"#3BD973"}},{"if":{"column":"status","fn":"=","literal":"false"},"value":{"foregroundColor":"#E92828"}}]},"displayName":"recon_id"},{"fieldName":"source_type","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":1,"title":"source_type","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_type"},{"fieldName":"report_type","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":2,"title":"report_type","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"report_type"},{"fieldName":"source_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":5,"title":"source_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_table"},{"fieldName":"target_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":8,"title":"target_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"target_table"},{"fieldName":"status","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"boolean","displayAs":"boolean","visible":true,"order":9,"title":"status","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"status"},{"fieldName":"exception","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":10,"title":"exception","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"exception"},{"fieldName":"missing_in_source","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":11,"title":"missing_in_source","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"missing_in_source"},{"fieldName":"missing_in_target","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":12,"title":"missing_in_target","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"missing_in_target"},{"fieldName":"absolute_mismatch","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":13,"title":"absolute_mismatch","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"absolute_mismatch"},{"fieldName":"threshold_mismatch","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":14,"title":"threshold_mismatch","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"threshold_mismatch"},{"fieldName":"mismatch_columns","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":15,"title":"mismatch_columns","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"mismatch_columns"},{"fieldName":"schema_comparison","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"boolean","displayAs":"boolean","visible":true,"order":16,"title":"schema_comparison","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"schema_comparison"},{"fieldName":"executed_by","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":19,"title":"executed_by","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"executed_by"},{"fieldName":"start_ts","dateTimeFormat":"YYYY-MM-DD HH:mm:ss.SSS","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"datetime","displayAs":"datetime","visible":true,"order":20,"title":"start_ts","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"start_ts"},{"fieldName":"end_ts","dateTimeFormat":"YYYY-MM-DD HH:mm:ss.SSS","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"datetime","displayAs":"datetime","visible":true,"order":21,"title":"end_ts","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"end_ts"}]},"invisibleColumns":[{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_catalog","type":"string","displayAs":"string","order":3,"title":"source_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_schema","type":"string","displayAs":"string","order":4,"title":"source_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_catalog","type":"string","displayAs":"string","order":6,"title":"target_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_schema","type":"string","displayAs":"string","order":7,"title":"target_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_table_name","type":"string","displayAs":"string","order":17,"title":"source_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_table_name","type":"string","displayAs":"string","order":18,"title":"target_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false}],"allowHTMLByDefault":false,"itemsPerPage":100,"paginationSize":"default","condensed":true,"withRowNumber":true,"frame":{"showDescription":true,"description":"Summary Table","showTitle":false}}},"position":{"x":0,"y":11,"width":6,"height":7}},{"widget":{"name":"e402660d","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_recon_id","query":{"datasetName":"98d4d60d","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_recon_id","query":{"datasetName":"26df7b0f","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_id","query":{"datasetName":"f34c6502","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_recon_id"},{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_recon_id"},{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_id"}]},"frame":{"showTitle":true,"showDescription":false,"title":"Recon Id","description":""}}},"position":{"x":0,"y":5,"width":2,"height":2}},{"widget":{"name":"d0e33d71","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_target_table","query":{"datasetName":"98d4d60d","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"target_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_target_table","query":{"datasetName":"26df7b0f","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"target_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_target_table","query":{"datasetName":"f34c6502","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"target_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"target_table","displayName":"target_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_target_table"},{"fieldName":"target_table","displayName":"target_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_target_table"},{"fieldName":"target_table","displayName":"target_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_target_table"}]},"frame":{"showTitle":true,"showDescription":false,"title":"Traget Table Name","description":""}}},"position":{"x":2,"y":5,"width":2,"height":2}},{"widget":{"name":"7ef4730e","queries":[{"name":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table","query":{"datasetName":"98d4d60d","fields":[{"name":"source_table","expression":"`source_table`"},{"name":"source_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table","query":{"datasetName":"26df7b0f","fields":[{"name":"source_table","expression":"`source_table`"},{"name":"source_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table","query":{"datasetName":"f34c6502","fields":[{"name":"source_table","expression":"`source_table`"},{"name":"source_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"source_table","displayName":"source_table","queryName":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table"},{"fieldName":"source_table","displayName":"source_table","queryName":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table"},{"fieldName":"source_table","displayName":"source_table","queryName":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table"}]},"frame":{"showTitle":true,"showDescription":false,"title":"Source Table Name","description":""}}},"position":{"x":4,"y":5,"width":2,"height":2}},{"widget":{"name":"7ef9ccf9","queries":[{"name":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type","query":{"datasetName":"98d4d60d","fields":[{"name":"source_type","expression":"`source_type`"},{"name":"source_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"source_type","displayName":"source_type","queryName":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Source Type","description":"Applied only on summary table"}}},"position":{"x":0,"y":7,"width":2,"height":2}},{"widget":{"name":"17923bfb","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_type","query":{"datasetName":"f34c6502","fields":[{"name":"recon_type","expression":"`recon_type`"},{"name":"recon_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"recon_type","displayName":"recon_type","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_type"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Category","description":"Applied only on details table"}}},"position":{"x":3,"y":28,"width":3,"height":1}},{"widget":{"name":"7284be9e","queries":[{"name":"main_query","query":{"datasetName":"98d4d60d","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"absolute_mismatch","expression":"`absolute_mismatch`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"absolute_mismatch","scale":{"type":"quantitative"},"displayName":"absolute_mismatch"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"},"label":{"show":false}},"frame":{"showTitle":true,"title":"Mismatched Records","showDescription":false}}},"position":{"x":0,"y":48,"width":3,"height":6}},{"widget":{"name":"1e3e9ab9","queries":[{"name":"main_query","query":{"datasetName":"98d4d60d","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"missing_in_target","expression":"`missing_in_target`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"missing_in_target","scale":{"type":"quantitative"},"displayName":"missing_in_target"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"},"label":{"show":false}},"frame":{"showTitle":true,"title":"Missing in Source"}}},"position":{"x":3,"y":54,"width":3,"height":6}},{"widget":{"name":"0484cb19","queries":[{"name":"main_query","query":{"datasetName":"98d4d60d","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"missing_in_source","expression":"`missing_in_source`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"missing_in_source","scale":{"type":"quantitative"},"displayName":"missing_in_source"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"},"label":{"show":false}},"frame":{"showTitle":true,"title":"Missing in Databricks"}}},"position":{"x":0,"y":54,"width":3,"height":6}},{"widget":{"name":"5af719f5","queries":[{"name":"main_query","query":{"datasetName":"98d4d60d","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"threshold_mismatch","expression":"`threshold_mismatch`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"threshold_mismatch","scale":{"type":"quantitative"},"displayName":"threshold_mismatch"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"}},"frame":{"showTitle":true,"title":"Threshold Mismatches"}}},"position":{"x":3,"y":48,"width":3,"height":6}},{"widget":{"name":"cc162059","queries":[{"name":"main_query","query":{"datasetName":"26df7b0f","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"source_table","expression":"`source_table`"},{"name":"target_table","expression":"`target_table`"},{"name":"source_column","expression":"`source_column`"},{"name":"source_datatype","expression":"`source_datatype`"},{"name":"databricks_column","expression":"`databricks_column`"},{"name":"databricks_datatype","expression":"`databricks_datatype`"},{"name":"is_valid","expression":"`is_valid`"}],"disaggregated":true}}],"spec":{"version":1,"widgetType":"table","encodings":{"columns":[{"fieldName":"recon_id","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":0,"title":"recon_id","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"cellFormat":{"default":{"foregroundColor":null},"rules":[{"if":{"column":"is_valid","fn":"=","literal":"false"},"value":{"foregroundColor":"#E92828"}},{"if":{"column":"is_valid","fn":"=","literal":"true"},"value":{"foregroundColor":"#3BD973"}}]},"displayName":"recon_id"},{"fieldName":"source_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":4,"title":"source_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_table"},{"fieldName":"target_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":8,"title":"target_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"target_table"},{"fieldName":"source_column","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":9,"title":"source_column","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_column"},{"fieldName":"source_datatype","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":10,"title":"source_datatype","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_datatype"},{"fieldName":"databricks_column","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":11,"title":"databricks_column","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"databricks_column"},{"fieldName":"databricks_datatype","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":12,"title":"databricks_datatype","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"databricks_datatype"},{"fieldName":"is_valid","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":13,"title":"is_valid","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"is_valid"}]},"invisibleColumns":[{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_table_name","type":"string","displayAs":"string","order":1,"title":"source_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_catalog","type":"string","displayAs":"string","order":2,"title":"source_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_schema","type":"string","displayAs":"string","order":3,"title":"source_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_catalog","type":"string","displayAs":"string","order":5,"title":"target_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_schema","type":"string","displayAs":"string","order":6,"title":"target_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_table_name","type":"string","displayAs":"string","order":7,"title":"target_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false}],"allowHTMLByDefault":false,"itemsPerPage":25,"paginationSize":"default","condensed":true,"withRowNumber":false,"frame":{"showTitle":true,"title":"Schema Details"}}},"position":{"x":0,"y":20,"width":6,"height":6}},{"widget":{"name":"46f02359","queries":[{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_catalog","query":{"datasetName":"98d4d60d","parameters":[{"name":"catalog","keyword":"catalog"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_catalog","query":{"datasetName":"26df7b0f","parameters":[{"name":"catalog","keyword":"catalog"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_catalog","query":{"datasetName":"f34c6502","parameters":[{"name":"catalog","keyword":"catalog"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"parameterName":"catalog","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_catalog"},{"parameterName":"catalog","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_catalog"},{"parameterName":"catalog","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_catalog"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Catalog","description":"’remorph' is the default catalog for the metrics table; specify a different catalog name if the metrics are stored elsewhere."}}},"position":{"x":0,"y":3,"width":3,"height":2}},{"widget":{"name":"58eff8fb","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_report_type","query":{"datasetName":"98d4d60d","fields":[{"name":"report_type","expression":"`report_type`"},{"name":"report_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"report_type","displayName":"report_type","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_report_type"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Report","description":"Applied only on summary table"}}},"position":{"x":2,"y":7,"width":2,"height":2}},{"widget":{"name":"f7560564","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_start_ts","query":{"datasetName":"98d4d60d","fields":[{"name":"start_ts","expression":"`start_ts`"},{"name":"start_ts_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-date-range-picker","encodings":{"fields":[{"fieldName":"start_ts","displayName":"start_ts","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_start_ts"}]},"frame":{"showTitle":true,"showDescription":true,"title":"started at","description":"Applied on all tables"}}},"position":{"x":0,"y":9,"width":6,"height":2}},{"widget":{"name":"352afec2","textbox_spec":"# Main Reconciliation Table\n### This table provides comprehensive information on the report's status, including failure indications, schema matching status, and details on missing and mismatched records.\n\n[Aggregates Reconciliation Table](#aggregates-reconcile-table-metrics)"},"position":{"x":0,"y":0,"width":6,"height":3}},{"widget":{"name":"c9fadeb6","textbox_spec":"# Drill Down\n### The details table contains all the sample records for mismatches and missing entries, providing users with exact details to pinpoint the issues."},"position":{"x":0,"y":26,"width":6,"height":2}},{"widget":{"name":"0314c7b5","textbox_spec":"# Schema Comparison Details\n### This table provides a detailed view of schema mismatches."},"position":{"x":0,"y":18,"width":6,"height":2}},{"widget":{"name":"e1f06373","textbox_spec":"# Visualization of Missing and Mismatched Records"},"position":{"x":0,"y":46,"width":6,"height":2}},{"widget":{"name":"d60aeec0","queries":[{"name":"main_query","query":{"datasetName":"f34c6502","fields":[{"name":"value","expression":"`value`"},{"name":"key","expression":"`key`"},{"name":"recon_id","expression":"`recon_id`"},{"name":"source_table","expression":"`source_table`"},{"name":"target_table","expression":"`target_table`"},{"name":"recon_type","expression":"`recon_type`"},{"name":"rn","expression":"`rn`"}],"disaggregated":false,"orders":[{"direction":"ASC","expression":"`recon_id`"},{"direction":"ASC","expression":"`source_table`"},{"direction":"ASC","expression":"`target_table`"},{"direction":"ASC","expression":"`recon_type`"},{"direction":"ASC","expression":"`rn`"},{"direction":"ASC","expression":"`key`"}]}}],"spec":{"version":3,"widgetType":"pivot","encodings":{"rows":[{"fieldName":"recon_id","displayName":"recon_id"},{"fieldName":"source_table","displayName":"source_table"},{"fieldName":"target_table","displayName":"target_table"},{"fieldName":"recon_type","displayName":"recon_type"},{"fieldName":"rn","displayName":"rn"}],"columns":[{"fieldName":"key","displayName":"key"}],"cell":{"fieldName":"value","cellType":"text","displayName":"value"}}}},"position":{"x":0,"y":30,"width":6,"height":6}},{"widget":{"name":"54bece92","queries":[{"name":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table","query":{"datasetName":"f34c6502","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"target_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"target_table","displayName":"target_table","queryName":"dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table"}]},"frame":{"showDescription":true,"showTitle":true,"title":"Target Table Name","description":"Applied only on details table"}}},"position":{"x":0,"y":29,"width":3,"height":1}},{"widget":{"name":"799c7327","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_source_table","query":{"datasetName":"f34c6502","fields":[{"name":"source_table","expression":"`source_table`"},{"name":"source_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"source_table","displayName":"source_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_source_table"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Source Table Name","description":"Applied only on details table"}}},"position":{"x":3,"y":29,"width":3,"height":1}},{"widget":{"name":"3b52a758","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_id","query":{"datasetName":"f34c6502","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_recon_id"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Recon Id","description":"Applied only on details table"}}},"position":{"x":0,"y":28,"width":3,"height":1}},{"widget":{"name":"e66542d5","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_executed_by","query":{"datasetName":"98d4d60d","fields":[{"name":"executed_by","expression":"`executed_by`"},{"name":"executed_by_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"executed_by","displayName":"executed_by","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_executed_by"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Executed by","description":"Applied only on summary table"}}},"position":{"x":4,"y":7,"width":2,"height":2}},{"widget":{"name":"e4de4cd6","queries":[{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_schema","query":{"datasetName":"98d4d60d","parameters":[{"name":"schema","keyword":"schema"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_schema","query":{"datasetName":"26df7b0f","parameters":[{"name":"schema","keyword":"schema"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_schema","query":{"datasetName":"f34c6502","parameters":[{"name":"schema","keyword":"schema"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"parameterName":"schema","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b5281d61b39c76b69f8fe569_schema"},{"parameterName":"schema","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52c13cf94eefbdd06f2cbae_schema"},{"parameterName":"schema","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef58b3b52f1b8889cb1781ce4731b8_schema"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Schema","description":"’reconcile’ is the default schema for the metrics table; specify a different schema name if the metrics are stored elsewhere."}}},"position":{"x":3,"y":3,"width":3,"height":2}},{"widget":{"name":"5e4e9439","textbox_spec":"# Daily Data Validation Issues Report\n### This summary report provides an overview of all data validation runs conducted on a specific day. It highlights whether each table has encountered any validation issues, without delving into the low-level details. This report aims to give a quick and clear status of data integrity across all tables for the day."},"position":{"x":0,"y":36,"width":6,"height":2}},{"widget":{"name":"67bf4362","queries":[{"name":"main_query","query":{"datasetName":"82e8fe4b","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"countdistinct(recon_id)","expression":"COUNT(DISTINCT `recon_id`)"}],"disaggregated":false}}],"spec":{"version":3,"widgetType":"bar","encodings":{"x":{"fieldName":"target_table","scale":{"type":"categorical","sort":{"by":"y-reversed"}},"displayName":"target_table"},"y":{"fieldName":"countdistinct(recon_id)","scale":{"type":"quantitative"},"displayName":"Count of Unique recon_id"},"label":{"show":true}},"frame":{"showDescription":false,"title":"Number of Distinct Recon IDs per Target Table Failed","showTitle":true}}},"position":{"x":0,"y":40,"width":5,"height":6}},{"widget":{"name":"dcd55896","queries":[{"name":"main_query","query":{"datasetName":"82e8fe4b","fields":[{"name":"countdistinct(target_table)","expression":"COUNT(DISTINCT `target_table`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"counter","encodings":{"value":{"fieldName":"countdistinct(target_table)","displayName":"Count of Unique target_table"}},"frame":{"title":"Unique target tables failed","showTitle":true,"showDescription":false,"description":"Unique target tables failed"}}},"position":{"x":5,"y":42,"width":1,"height":2}},{"widget":{"name":"a3e6e303","queries":[{"name":"main_query","query":{"datasetName":"82e8fe4b","fields":[{"name":"countdistinct(recon_id)","expression":"COUNT(DISTINCT `recon_id`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"counter","encodings":{"value":{"fieldName":"countdistinct(recon_id)","displayName":"Count of Unique recon_id"}},"frame":{"title":"Total number of runs failed","showTitle":true}}},"position":{"x":5,"y":40,"width":1,"height":2}},{"widget":{"name":"ab88e382","queries":[{"name":"main_query","query":{"datasetName":"6aae9526","fields":[{"name":"countdistinct(target_table)","expression":"COUNT(DISTINCT `target_table`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"counter","encodings":{"value":{"fieldName":"countdistinct(target_table)","displayName":"Count of Unique target_table"}},"frame":{"showTitle":true,"title":"Unique target tables success"}}},"position":{"x":5,"y":44,"width":1,"height":2}},{"widget":{"name":"bec1ff72","queries":[{"name":"dashboards/01ef2274f35812e19ad423f93c8e062b/datasets/01ef3c63c5c6160687607291378cdcfc_start_date","query":{"datasetName":"82e8fe4b","fields":[{"name":"start_date","expression":"`start_date`"},{"name":"start_date_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef2274f35812e19ad423f93c8e062b/datasets/01ef3cdefe09194f96a029cc72193b8f_start_date","query":{"datasetName":"6aae9526","fields":[{"name":"start_date","expression":"`start_date`"},{"name":"start_date_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-date-range-picker","encodings":{"fields":[{"fieldName":"start_date","displayName":"start_date","queryName":"dashboards/01ef2274f35812e19ad423f93c8e062b/datasets/01ef3c63c5c6160687607291378cdcfc_start_date"},{"fieldName":"start_date","displayName":"start_date","queryName":"dashboards/01ef2274f35812e19ad423f93c8e062b/datasets/01ef3cdefe09194f96a029cc72193b8f_start_date"}]}}},"position":{"x":0,"y":38,"width":6,"height":2}},{"widget":{"name":"6ce1e8e3","queries":[{"name":"main_query","query":{"datasetName":"fe962300","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"source_table","expression":"`source_table`"},{"name":"target_table","expression":"`target_table`"},{"name":"recon_type","expression":"`recon_type`"},{"name":"aggregate_column","expression":"`aggregate_column`"},{"name":"source_value","expression":"`source_value`"},{"name":"target_value","expression":"`target_value`"},{"name":"group_by_columns","expression":"`group_by_columns`"},{"name":"status","expression":"`status`"}],"disaggregated":true}}],"spec":{"version":1,"widgetType":"table","encodings":{"columns":[{"fieldName":"recon_id","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":0,"title":"recon_id","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"cellFormat":{"default":{"foregroundColor":null},"rules":[]},"displayName":"recon_id"},{"fieldName":"source_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":1,"title":"source_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_table"},{"fieldName":"target_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":2,"title":"target_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"target_table"},{"fieldName":"recon_type","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":3,"title":"recon_type","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"recon_type"},{"fieldName":"aggregate_column","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":4,"title":"aggregate_column","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"aggregate_column"},{"fieldName":"source_value","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":6,"title":"source_value","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_value"},{"fieldName":"target_value","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":7,"title":"target_value","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"target_value"},{"fieldName":"group_by_columns","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"complex","displayAs":"string","visible":true,"order":8,"title":"group_by_columns","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"group_by_columns"},{"fieldName":"status","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":9,"title":"status","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"status"}]},"invisibleColumns":[{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"aggregate_type","type":"string","displayAs":"string","order":5,"title":"aggregate_type","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false}],"allowHTMLByDefault":false,"itemsPerPage":25,"paginationSize":"default","condensed":true,"withRowNumber":false,"frame":{"showTitle":true,"title":"Aggregates Reconcile Details"}}},"position":{"x":0,"y":80,"width":6,"height":5}},{"widget":{"name":"4a046d4e","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_source_type","query":{"datasetName":"b09ed583","fields":[{"name":"source_type","expression":"`source_type`"},{"name":"source_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"source_type","displayName":"source_type","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_source_type"}]},"frame":{"showTitle":true,"title":"Source Type","showDescription":true,"description":"applied only on Aggregates Summary Table"}}},"position":{"x":0,"y":68,"width":3,"height":1}},{"widget":{"name":"c3d2deb0","textbox_spec":"# Aggregates Reconcile Table Metrics\n\n#### It provides the following information:\n* Mismatch\n* Missing in Source\n* Missing in Target\n"},"position":{"x":0,"y":61,"width":6,"height":3}},{"widget":{"name":"c61e2d74","textbox_spec":"----"},"position":{"x":0,"y":60,"width":6,"height":1}},{"widget":{"name":"9af38585","queries":[{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_catalog","query":{"datasetName":"b09ed583","parameters":[{"name":"catalog","keyword":"catalog"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_catalog","query":{"datasetName":"fe962300","parameters":[{"name":"catalog","keyword":"catalog"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"parameterName":"catalog","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_catalog"},{"parameterName":"catalog","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_catalog"}]},"frame":{"showTitle":true,"showDescription":true,"description":"’remorph' is the default catalog for the metrics table; specify a different catalog name if the metrics are stored elsewhere.","title":"catalog"}}},"position":{"x":0,"y":64,"width":3,"height":2}},{"widget":{"name":"5427812b","queries":[{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_schema","query":{"datasetName":"b09ed583","parameters":[{"name":"schema","keyword":"schema"}],"disaggregated":false}},{"name":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_schema","query":{"datasetName":"fe962300","parameters":[{"name":"schema","keyword":"schema"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"parameterName":"schema","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_schema"},{"parameterName":"schema","queryName":"parameter_dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_schema"}]},"frame":{"showTitle":true,"showDescription":true,"description":"’reconcile’ is the default schema for the metrics table; specify a different schema name if the metrics are stored elsewhere.","title":"schema"}}},"position":{"x":3,"y":64,"width":3,"height":2}},{"widget":{"name":"11c99a4e","queries":[{"name":"main_query","query":{"datasetName":"b09ed583","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"source_type","expression":"`source_type`"},{"name":"source_table","expression":"`source_table`"},{"name":"target_table","expression":"`target_table`"},{"name":"aggregate_column","expression":"`aggregate_column`"},{"name":"group_by_columns","expression":"`group_by_columns`"},{"name":"status","expression":"`status`"},{"name":"exception","expression":"`exception`"},{"name":"missing_in_source","expression":"`missing_in_source`"},{"name":"missing_in_target","expression":"`missing_in_target`"},{"name":"mismatch","expression":"`mismatch`"},{"name":"executed_by","expression":"`executed_by`"},{"name":"start_ts","expression":"`start_ts`"},{"name":"end_ts","expression":"`end_ts`"}],"disaggregated":true}}],"spec":{"version":1,"widgetType":"table","encodings":{"columns":[{"fieldName":"recon_id","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100000,"title":"recon_id","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"cellFormat":{"default":{"foregroundColor":null},"rules":[{"if":{"column":"status","fn":"=","literal":"true"},"value":{"foregroundColor":"#3BD973"}},{"if":{"column":"status","fn":"=","literal":"false"},"value":{"foregroundColor":"#E92828"}}]},"displayName":"recon_id"},{"fieldName":"source_type","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100001,"title":"source_type","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_type"},{"fieldName":"source_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100005,"title":"source_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"source_table"},{"fieldName":"target_table","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100009,"title":"target_table","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"target_table"},{"fieldName":"aggregate_column","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100010,"title":"aggregate_column","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"aggregate_column"},{"fieldName":"group_by_columns","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100011,"title":"group_by_columns","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"group_by_columns"},{"fieldName":"status","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"boolean","displayAs":"boolean","visible":true,"order":100012,"title":"status","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"status"},{"fieldName":"exception","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100013,"title":"exception","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"exception"},{"fieldName":"missing_in_source","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":100014,"title":"missing_in_source","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"missing_in_source"},{"fieldName":"missing_in_target","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":100015,"title":"missing_in_target","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"missing_in_target"},{"fieldName":"mismatch","numberFormat":"0","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"integer","displayAs":"number","visible":true,"order":100016,"title":"mismatch","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"mismatch"},{"fieldName":"executed_by","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"string","displayAs":"string","visible":true,"order":100017,"title":"executed_by","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"executed_by"},{"fieldName":"start_ts","dateTimeFormat":"DD/MM/YY HH:mm:ss.SSS","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"datetime","displayAs":"datetime","visible":true,"order":100018,"title":"start_ts","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"start_ts"},{"fieldName":"end_ts","dateTimeFormat":"DD/MM/YY HH:mm:ss.SSS","booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"type":"datetime","displayAs":"datetime","visible":true,"order":100019,"title":"end_ts","allowSearch":false,"alignContent":"right","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false,"displayName":"end_ts"}]},"invisibleColumns":[{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_catalog","type":"string","displayAs":"string","order":100002,"title":"source_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_schema","type":"string","displayAs":"string","order":100003,"title":"source_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"source_table_name","type":"string","displayAs":"string","order":100004,"title":"source_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_catalog","type":"string","displayAs":"string","order":100006,"title":"target_catalog","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_schema","type":"string","displayAs":"string","order":100007,"title":"target_schema","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false},{"booleanValues":["false","true"],"imageUrlTemplate":"{{ @ }}","imageTitleTemplate":"{{ @ }}","imageWidth":"","imageHeight":"","linkUrlTemplate":"{{ @ }}","linkTextTemplate":"{{ @ }}","linkTitleTemplate":"{{ @ }}","linkOpenInNewTab":true,"name":"target_table_name","type":"string","displayAs":"string","order":100008,"title":"target_table_name","allowSearch":false,"alignContent":"left","allowHTML":false,"highlightLinks":false,"useMonospaceFont":false,"preserveWhitespace":false}],"allowHTMLByDefault":false,"itemsPerPage":25,"paginationSize":"default","condensed":true,"withRowNumber":true,"frame":{"showTitle":true,"showDescription":false,"title":"Aggregates Summary Table"}}},"position":{"x":0,"y":70,"width":6,"height":6}},{"widget":{"name":"76185bc5","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_recon_id","query":{"datasetName":"b09ed583","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}},{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_id","query":{"datasetName":"fe962300","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_recon_id"},{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_id"}]},"frame":{"showTitle":true,"title":"Recon Id"}}},"position":{"x":0,"y":66,"width":2,"height":2}},{"widget":{"name":"b31c988a","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_target_table_name","query":{"datasetName":"b09ed583","fields":[{"name":"target_table_name","expression":"`target_table_name`"},{"name":"target_table_name_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"target_table_name","displayName":"target_table_name","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_target_table_name"}]},"frame":{"showTitle":true,"title":"Target Table Name"}}},"position":{"x":2,"y":66,"width":2,"height":2}},{"widget":{"name":"6e647a85","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_source_table_name","query":{"datasetName":"b09ed583","fields":[{"name":"source_table_name","expression":"`source_table_name`"},{"name":"source_table_name_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"source_table_name","displayName":"source_table_name","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_source_table_name"}]},"frame":{"showTitle":true,"title":"Source Table Name"}}},"position":{"x":4,"y":66,"width":2,"height":2}},{"widget":{"name":"2c0972dc","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_executed_by","query":{"datasetName":"b09ed583","fields":[{"name":"executed_by","expression":"`executed_by`"},{"name":"executed_by_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"executed_by","displayName":"executed_by","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_executed_by"}]},"frame":{"showTitle":true,"title":"Executed by","showDescription":true,"description":"applied only on Aggregates Summary Table"}}},"position":{"x":3,"y":68,"width":3,"height":1}},{"widget":{"name":"a9d993fc","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_aggregate_type","query":{"datasetName":"fe962300","fields":[{"name":"aggregate_type","expression":"`aggregate_type`"},{"name":"aggregate_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-multi-select","encodings":{"fields":[{"fieldName":"aggregate_type","displayName":"aggregate_type","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_aggregate_type"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Aggregate","description":"applied only on Aggregates Details Table"}}},"position":{"x":4,"y":78,"width":2,"height":1}},{"widget":{"name":"3e767f78","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_start_ts","query":{"datasetName":"b09ed583","fields":[{"name":"start_ts","expression":"`start_ts`"},{"name":"start_ts_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-date-range-picker","encodings":{"fields":[{"fieldName":"start_ts","displayName":"start_ts","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef59750f6b1b91910b8c012f111b66_start_ts"}]},"frame":{"showTitle":true,"showDescription":true,"title":"started at","description":"applied on all Tables"}}},"position":{"x":0,"y":69,"width":6,"height":1}},{"widget":{"name":"8f464f29","textbox_spec":"# Drill Down\n### The Aggregates Reconcile details table contains all the sample records information of mismatches and missing entries."},"position":{"x":0,"y":76,"width":6,"height":2}},{"widget":{"name":"40a8f92a","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_id","query":{"datasetName":"fe962300","fields":[{"name":"recon_id","expression":"`recon_id`"},{"name":"recon_id_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"recon_id","displayName":"recon_id","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_id"}]},"frame":{"showTitle":true,"title":"Recon Id","showDescription":true,"description":"applied only on Aggregates Details Table"}}},"position":{"x":0,"y":78,"width":2,"height":1}},{"widget":{"name":"5dca0363","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_type","query":{"datasetName":"fe962300","fields":[{"name":"recon_type","expression":"`recon_type`"},{"name":"recon_type_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"recon_type","displayName":"recon_type","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_recon_type"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Category","description":"applied only on Aggregates Details Table"}}},"position":{"x":2,"y":78,"width":2,"height":1}},{"widget":{"name":"5035f2ed","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_target_table","query":{"datasetName":"fe962300","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"target_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"target_table","displayName":"target_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_target_table"}]},"frame":{"showTitle":true,"title":"Target Table Name","showDescription":true,"description":"applied only on Aggregates Details Table"}}},"position":{"x":0,"y":79,"width":3,"height":1}},{"widget":{"name":"49382af5","queries":[{"name":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_source_table","query":{"datasetName":"fe962300","fields":[{"name":"source_table","expression":"`source_table`"},{"name":"source_table_associativity","expression":"COUNT_IF(`associative_filter_predicate_group`)"}],"disaggregated":false}}],"spec":{"version":2,"widgetType":"filter-single-select","encodings":{"fields":[{"fieldName":"source_table","displayName":"source_table","queryName":"dashboards/01ef58b3b5251415a858a136df5abc97/datasets/01ef5938deff10849a2f55ccbba93f70_source_table"}]},"frame":{"showTitle":true,"showDescription":true,"title":"Source Table Name","description":"applied only on Aggregates Details Table"}}},"position":{"x":3,"y":79,"width":3,"height":1}},{"widget":{"name":"c2e89c61","textbox_spec":"# Visualization of Missing and Mismatched Records"},"position":{"x":0,"y":85,"width":6,"height":2}},{"widget":{"name":"44b7b58a","queries":[{"name":"main_query","query":{"datasetName":"b09ed583","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"mismatch","expression":"`mismatch`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"mismatch","scale":{"type":"quantitative"},"displayName":"mismatch"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"}},"frame":{"showTitle":true,"title":"Mismatched Records"}}},"position":{"x":0,"y":87,"width":6,"height":6}},{"widget":{"name":"53b16425","queries":[{"name":"main_query","query":{"datasetName":"b09ed583","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"missing_in_target","expression":"`missing_in_target`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"missing_in_target","scale":{"type":"quantitative"},"displayName":"missing_in_target"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"}},"frame":{"showTitle":true,"title":"Missing in Databricks"}}},"position":{"x":0,"y":93,"width":3,"height":6}},{"widget":{"name":"54ac1e58","queries":[{"name":"main_query","query":{"datasetName":"b09ed583","fields":[{"name":"target_table","expression":"`target_table`"},{"name":"hourly(start_ts)","expression":"DATE_TRUNC(\"HOUR\", `start_ts`)"},{"name":"missing_in_source","expression":"`missing_in_source`"}],"disaggregated":true}}],"spec":{"version":3,"widgetType":"area","encodings":{"x":{"fieldName":"hourly(start_ts)","scale":{"type":"temporal"},"displayName":"start_ts"},"y":{"fieldName":"missing_in_source","scale":{"type":"quantitative"},"displayName":"missing_in_source"},"color":{"fieldName":"target_table","scale":{"type":"categorical"},"displayName":"target_table"}},"frame":{"showTitle":true,"title":"Missing in Source"}}},"position":{"x":3,"y":93,"width":3,"height":6}}]}]} \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/00_0_aggregate_recon_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/00_0_aggregate_recon_header.md new file mode 100644 index 000000000..043ef2354 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/00_0_aggregate_recon_header.md @@ -0,0 +1,6 @@ +# Aggregates Reconcile Table Metrics +### It provides the following information: + +* Mismatch +* Missing in Source +* Missing in Target diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_0_recon_id.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_0_recon_id.filter.yml new file mode 100644 index 000000000..609dd9aca --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_0_recon_id.filter.yml @@ -0,0 +1,6 @@ +columns: +- recon_id +- dd_recon_id +type: MULTI_SELECT +title: Recon Id +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_1_executed_by.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_1_executed_by.filter.yml new file mode 100644 index 000000000..7a3b36f6c --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_1_executed_by.filter.yml @@ -0,0 +1,5 @@ +columns: +- executed_by +type: MULTI_SELECT +title: Executed by +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_2_started_at.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_2_started_at.filter.yml new file mode 100644 index 000000000..3d492465c --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/01_2_started_at.filter.yml @@ -0,0 +1,5 @@ +columns: +- start_ts +title: Started At +type: DATE_RANGE_PICKER +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_0_source_type.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_0_source_type.filter.yml new file mode 100644 index 000000000..427a7b01e --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_0_source_type.filter.yml @@ -0,0 +1,5 @@ +columns: +- source_type +type: MULTI_SELECT +title: Source Type +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_1_source_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_1_source_table.filter.yml new file mode 100644 index 000000000..bd7632fec --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_1_source_table.filter.yml @@ -0,0 +1,5 @@ +columns: +- source_table +type: MULTI_SELECT +title: Source Table Name +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_2_target_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_2_target_table.filter.yml new file mode 100644 index 000000000..8cec7edfd --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/02_2_target_table.filter.yml @@ -0,0 +1,5 @@ +columns: +- target_table +type: MULTI_SELECT +title: Target Table Name +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/04_0_aggregate_summary_table.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/04_0_aggregate_summary_table.sql new file mode 100644 index 000000000..a35c69956 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/04_0_aggregate_summary_table.sql @@ -0,0 +1,46 @@ +/* --title 'Aggregates Summary Table' --width 6 --height 6 */ +SELECT + main.recon_id, + main.source_type, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name, + IF( + ISNULL(source_catalog), + CONCAT_WS('.', source_schema, source_table_name), + CONCAT_WS( + '.', + source_catalog, + source_schema, + source_table_name + ) + ) AS source_table, + main.target_table.`catalog` AS target_catalog, + main.target_table.`schema` AS target_schema, + main.target_table.table_name AS target_table_name, + CONCAT_WS( + '.', + target_catalog, + target_schema, + target_table_name + ) AS target_table, + UPPER(rules.rule_info.agg_type) || CONCAT('(', rules.rule_info.agg_column, ')') AS aggregate_column, + rules.rule_info.group_by_columns, + metrics.run_metrics.status AS status, + metrics.run_metrics.exception_message AS exception, + metrics.recon_metrics.missing_in_source AS missing_in_source, + metrics.recon_metrics.missing_in_target AS missing_in_target, + metrics.recon_metrics.mismatch AS mismatch, + metrics.run_metrics.run_by_user AS executed_by, + main.start_ts AS start_ts, + main.end_ts AS end_ts +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.aggregate_metrics metrics + INNER JOIN remorph.reconcile.aggregate_rules rules + ON main.recon_table_id = metrics.recon_table_id + AND rules.rule_id = metrics.rule_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/05_0_aggregate_recon_drilldown_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/05_0_aggregate_recon_drilldown_header.md new file mode 100644 index 000000000..80ab14eed --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/05_0_aggregate_recon_drilldown_header.md @@ -0,0 +1,2 @@ +# Drill Down +### The Aggregates Reconcile details table contains all the sample records information of mismatches and missing entries. \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_0_recon_id.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_0_recon_id.filter.yml new file mode 100644 index 000000000..93cd7e61d --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_0_recon_id.filter.yml @@ -0,0 +1,5 @@ +columns: +- dd_recon_id +type: MULTI_SELECT +title: Recon Id +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_1_category.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_1_category.filter.yml new file mode 100644 index 000000000..3315ef7f9 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_1_category.filter.yml @@ -0,0 +1,5 @@ +columns: +- dd_recon_type +type: MULTI_SELECT +title: Category +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_2_aggregate_type.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_2_aggregate_type.filter.yml new file mode 100644 index 000000000..4be0a0e3f --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/06_2_aggregate_type.filter.yml @@ -0,0 +1,5 @@ +columns: +- dd_aggregate_type +type: MULTI_SELECT +title: Aggregate Type +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_0_target_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_0_target_table.filter.yml new file mode 100644 index 000000000..61888e08e --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_0_target_table.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_target_table +type: MULTI_SELECT +title: Target Table Name \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_1_source_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_1_source_table.filter.yml new file mode 100644 index 000000000..31366f2b0 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/07_1_source_table.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_source_table +type: MULTI_SELECT +title: Source Table Name \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/08_0_aggregate_details_table.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/08_0_aggregate_details_table.sql new file mode 100644 index 000000000..ee3f70b1b --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/08_0_aggregate_details_table.sql @@ -0,0 +1,92 @@ +/* --title 'Aggregates Reconciliation Details' --width 6 --height 6 */ +WITH details_view AS ( + SELECT + recon_table_id, + rule_id, + recon_type, + explode(data) AS agg_details + FROM + remorph.reconcile.aggregate_details +), + metrics_view AS ( + SELECT + recon_table_id, + rule_id, + recon_metrics, + run_metrics + FROM + remorph.reconcile.aggregate_metrics + ) +SELECT + recon_id AS dd_recon_id, + source_table AS dd_source_table, + target_table AS dd_target_table, + recon_type AS dd_recon_type, + aggregate_type AS dd_aggregate_type, + rule AS aggregate_column, + source_value, + target_value, + zip_with(rule_group_by_columns, group_by_column_values, (groupby, value) -> CONCAT_WS(':', TRIM(groupby), value)) AS group_by_columns, + COALESCE(status, 'false') AS status +FROM ( + SELECT + main.recon_id, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name, + IF( + ISNULL(source_catalog), + CONCAT_WS('.', source_schema, source_table_name), + CONCAT_WS( + '.', + source_catalog, + source_schema, + source_table_name + ) + ) AS source_table, + main.target_table.`catalog` AS target_catalog, + main.target_table.`schema` AS target_schema, + main.target_table.table_name AS target_table_name, + CONCAT_WS( + '.', + target_catalog, + target_schema, + target_table_name + ) AS target_table, + dtl.recon_type, + rul.rule_info.agg_type AS aggregate_type, + UPPER(rul.rule_info.agg_type) || CONCAT('(', rul.rule_info.agg_column, ')') AS rule, + CONCAT_WS( + '_', + 'source', + rul.rule_info.agg_type, + rul.rule_info.agg_column + ) AS source_agg_column, + dtl.agg_details[source_agg_column] AS source_value, + CONCAT_WS( + '_', + 'target', + rul.rule_info.agg_type, + rul.rule_info.agg_column + ) AS target_agg_column, + dtl.agg_details[target_agg_column] AS target_value, + SPLIT(rul.rule_info.group_by_columns, ',') AS rule_group_by_columns, + TRANSFORM(rule_group_by_columns, colm -> + COALESCE(dtl.agg_details[CONCAT('source_group_by_', TRIM(colm))], + dtl.agg_details[CONCAT('target_group_by_', TRIM(colm))])) AS group_by_column_values, + CONCAT_WS( + '_', + 'match', + rul.rule_info.agg_type, + rul.rule_info.agg_column + ) AS status_column, + dtl.agg_details[status_column] AS status + FROM + metrics_view mtc + INNER JOIN remorph.reconcile.main main ON main.recon_table_id = mtc.recon_table_id + INNER JOIN details_view dtl ON mtc.recon_table_id = dtl.recon_table_id + INNER JOIN remorph.reconcile.aggregate_rules rul ON mtc.rule_id = dtl.rule_id + AND dtl.rule_id = rul.rule_id + ) +ORDER BY + recon_id diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/09_0_aggregate_missing_mismatch_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/09_0_aggregate_missing_mismatch_header.md new file mode 100644 index 000000000..b6b3dc271 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/09_0_aggregate_missing_mismatch_header.md @@ -0,0 +1 @@ +# Visualization of Missing and Mismatched Records \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/10_0_aggr_mismatched_records.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/10_0_aggr_mismatched_records.sql new file mode 100644 index 000000000..26b5fe9af --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/10_0_aggr_mismatched_records.sql @@ -0,0 +1,19 @@ +/* --title 'Mismatched Records' --width 6 */ +SELECT + main.recon_id, + CONCAT_WS( + '.', + main.target_table.`catalog`, + main.target_table.`schema`, + main.target_table.table_name + ) AS target_table, + main.start_ts, + metrics.recon_metrics.mismatch AS mismatch +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.aggregate_metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_0_aggr_missing_in_databricks.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_0_aggr_missing_in_databricks.sql new file mode 100644 index 000000000..bcd0113d7 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_0_aggr_missing_in_databricks.sql @@ -0,0 +1,19 @@ +/* --title 'Missing in Databricks' --width 3 */ +SELECT + main.recon_id, + CONCAT_WS( + '.', + main.target_table.`catalog`, + main.target_table.`schema`, + main.target_table.table_name + ) AS target_table, + main.start_ts, + metrics.recon_metrics.missing_in_target AS missing_in_target +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.aggregate_metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_1_aggr_missing_in_source.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_1_aggr_missing_in_source.sql new file mode 100644 index 000000000..4bde21239 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/11_1_aggr_missing_in_source.sql @@ -0,0 +1,19 @@ +/* --title 'Missing in Source' --width 3 */ +SELECT + main.recon_id, + CONCAT_WS( + '.', + main.target_table.`catalog`, + main.target_table.`schema`, + main.target_table.table_name + ) AS target_table, + main.start_ts, + metrics.recon_metrics.missing_in_source AS missing_in_source +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.aggregate_metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/dashboard.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/dashboard.yml new file mode 100644 index 000000000..bc24691ea --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/aggregate_reconciliation_metrics/dashboard.yml @@ -0,0 +1,365 @@ +display_name: "Aggregate Reconciliation Metrics" +tiles: + 04_0_aggregate_summary_table: + overrides: + spec: + withRowNumber: true + encodings: + columns: + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: recon_id + title: recon_id + type: string + cellFormat: + default: + foregroundColor: + rules: + - if: + column: status + fn: '=' + literal: 'true' + value: + foregroundColor: '#3BD973' + - if: + column: status + fn: '=' + literal: 'false' + value: + foregroundColor: '#E92828' + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_type + title: source_type + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_catalog + title: source_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_schema + title: source_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table_name + title: source_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table + title: source_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_catalog + title: target_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_schema + title: target_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table_name + title: target_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table + title: target_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: aggregate_column + title: aggregate_column + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: group_by_columns + title: group_by_columns + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: status + title: status + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: exception + title: exception + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: missing_in_source + title: missing_in_source + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: missing_in_target + title: missing_in_target + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: mismatch + title: mismatch + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: executed_by + title: executed_by + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: datetime + fieldName: start_ts + title: start_ts + type: datetime + dateTimeFormat: 'YYYY-MM-DD HH:mm:ss.SSS' + - booleanValues: + - 'false' + - 'true' + displayAs: datetime + fieldName: end_ts + title: end_ts + type: datetime + dateTimeFormat: 'YYYY-MM-DD HH:mm:ss.SSS' + 08_0_aggregate_details_table: + overrides: + spec: + withRowNumber: true + encodings: + columns: + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: dd_recon_id + title: recon_id + type: string + cellFormat: + default: + foregroundColor: + rules: + - if: + column: status + fn: '=' + literal: 'true' + value: + foregroundColor: '#3BD973' + - if: + column: status + fn: '=' + literal: 'false' + value: + foregroundColor: '#E92828' + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: dd_source_table + title: source_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: dd_target_table + title: target_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: dd_recon_type + title: recon_type + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: dd_aggregate_type + title: aggregate_type + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: aggregate_column + title: aggregate_column + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_value + title: source_value + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_value + title: target_value + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: group_by_columns + title: group_by_columns + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: status + title: status + type: string + 10_0_aggr_mismatched_records: + overrides: + queries: + - name: main_query + query: + datasetName: 10_0_aggr_mismatched_records + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: mismatch + expression: '`mismatch`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: mismatch + scale: + type: quantitative + displayName: mismatch + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + 11_0_aggr_missing_in_databricks: + overrides: + queries: + - name: main_query + query: + datasetName: 11_0_aggr_missing_in_databricks + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: missing_in_target + expression: '`missing_in_target`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: missing_in_target + scale: + type: quantitative + displayName: missing_in_target + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + 11_1_aggr_missing_in_source: + overrides: + queries: + - name: main_query + query: + datasetName: 11_1_aggr_missing_in_source + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: missing_in_source + expression: '`missing_in_source`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: missing_in_source + scale: + type: quantitative + displayName: missing_in_source + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + + diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/00_0_recon_main.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/00_0_recon_main.md new file mode 100644 index 000000000..aaa3cf8aa --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/00_0_recon_main.md @@ -0,0 +1,3 @@ +# Main Reconciliation Table + +### This table provides comprehensive information on the report's status, including failure indications, schema matching status, and details on missing and mismatched records. diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_0_recon_id.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_0_recon_id.filter.yml new file mode 100644 index 000000000..609dd9aca --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_0_recon_id.filter.yml @@ -0,0 +1,6 @@ +columns: +- recon_id +- dd_recon_id +type: MULTI_SELECT +title: Recon Id +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_1_report_type.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_1_report_type.filter.yml new file mode 100644 index 000000000..bdb852bae --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_1_report_type.filter.yml @@ -0,0 +1,5 @@ +columns: +- report_type +type: MULTI_SELECT +title: Report Type +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_2_executed_by.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_2_executed_by.filter.yml new file mode 100644 index 000000000..7a3b36f6c --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/01_2_executed_by.filter.yml @@ -0,0 +1,5 @@ +columns: +- executed_by +type: MULTI_SELECT +title: Executed by +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_0_source_type.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_0_source_type.filter.yml new file mode 100644 index 000000000..427a7b01e --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_0_source_type.filter.yml @@ -0,0 +1,5 @@ +columns: +- source_type +type: MULTI_SELECT +title: Source Type +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_1_source_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_1_source_table.filter.yml new file mode 100644 index 000000000..8e6c442f8 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_1_source_table.filter.yml @@ -0,0 +1,6 @@ +columns: +- source_table +- dd_source_table +type: MULTI_SELECT +title: Source Table Name +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_2_target_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_2_target_table.filter.yml new file mode 100644 index 000000000..202ae3631 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/02_2_target_table.filter.yml @@ -0,0 +1,6 @@ +columns: +- target_table +- dd_target_table +type: MULTI_SELECT +title: Target Table Name +width: 2 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/03_0_started_at.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/03_0_started_at.filter.yml new file mode 100644 index 000000000..ca069bef8 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/03_0_started_at.filter.yml @@ -0,0 +1,5 @@ +columns: +- start_ts +title: Started At +type: DATE_RANGE_PICKER +width: 6 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/05_0_summary_table.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/05_0_summary_table.sql new file mode 100644 index 000000000..4183d241a --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/05_0_summary_table.sql @@ -0,0 +1,38 @@ +/* --title 'Summary Table' --width 6 --height 6 */ +SELECT main.recon_id, + main.source_type, + main.report_type, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name, + IF( + ISNULL(source_catalog), + CONCAT_WS('.', source_schema, source_table_name), + CONCAT_WS( + '.', + source_catalog, + source_schema, + source_table_name + ) + ) AS source_table, + main.target_table.`catalog` AS target_catalog, + main.target_table.`schema` AS target_schema, + main.target_table.table_name AS target_table_name, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS target_table, + metrics.run_metrics.status AS status, + metrics.run_metrics.exception_message AS exception, + metrics.recon_metrics.row_comparison.missing_in_source AS missing_in_source, + metrics.recon_metrics.row_comparison.missing_in_target AS missing_in_target, + metrics.recon_metrics.column_comparison.absolute_mismatch AS absolute_mismatch, + metrics.recon_metrics.column_comparison.threshold_mismatch AS threshold_mismatch, + metrics.recon_metrics.column_comparison.mismatch_columns AS mismatch_columns, + metrics.recon_metrics.schema_comparison AS schema_comparison, + metrics.run_metrics.run_by_user AS executed_by, + main.start_ts AS start_ts, + main.end_ts AS end_ts +FROM remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/06_0_schema_comparison_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/06_0_schema_comparison_header.md new file mode 100644 index 000000000..ee4cb4018 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/06_0_schema_comparison_header.md @@ -0,0 +1,3 @@ +# Schema Comparison Details + +### This table provides a detailed view of schema mismatches. \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/07_0_schema_details_table.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/07_0_schema_details_table.sql new file mode 100644 index 000000000..9f7cc115b --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/07_0_schema_details_table.sql @@ -0,0 +1,42 @@ +/* --title 'Schema Details' --width 6 */ +WITH tmp AS ( + SELECT + recon_table_id, + inserted_ts, + explode(data) AS schema_data + FROM + remorph.reconcile.details + WHERE + recon_type = 'schema' +) +SELECT + main.recon_id, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name, + IF( + ISNULL(source_catalog), + CONCAT_WS('.', source_schema, source_table_name), + CONCAT_WS( + '.', + source_catalog, + source_schema, + source_table_name + ) + ) AS source_table, + main.target_table.`catalog` AS target_catalog, + main.target_table.`schema` AS target_schema, + main.target_table.table_name AS target_table_name, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS target_table, + schema_data['source_column'] AS source_column, + schema_data['source_datatype'] AS source_datatype, + schema_data['databricks_column'] AS databricks_column, + schema_data['databricks_datatype'] AS databricks_datatype, + schema_data['is_valid'] AS is_valid +FROM + remorph.reconcile.main main + INNER JOIN tmp ON main.recon_table_id = tmp.recon_table_id +ORDER BY + tmp.inserted_ts DESC, + main.recon_id, + main.target_table diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/08_0_drill_down_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/08_0_drill_down_header.md new file mode 100644 index 000000000..699e879d1 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/08_0_drill_down_header.md @@ -0,0 +1,3 @@ +# Drill Down + +### The details table contains all the sample records for mismatches and missing entries, providing users with exact details to pinpoint the issues. \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_0_recon_id.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_0_recon_id.filter.yml new file mode 100644 index 000000000..205b9a8ba --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_0_recon_id.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_recon_id +type: MULTI_SELECT +title: Recon Id \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_1_category.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_1_category.filter.yml new file mode 100644 index 000000000..094ec93ab --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/09_1_category.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_recon_type +type: MULTI_SELECT +title: Category \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_0_target_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_0_target_table.filter.yml new file mode 100644 index 000000000..61888e08e --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_0_target_table.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_target_table +type: MULTI_SELECT +title: Target Table Name \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_1_source_table.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_1_source_table.filter.yml new file mode 100644 index 000000000..31366f2b0 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/10_1_source_table.filter.yml @@ -0,0 +1,4 @@ +columns: +- dd_source_table +type: MULTI_SELECT +title: Source Table Name \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/11_0_recon_details_pivot.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/11_0_recon_details_pivot.sql new file mode 100644 index 000000000..6edad4d9a --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/11_0_recon_details_pivot.sql @@ -0,0 +1,40 @@ +/* --title 'Recon Details Drill Down' --height 6 --width 6 */ +WITH tmp AS ( + SELECT + recon_table_id, + inserted_ts, + recon_type, + explode(data) AS data, + row_number() OVER (PARTITION BY recon_table_id, recon_type ORDER BY recon_table_id) AS rn + FROM + remorph.reconcile.details + WHERE + recon_type != 'schema' +) +SELECT + main.recon_id AS dd_recon_id, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name, + IF( + ISNULL(source_catalog), + CONCAT_WS('.', source_schema, source_table_name), + CONCAT_WS( + '.', + source_catalog, + source_schema, + source_table_name + ) + ) AS dd_source_table, + main.target_table.`catalog` AS target_catalog, + main.target_table.`schema` AS target_schema, + main.target_table.table_name AS target_table_name, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS dd_target_table, + recon_type AS dd_recon_type, + key, + value, + rn +FROM tmp + INNER JOIN remorph.reconcile.main main + ON main.recon_table_id = tmp.recon_table_id + LATERAL VIEW explode(data) exploded_data AS key, value diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/12_0_daily_data_validation_issue_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/12_0_daily_data_validation_issue_header.md new file mode 100644 index 000000000..a83e27ebb --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/12_0_daily_data_validation_issue_header.md @@ -0,0 +1,3 @@ +# Daily Data Validation Issues Report + +### This summary report provides an overview of all data validation runs conducted on a specific day. It highlights whether each table has encountered any validation issues, without delving into the low-level details. This report aims to give a quick and clear status of data integrity across all tables for the day. \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/13_0_success_fail_.filter.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/13_0_success_fail_.filter.yml new file mode 100644 index 000000000..899e08681 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/13_0_success_fail_.filter.yml @@ -0,0 +1,4 @@ +columns: +- start_date +type: DATE_RANGE_PICKER +width: 6 \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/14_0_failed_recon_ids.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/14_0_failed_recon_ids.sql new file mode 100644 index 000000000..89228a8ad --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/14_0_failed_recon_ids.sql @@ -0,0 +1,15 @@ +/* --title 'Number of Distinct Recon IDs per Target Table Failed' --width 6 */ +SELECT + main.recon_id AS rec_id, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS t_table, + DATE(main.start_ts) AS start_date +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics +ON main.recon_table_id = metrics.recon_table_id +WHERE + metrics.run_metrics.status = FALSE +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_0_total_failed_runs.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_0_total_failed_runs.sql new file mode 100644 index 000000000..71fa10faa --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_0_total_failed_runs.sql @@ -0,0 +1,10 @@ +/* --title 'Total number of runs failed' --width 2 */ +SELECT + main.recon_id AS rec_id, + DATE(main.start_ts) AS start_date +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics +ON main.recon_table_id = metrics.recon_table_id +WHERE + metrics.run_metrics.status = FALSE diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_1_failed_targets.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_1_failed_targets.sql new file mode 100644 index 000000000..ff16c4355 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_1_failed_targets.sql @@ -0,0 +1,10 @@ +/* --title 'Unique target tables failed' --width 2 */ +SELECT + CONCAT_WS('.', main.target_table.catalog, main.target_table.schema, main.target_table.table_name) AS t_table, + DATE(main.start_ts) AS start_date +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics +ON main.recon_table_id = metrics.recon_table_id +WHERE + metrics.run_metrics.status = FALSE diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_2_successful_targets.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_2_successful_targets.sql new file mode 100644 index 000000000..2d2a6eb19 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/15_2_successful_targets.sql @@ -0,0 +1,10 @@ +/* --title 'Unique target tables successful' --width 2 */ +SELECT + CONCAT_WS('.', main.target_table.catalog, main.target_table.schema, main.target_table.table_name) AS t_table, + DATE(main.start_ts) AS start_date +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics +ON main.recon_table_id = metrics.recon_table_id +WHERE + metrics.run_metrics.status = TRUE diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/16_0_missing_mismatch_header.md b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/16_0_missing_mismatch_header.md new file mode 100644 index 000000000..b6b3dc271 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/16_0_missing_mismatch_header.md @@ -0,0 +1 @@ +# Visualization of Missing and Mismatched Records \ No newline at end of file diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_0_mismatched_records.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_0_mismatched_records.sql new file mode 100644 index 000000000..09eea3885 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_0_mismatched_records.sql @@ -0,0 +1,14 @@ +/* --title 'Mismatched Records' --width 3 */ +SELECT + main.recon_id, + CONCAT_WS('.', main.target_table.catalog, main.target_table.schema, main.target_table.table_name) AS target_table, + metrics.recon_metrics.column_comparison.absolute_mismatch AS absolute_mismatch, + main.start_ts AS start_ts +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_1_threshold_mismatches.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_1_threshold_mismatches.sql new file mode 100644 index 000000000..781fa2c10 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/17_1_threshold_mismatches.sql @@ -0,0 +1,14 @@ +/* --title 'Threshold Mismatches' --width 3 */ +SELECT + main.recon_id, + CONCAT_WS('.', main.target_table.catalog, main.target_table.schema, main.target_table.table_name) AS target_table, + metrics.recon_metrics.column_comparison.threshold_mismatch AS threshold_mismatch, + main.start_ts AS start_ts +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_0_missing_in_databricks.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_0_missing_in_databricks.sql new file mode 100644 index 000000000..1dbcf8073 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_0_missing_in_databricks.sql @@ -0,0 +1,14 @@ +/* --title 'Missing in Databricks' --width 3 */ +SELECT + main.recon_id, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS target_table, + metrics.recon_metrics.row_comparison.missing_in_target AS missing_in_target, + main.start_ts AS start_ts +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_1_missing_in_source.sql b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_1_missing_in_source.sql new file mode 100644 index 000000000..f6b392d99 --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/18_1_missing_in_source.sql @@ -0,0 +1,14 @@ +/* --title 'Missing in Source' --width 3 */ +SELECT + main.recon_id, + CONCAT(main.target_table.catalog, '.', main.target_table.schema, '.', main.target_table.table_name) AS target_table, + metrics.recon_metrics.row_comparison.missing_in_source AS missing_in_source, + main.start_ts AS start_ts +FROM + remorph.reconcile.main main + INNER JOIN remorph.reconcile.metrics metrics + ON main.recon_table_id = metrics.recon_table_id +ORDER BY + metrics.inserted_ts DESC, + main.recon_id, + main.target_table.table_name diff --git a/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/dashboard.yml b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/dashboard.yml new file mode 100644 index 000000000..699575c3c --- /dev/null +++ b/src/databricks/labs/remorph/resources/reconcile/dashboards/reconciliation_metrics/dashboard.yml @@ -0,0 +1,545 @@ +display_name: "Reconciliation Metrics" +tiles: + 05_0_summary_table: + overrides: + spec: + withRowNumber: true + encodings: + columns: + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: recon_id + title: recon_id + type: string + cellFormat: + default: + foregroundColor: + rules: + - if: + column: status + fn: '=' + literal: 'true' + value: + foregroundColor: '#3BD973' + - if: + column: status + fn: '=' + literal: 'false' + value: + foregroundColor: '#E92828' + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_type + title: source_type + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: report_type + title: report_type + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_catalog + title: source_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_schema + title: source_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table_name + title: source_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table + title: source_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_catalog + title: target_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_schema + title: target_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table_name + title: target_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table + title: target_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: status + title: status + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: exception + title: exception + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: missing_in_source + title: missing_in_source + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: missing_in_target + title: missing_in_target + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: absolute_mismatch + title: absolute_mismatch + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: threshold_mismatch + title: threshold_mismatch + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: mismatch_columns + title: mismatch_columns + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: schema_comparison + title: schema_comparison + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: executed_by + title: executed_by + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: datetime + fieldName: start_ts + title: start_ts + type: datetime + dateTimeFormat: 'YYYY-MM-DD HH:mm:ss.SSS' + - booleanValues: + - 'false' + - 'true' + displayAs: datetime + fieldName: end_ts + title: end_ts + type: datetime + dateTimeFormat: 'YYYY-MM-DD HH:mm:ss.SSS' + 07_0_schema_details_table: + overrides: + spec: + withRowNumber: true + encodings: + columns: + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: recon_id + title: recon_id + type: string + cellFormat: + default: + foregroundColor: + rules: + - if: + column: is_valid + fn: '=' + literal: 'false' + value: + foregroundColor: '#E92828' + - if: + column: is_valid + fn: '=' + literal: 'true' + value: + foregroundColor: '#3BD973' + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_catalog + title: source_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_schema + title: source_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table_name + title: source_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_table + title: source_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_catalog + title: target_catalog + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_schema + title: target_schema + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table_name + title: target_table_name + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: target_table + title: target_table + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_column + title: source_column + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: source_datatype + title: source_datatype + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: databricks_column + title: databricks_column + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: databricks_datatype + title: databricks_datatype + type: string + - booleanValues: + - 'false' + - 'true' + displayAs: string + fieldName: is_valid + title: is_valid + type: string + 11_0_recon_details_pivot: + overrides: + spec: + version: 3 + widgetType: pivot + encodings: + rows: + - fieldName: dd_recon_id + displayName: recon_id + - fieldName: dd_source_table + displayName: source_table + - fieldName: dd_target_table + displayName: target_table + - fieldName: dd_recon_type + displayName: recon_type + - fieldName: rn + displayName: rn + columns: + - fieldName: key + displayName: key + cell: + fieldName: value + cellType: text + displayName: value + 14_0_failed_recon_ids: + overrides: + spec: + version: 3 + widgetType: bar + encodings: + x: + fieldName: t_table + scale: + type: categorical + sort: + by: y-reversed + displayName: Target table + 'y': + fieldName: countdistinct(rec_id) + scale: + type: quantitative + displayName: Count of Unique Recon Ids + label: + show: true + queries: + - name: main_query + query: + datasetName: 14_0_failed_recon_ids + fields: + - name: t_table + expression: '`t_table`' + - name: countdistinct(rec_id) + expression: COUNT(DISTINCT `rec_id`) + disaggregated: false + 15_0_total_failed_runs: + overrides: + spec: + version: 2 + widgetType: counter + encodings: + value: + fieldName: countdistinct(rec_id) + displayName: countdistinct(rec_id) + queries: + - name: main_query + query: + datasetName: 15_0_total_failed_runs + fields: + - name: countdistinct(rec_id) + expression: 'COUNT(DISTINCT `rec_id`)' + disaggregated: false + 15_1_failed_targets: + overrides: + spec: + version: 2 + widgetType: counter + encodings: + value: + fieldName: countdistinct(t_table) + displayName: countdistinct(t_table) + queries: + - name: main_query + query: + datasetName: 15_1_failed_targets + fields: + - name: countdistinct(t_table) + expression: 'COUNT(DISTINCT `t_table`)' + disaggregated: false + 15_2_successful_targets: + overrides: + spec: + version: 2 + widgetType: counter + encodings: + value: + fieldName: countdistinct(t_table) + displayName: countdistinct(t_table) + queries: + - name: main_query + query: + datasetName: 15_2_successful_targets + fields: + - name: countdistinct(t_table) + expression: 'COUNT(DISTINCT `t_table`)' + disaggregated: false + 17_0_mismatched_records: + overrides: + queries: + - name: main_query + query: + datasetName: 17_0_mismatched_records + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: absolute_mismatch + expression: '`absolute_mismatch`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: absolute_mismatch + scale: + type: quantitative + displayName: absolute_mismatch + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + 17_1_threshold_mismatches: + overrides: + queries: + - name: main_query + query: + datasetName: 17_1_threshold_mismatches + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: threshold_mismatch + expression: '`threshold_mismatch`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: threshold_mismatch + scale: + type: quantitative + displayName: threshold_mismatch + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + 18_0_missing_in_databricks: + overrides: + queries: + - name: main_query + query: + datasetName: 18_0_missing_in_databricks + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: missing_in_target + expression: '`missing_in_target`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: missing_in_target + scale: + type: quantitative + displayName: missing_in_target + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false + 18_1_missing_in_source: + overrides: + queries: + - name: main_query + query: + datasetName: 18_1_missing_in_source + fields: + - name: target_table + expression: '`target_table`' + - name: hourly(start_ts) + expression: 'DATE_TRUNC("HOUR", `start_ts`)' + - name: missing_in_source + expression: '`missing_in_source`' + disaggregated: true + spec: + version: 3 + widgetType: area + encodings: + x: + fieldName: hourly(start_ts) + scale: + type: temporal + displayName: start_ts + 'y': + fieldName: missing_in_source + scale: + type: quantitative + displayName: missing_in_source + color: + fieldName: target_table + scale: + type: categorical + displayName: target_table + label: + show: false diff --git a/tests/resources/Remorph-Reconciliation-Substituted.lvdash.json b/tests/resources/Remorph-Reconciliation-Substituted.lvdash.json deleted file mode 100644 index f91b46714..000000000 --- a/tests/resources/Remorph-Reconciliation-Substituted.lvdash.json +++ /dev/null @@ -1 +0,0 @@ -{"datasets": [{"name": "088adb8a", "displayName": "recon_main", "query": "select \nmain.recon_id,\nmain.source_type,\nmain.report_type,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table, \nmetrics.run_metrics.status as status,\nmetrics.run_metrics.exception_message as exception,\nmetrics.recon_metrics.row_comparison.missing_in_source as missing_in_source,\nmetrics.recon_metrics.row_comparison.missing_in_target as missing_in_target,\nmetrics.recon_metrics.column_comparison.absolute_mismatch as absolute_mismatch,\nmetrics.recon_metrics.column_comparison.threshold_mismatch as threshold_mismatch,\nmetrics.recon_metrics.column_comparison.mismatch_columns as mismatch_columns,\nmetrics.recon_metrics.schema_comparison as schema_comparison,\nmetrics.run_metrics.run_by_user as executed_by,\nmain.start_ts as start_ts,\nmain.end_ts as end_ts\nfrom IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \nIDENTIFIER(:catalog || '.' || :schema || '.metrics' ) metrics\non main.recon_table_id = metrics.recon_table_id\norder by metrics.inserted_ts desc, main.recon_id, main.target_table.table_name", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph1"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile1"}]}}}]}, {"name": "07f3d7d5", "displayName": "recon_details_schema", "query": "with tmp as (select \nrecon_table_id,\ninserted_ts,\nexplode(data) as schema_data\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type='schema'\n) \nselect \nmain.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nschema_data['source_column'] as source_column,\nschema_data['source_datatype'] as source_datatype,\nschema_data['databricks_column'] as databricks_column,\nschema_data['databricks_datatype'] as databricks_datatype,\nschema_data['is_valid'] as is_valid\nfrom \nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \ntmp\non main.recon_table_id = tmp.recon_table_id\norder by tmp.inserted_ts desc, main.recon_id, main.target_table\n", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph1"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile1"}]}}}]}, {"name": "aec31225", "displayName": "recon_details_pivot", "query": "with tmp as (select recon_table_id, inserted_ts ,recon_type, explode(data) as data, \nrow_number() over(partition by recon_table_id,recon_type order by recon_table_id) as rn\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type != 'schema')\nselect main.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nrecon_type, key, value, rn\nfrom tmp\ninner join\nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\non main.recon_table_id = tmp.recon_table_id\nlateral view explode(data) exploded_data AS key, value\n", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph1"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile1"}]}}}]}], "pages": [{"name": "1ac59528", "displayName": "New Page", "layout": [{"widget": {"name": "b3208d80", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_type", "expression": "`source_type`"}, {"name": "report_type", "expression": "`report_type`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "status", "expression": "`status`"}, {"name": "exception", "expression": "`exception`"}, {"name": "missing_in_source", "expression": "`missing_in_source`"}, {"name": "missing_in_target", "expression": "`missing_in_target`"}, {"name": "absolute_mismatch", "expression": "`absolute_mismatch`"}, {"name": "threshold_mismatch", "expression": "`threshold_mismatch`"}, {"name": "mismatch_columns", "expression": "`mismatch_columns`"}, {"name": "schema_comparison", "expression": "`schema_comparison`"}, {"name": "executed_by", "expression": "`executed_by`"}, {"name": "start_ts", "expression": "`start_ts`"}, {"name": "end_ts", "expression": "`end_ts`"}], "disaggregated": true}}], "spec": {"version": 1, "widgetType": "table", "encodings": {"columns": [{"fieldName": "recon_id", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 0, "title": "recon_id", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "cellFormat": {"default": {"foregroundColor": null}, "rules": [{"if": {"column": "status", "fn": "=", "literal": "true"}, "value": {"foregroundColor": "#3BD973"}}, {"if": {"column": "status", "fn": "=", "literal": "false"}, "value": {"foregroundColor": "#E92828"}}]}, "displayName": "recon_id"}, {"fieldName": "source_type", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 1, "title": "source_type", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_type"}, {"fieldName": "report_type", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 2, "title": "report_type", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "report_type"}, {"fieldName": "source_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 5, "title": "source_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_table"}, {"fieldName": "target_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 8, "title": "target_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "target_table"}, {"fieldName": "status", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "boolean", "displayAs": "boolean", "visible": true, "order": 9, "title": "status", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "status"}, {"fieldName": "exception", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 10, "title": "exception", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "exception"}, {"fieldName": "missing_in_source", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 11, "title": "missing_in_source", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "missing_in_source"}, {"fieldName": "missing_in_target", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 12, "title": "missing_in_target", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "missing_in_target"}, {"fieldName": "absolute_mismatch", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 13, "title": "absolute_mismatch", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "absolute_mismatch"}, {"fieldName": "threshold_mismatch", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 14, "title": "threshold_mismatch", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "threshold_mismatch"}, {"fieldName": "mismatch_columns", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 15, "title": "mismatch_columns", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "mismatch_columns"}, {"fieldName": "schema_comparison", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "boolean", "displayAs": "boolean", "visible": true, "order": 16, "title": "schema_comparison", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "schema_comparison"}, {"fieldName": "executed_by", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 19, "title": "executed_by", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "executed_by"}, {"fieldName": "start_ts", "dateTimeFormat": "YYYY-MM-DD HH:mm:ss.SSS", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "datetime", "displayAs": "datetime", "visible": true, "order": 20, "title": "start_ts", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "start_ts"}, {"fieldName": "end_ts", "dateTimeFormat": "YYYY-MM-DD HH:mm:ss.SSS", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "datetime", "displayAs": "datetime", "visible": true, "order": 21, "title": "end_ts", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "end_ts"}]}, "invisibleColumns": [{"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_catalog", "type": "string", "displayAs": "string", "order": 3, "title": "source_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_schema", "type": "string", "displayAs": "string", "order": 4, "title": "source_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_catalog", "type": "string", "displayAs": "string", "order": 6, "title": "target_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_schema", "type": "string", "displayAs": "string", "order": 7, "title": "target_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_table_name", "type": "string", "displayAs": "string", "order": 17, "title": "source_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_table_name", "type": "string", "displayAs": "string", "order": 18, "title": "target_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}], "allowHTMLByDefault": false, "itemsPerPage": 100, "paginationSize": "default", "condensed": true, "withRowNumber": true, "frame": {"showDescription": true, "description": "Summary Table", "showTitle": false}}}, "position": {"x": 0, "y": 10, "width": 6, "height": 7}}, {"widget": {"name": "c27cb052", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_recon_id", "query": {"datasetName": "088adb8a", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_recon_id", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_recon_id"}, {"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_recon_id"}, {"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Recon Id", "description": ""}}}, "position": {"x": 0, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "3ff15994", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_target_table", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_target_table", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table", "query": {"datasetName": "aec31225", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_target_table"}, {"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_target_table"}, {"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Traget Table Name", "description": ""}}}, "position": {"x": 2, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "bec17005", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table", "query": {"datasetName": "088adb8a", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table", "query": {"datasetName": "aec31225", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table"}, {"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table"}, {"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Source Table Name", "description": ""}}}, "position": {"x": 4, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "41d78bbf", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type", "query": {"datasetName": "088adb8a", "fields": [{"name": "source_type", "expression": "`source_type`"}, {"name": "source_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_type", "displayName": "source_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Source Type", "description": "Applied only on summary table"}}}, "position": {"x": 0, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "4bcc8273", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_type", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_type", "expression": "`recon_type`"}, {"name": "recon_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"fieldName": "recon_type", "displayName": "recon_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Category", "description": "Applied only on details table"}}}, "position": {"x": 3, "y": 27, "width": 3, "height": 1}}, {"widget": {"name": "b7d8efe7", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "absolute_mismatch", "expression": "`absolute_mismatch`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "absolute_mismatch", "scale": {"type": "quantitative"}, "displayName": "absolute_mismatch"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Mismatched Records", "showDescription": false}}}, "position": {"x": 0, "y": 37, "width": 3, "height": 6}}, {"widget": {"name": "13d8f169", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "missing_in_target", "expression": "`missing_in_target`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "missing_in_target", "scale": {"type": "quantitative"}, "displayName": "missing_in_target"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Missing in Source"}}}, "position": {"x": 3, "y": 43, "width": 3, "height": 6}}, {"widget": {"name": "1b4c4556", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "missing_in_source", "expression": "`missing_in_source`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "missing_in_source", "scale": {"type": "quantitative"}, "displayName": "missing_in_source"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Missing in Databricks"}}}, "position": {"x": 0, "y": 43, "width": 3, "height": 6}}, {"widget": {"name": "aedc94de", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "threshold_mismatch", "expression": "`threshold_mismatch`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "threshold_mismatch", "scale": {"type": "quantitative"}, "displayName": "threshold_mismatch"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}}, "frame": {"showTitle": true, "title": "Threshold Mismatches"}}}, "position": {"x": 3, "y": 37, "width": 3, "height": 6}}, {"widget": {"name": "7d60be10", "queries": [{"name": "main_query", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "source_column", "expression": "`source_column`"}, {"name": "source_datatype", "expression": "`source_datatype`"}, {"name": "databricks_column", "expression": "`databricks_column`"}, {"name": "databricks_datatype", "expression": "`databricks_datatype`"}, {"name": "is_valid", "expression": "`is_valid`"}], "disaggregated": true}}], "spec": {"version": 1, "widgetType": "table", "encodings": {"columns": [{"fieldName": "recon_id", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 0, "title": "recon_id", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "cellFormat": {"default": {"foregroundColor": null}, "rules": [{"if": {"column": "is_valid", "fn": "=", "literal": "false"}, "value": {"foregroundColor": "#E92828"}}, {"if": {"column": "is_valid", "fn": "=", "literal": "true"}, "value": {"foregroundColor": "#3BD973"}}]}, "displayName": "recon_id"}, {"fieldName": "source_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 4, "title": "source_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_table"}, {"fieldName": "target_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 8, "title": "target_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "target_table"}, {"fieldName": "source_column", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 9, "title": "source_column", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_column"}, {"fieldName": "source_datatype", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 10, "title": "source_datatype", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_datatype"}, {"fieldName": "databricks_column", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 11, "title": "databricks_column", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "databricks_column"}, {"fieldName": "databricks_datatype", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 12, "title": "databricks_datatype", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "databricks_datatype"}, {"fieldName": "is_valid", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 13, "title": "is_valid", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "is_valid"}]}, "invisibleColumns": [{"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_table_name", "type": "string", "displayAs": "string", "order": 1, "title": "source_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_catalog", "type": "string", "displayAs": "string", "order": 2, "title": "source_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_schema", "type": "string", "displayAs": "string", "order": 3, "title": "source_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_catalog", "type": "string", "displayAs": "string", "order": 5, "title": "target_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_schema", "type": "string", "displayAs": "string", "order": 6, "title": "target_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_table_name", "type": "string", "displayAs": "string", "order": 7, "title": "target_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}], "allowHTMLByDefault": false, "itemsPerPage": 25, "paginationSize": "default", "condensed": true, "withRowNumber": false, "frame": {"showTitle": true, "title": "Schema Details"}}}, "position": {"x": 0, "y": 19, "width": 6, "height": 6}}, {"widget": {"name": "15be1fef", "queries": [{"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_catalog", "query": {"datasetName": "088adb8a", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_catalog", "query": {"datasetName": "07f3d7d5", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_catalog", "query": {"datasetName": "aec31225", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_catalog"}, {"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_catalog"}, {"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_catalog"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Catalog", "description": "\u2019remorph' is the default catalog for the metrics table; specify a different catalog name if the metrics are stored elsewhere."}}}, "position": {"x": 0, "y": 2, "width": 3, "height": 2}}, {"widget": {"name": "d1e11541", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_report_type", "query": {"datasetName": "088adb8a", "fields": [{"name": "report_type", "expression": "`report_type`"}, {"name": "report_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "report_type", "displayName": "report_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_report_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Report", "description": "Applied only on summary table"}}}, "position": {"x": 2, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "8e9a7f6d", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_start_ts", "query": {"datasetName": "088adb8a", "fields": [{"name": "start_ts", "expression": "`start_ts`"}, {"name": "start_ts_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-date-range-picker", "encodings": {"fields": [{"fieldName": "start_ts", "displayName": "start_ts", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_start_ts"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "started at", "description": "Applied on all tables"}}}, "position": {"x": 0, "y": 8, "width": 6, "height": 2}}, {"widget": {"name": "12568355", "textbox_spec": "# Main Reconciliation Table\n### This table provides comprehensive information on the report's status, including failure indications, schema matching status, and details on missing and mismatched records."}, "position": {"x": 0, "y": 0, "width": 6, "height": 2}}, {"widget": {"name": "72f3dc17", "textbox_spec": "# Drill Down\n### The details table contains all the sample records for mismatches and missing entries, providing users with exact details to pinpoint the issues."}, "position": {"x": 0, "y": 25, "width": 6, "height": 2}}, {"widget": {"name": "67d6c36d", "textbox_spec": "# Schema Comparison Details\n### This table provides a detailed view of schema mismatches."}, "position": {"x": 0, "y": 17, "width": 6, "height": 2}}, {"widget": {"name": "fca3d649", "textbox_spec": "# Visualization of Missing and Mismatched Records"}, "position": {"x": 0, "y": 35, "width": 6, "height": 2}}, {"widget": {"name": "71206c9b", "queries": [{"name": "main_query", "query": {"datasetName": "aec31225", "fields": [{"name": "value", "expression": "`value`"}, {"name": "key", "expression": "`key`"}, {"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "recon_type", "expression": "`recon_type`"}, {"name": "rn", "expression": "`rn`"}], "disaggregated": false, "orders": [{"direction": "ASC", "expression": "`recon_id`"}, {"direction": "ASC", "expression": "`source_table`"}, {"direction": "ASC", "expression": "`target_table`"}, {"direction": "ASC", "expression": "`recon_type`"}, {"direction": "ASC", "expression": "`rn`"}, {"direction": "ASC", "expression": "`key`"}]}}], "spec": {"version": 3, "widgetType": "pivot", "encodings": {"rows": [{"fieldName": "recon_id", "displayName": "recon_id"}, {"fieldName": "source_table", "displayName": "source_table"}, {"fieldName": "target_table", "displayName": "target_table"}, {"fieldName": "recon_type", "displayName": "recon_type"}, {"fieldName": "rn", "displayName": "rn"}], "columns": [{"fieldName": "key", "displayName": "key"}], "cell": {"fieldName": "value", "cellType": "text", "displayName": "value"}}}}, "position": {"x": 0, "y": 29, "width": 6, "height": 6}}, {"widget": {"name": "14576173", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table", "query": {"datasetName": "aec31225", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table"}]}, "frame": {"showDescription": true, "showTitle": true, "title": "Target Table Name", "description": "Applied only on details table"}}}, "position": {"x": 0, "y": 28, "width": 3, "height": 1}}, {"widget": {"name": "81426d8f", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table", "query": {"datasetName": "aec31225", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Source Table Name", "description": "Applied only on details table"}}}, "position": {"x": 3, "y": 28, "width": 3, "height": 1}}, {"widget": {"name": "6f02d764", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Recon Id", "description": "Applied only on details table"}}}, "position": {"x": 0, "y": 27, "width": 3, "height": 1}}, {"widget": {"name": "832b10ed", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_executed_by", "query": {"datasetName": "088adb8a", "fields": [{"name": "executed_by", "expression": "`executed_by`"}, {"name": "executed_by_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "executed_by", "displayName": "executed_by", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_executed_by"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Executed by", "description": "Applied only on summary table"}}}, "position": {"x": 4, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "27298c01", "queries": [{"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_schema", "query": {"datasetName": "088adb8a", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_schema", "query": {"datasetName": "07f3d7d5", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_schema", "query": {"datasetName": "aec31225", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_schema"}, {"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_schema"}, {"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_schema"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Schema", "description": "\u2019reconcile\u2019 is the default schema for the metrics table; specify a different schema name if the metrics are stored elsewhere."}}}, "position": {"x": 3, "y": 2, "width": 3, "height": 2}}]}]} \ No newline at end of file diff --git a/tests/resources/Remorph-Reconciliation.lvdash.json b/tests/resources/Remorph-Reconciliation.lvdash.json deleted file mode 100644 index fb6093e77..000000000 --- a/tests/resources/Remorph-Reconciliation.lvdash.json +++ /dev/null @@ -1 +0,0 @@ -{"datasets": [{"name": "088adb8a", "displayName": "recon_main", "query": "select \nmain.recon_id,\nmain.source_type,\nmain.report_type,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table, \nmetrics.run_metrics.status as status,\nmetrics.run_metrics.exception_message as exception,\nmetrics.recon_metrics.row_comparison.missing_in_source as missing_in_source,\nmetrics.recon_metrics.row_comparison.missing_in_target as missing_in_target,\nmetrics.recon_metrics.column_comparison.absolute_mismatch as absolute_mismatch,\nmetrics.recon_metrics.column_comparison.threshold_mismatch as threshold_mismatch,\nmetrics.recon_metrics.column_comparison.mismatch_columns as mismatch_columns,\nmetrics.recon_metrics.schema_comparison as schema_comparison,\nmetrics.run_metrics.run_by_user as executed_by,\nmain.start_ts as start_ts,\nmain.end_ts as end_ts\nfrom IDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \nIDENTIFIER(:catalog || '.' || :schema || '.metrics' ) metrics\non main.recon_table_id = metrics.recon_table_id\norder by metrics.inserted_ts desc, main.recon_id, main.target_table.table_name", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile"}]}}}]}, {"name": "07f3d7d5", "displayName": "recon_details_schema", "query": "with tmp as (select \nrecon_table_id,\ninserted_ts,\nexplode(data) as schema_data\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type='schema'\n) \nselect \nmain.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nschema_data['source_column'] as source_column,\nschema_data['source_datatype'] as source_datatype,\nschema_data['databricks_column'] as databricks_column,\nschema_data['databricks_datatype'] as databricks_datatype,\nschema_data['is_valid'] as is_valid\nfrom \nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\ninner join \ntmp\non main.recon_table_id = tmp.recon_table_id\norder by tmp.inserted_ts desc, main.recon_id, main.target_table\n", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile"}]}}}]}, {"name": "aec31225", "displayName": "recon_details_pivot", "query": "with tmp as (select recon_table_id, inserted_ts ,recon_type, explode(data) as data, \nrow_number() over(partition by recon_table_id,recon_type order by recon_table_id) as rn\nfrom IDENTIFIER(:catalog || '.' || :schema || '.details' )\nwhere recon_type != 'schema')\nselect main.recon_id,\nmain.source_table.`catalog` as source_catalog,\nmain.source_table.`schema` as source_schema,\nmain.source_table.table_name as source_table_name,\nCASE \n WHEN COALESCE(MAIN.SOURCE_TABLE.CATALOG, '') <> '' THEN CONCAT(MAIN.SOURCE_TABLE.CATALOG, '.', MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n ELSE CONCAT(MAIN.SOURCE_TABLE.SCHEMA, '.', MAIN.SOURCE_TABLE.TABLE_NAME) \n END AS source_table,\nmain.target_table.`catalog` as target_catalog,\nmain.target_table.`schema` as target_schema,\nmain.target_table.table_name as target_table_name,\nCONCAT(MAIN.TARGET_TABLE.CATALOG, '.', MAIN.TARGET_TABLE.SCHEMA, '.', MAIN.TARGET_TABLE.TABLE_NAME) AS target_table,\nrecon_type, key, value, rn\nfrom tmp\ninner join\nIDENTIFIER(:catalog || '.' || :schema || '.main' ) main\non main.recon_table_id = tmp.recon_table_id\nlateral view explode(data) exploded_data AS key, value\n", "parameters": [{"displayName": "catalog", "keyword": "catalog", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "remorph"}]}}}, {"displayName": "schema", "keyword": "schema", "dataType": "STRING", "defaultSelection": {"values": {"dataType": "STRING", "values": [{"value": "reconcile"}]}}}]}], "pages": [{"name": "1ac59528", "displayName": "New Page", "layout": [{"widget": {"name": "b3208d80", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_type", "expression": "`source_type`"}, {"name": "report_type", "expression": "`report_type`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "status", "expression": "`status`"}, {"name": "exception", "expression": "`exception`"}, {"name": "missing_in_source", "expression": "`missing_in_source`"}, {"name": "missing_in_target", "expression": "`missing_in_target`"}, {"name": "absolute_mismatch", "expression": "`absolute_mismatch`"}, {"name": "threshold_mismatch", "expression": "`threshold_mismatch`"}, {"name": "mismatch_columns", "expression": "`mismatch_columns`"}, {"name": "schema_comparison", "expression": "`schema_comparison`"}, {"name": "executed_by", "expression": "`executed_by`"}, {"name": "start_ts", "expression": "`start_ts`"}, {"name": "end_ts", "expression": "`end_ts`"}], "disaggregated": true}}], "spec": {"version": 1, "widgetType": "table", "encodings": {"columns": [{"fieldName": "recon_id", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 0, "title": "recon_id", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "cellFormat": {"default": {"foregroundColor": null}, "rules": [{"if": {"column": "status", "fn": "=", "literal": "true"}, "value": {"foregroundColor": "#3BD973"}}, {"if": {"column": "status", "fn": "=", "literal": "false"}, "value": {"foregroundColor": "#E92828"}}]}, "displayName": "recon_id"}, {"fieldName": "source_type", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 1, "title": "source_type", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_type"}, {"fieldName": "report_type", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 2, "title": "report_type", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "report_type"}, {"fieldName": "source_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 5, "title": "source_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_table"}, {"fieldName": "target_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 8, "title": "target_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "target_table"}, {"fieldName": "status", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "boolean", "displayAs": "boolean", "visible": true, "order": 9, "title": "status", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "status"}, {"fieldName": "exception", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 10, "title": "exception", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "exception"}, {"fieldName": "missing_in_source", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 11, "title": "missing_in_source", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "missing_in_source"}, {"fieldName": "missing_in_target", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 12, "title": "missing_in_target", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "missing_in_target"}, {"fieldName": "absolute_mismatch", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 13, "title": "absolute_mismatch", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "absolute_mismatch"}, {"fieldName": "threshold_mismatch", "numberFormat": "0", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "integer", "displayAs": "number", "visible": true, "order": 14, "title": "threshold_mismatch", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "threshold_mismatch"}, {"fieldName": "mismatch_columns", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 15, "title": "mismatch_columns", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "mismatch_columns"}, {"fieldName": "schema_comparison", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "boolean", "displayAs": "boolean", "visible": true, "order": 16, "title": "schema_comparison", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "schema_comparison"}, {"fieldName": "executed_by", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 19, "title": "executed_by", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "executed_by"}, {"fieldName": "start_ts", "dateTimeFormat": "YYYY-MM-DD HH:mm:ss.SSS", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "datetime", "displayAs": "datetime", "visible": true, "order": 20, "title": "start_ts", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "start_ts"}, {"fieldName": "end_ts", "dateTimeFormat": "YYYY-MM-DD HH:mm:ss.SSS", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "datetime", "displayAs": "datetime", "visible": true, "order": 21, "title": "end_ts", "allowSearch": false, "alignContent": "right", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "end_ts"}]}, "invisibleColumns": [{"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_catalog", "type": "string", "displayAs": "string", "order": 3, "title": "source_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_schema", "type": "string", "displayAs": "string", "order": 4, "title": "source_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_catalog", "type": "string", "displayAs": "string", "order": 6, "title": "target_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_schema", "type": "string", "displayAs": "string", "order": 7, "title": "target_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_table_name", "type": "string", "displayAs": "string", "order": 17, "title": "source_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_table_name", "type": "string", "displayAs": "string", "order": 18, "title": "target_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}], "allowHTMLByDefault": false, "itemsPerPage": 100, "paginationSize": "default", "condensed": true, "withRowNumber": true, "frame": {"showDescription": true, "description": "Summary Table", "showTitle": false}}}, "position": {"x": 0, "y": 10, "width": 6, "height": 7}}, {"widget": {"name": "c27cb052", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_recon_id", "query": {"datasetName": "088adb8a", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_recon_id", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_recon_id"}, {"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_recon_id"}, {"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Recon Id", "description": ""}}}, "position": {"x": 0, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "3ff15994", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_target_table", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_target_table", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table", "query": {"datasetName": "aec31225", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_target_table"}, {"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_target_table"}, {"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Traget Table Name", "description": ""}}}, "position": {"x": 2, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "bec17005", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table", "query": {"datasetName": "088adb8a", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}, {"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table", "query": {"datasetName": "aec31225", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_table"}, {"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_source_table"}, {"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table"}]}, "frame": {"showTitle": true, "showDescription": false, "title": "Source Table Name", "description": ""}}}, "position": {"x": 4, "y": 4, "width": 2, "height": 2}}, {"widget": {"name": "41d78bbf", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type", "query": {"datasetName": "088adb8a", "fields": [{"name": "source_type", "expression": "`source_type`"}, {"name": "source_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_type", "displayName": "source_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_source_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Source Type", "description": "Applied only on summary table"}}}, "position": {"x": 0, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "4bcc8273", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_type", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_type", "expression": "`recon_type`"}, {"name": "recon_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"fieldName": "recon_type", "displayName": "recon_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Category", "description": "Applied only on details table"}}}, "position": {"x": 3, "y": 27, "width": 3, "height": 1}}, {"widget": {"name": "b7d8efe7", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "absolute_mismatch", "expression": "`absolute_mismatch`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "absolute_mismatch", "scale": {"type": "quantitative"}, "displayName": "absolute_mismatch"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Mismatched Records", "showDescription": false}}}, "position": {"x": 0, "y": 37, "width": 3, "height": 6}}, {"widget": {"name": "13d8f169", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "missing_in_target", "expression": "`missing_in_target`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "missing_in_target", "scale": {"type": "quantitative"}, "displayName": "missing_in_target"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Missing in Source"}}}, "position": {"x": 3, "y": 43, "width": 3, "height": 6}}, {"widget": {"name": "1b4c4556", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "missing_in_source", "expression": "`missing_in_source`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "missing_in_source", "scale": {"type": "quantitative"}, "displayName": "missing_in_source"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}, "label": {"show": false}}, "frame": {"showTitle": true, "title": "Missing in Databricks"}}}, "position": {"x": 0, "y": 43, "width": 3, "height": 6}}, {"widget": {"name": "aedc94de", "queries": [{"name": "main_query", "query": {"datasetName": "088adb8a", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "hourly(start_ts)", "expression": "DATE_TRUNC(\"HOUR\", `start_ts`)"}, {"name": "threshold_mismatch", "expression": "`threshold_mismatch`"}], "disaggregated": true}}], "spec": {"version": 3, "widgetType": "area", "encodings": {"x": {"fieldName": "hourly(start_ts)", "scale": {"type": "temporal"}, "displayName": "start_ts"}, "y": {"fieldName": "threshold_mismatch", "scale": {"type": "quantitative"}, "displayName": "threshold_mismatch"}, "color": {"fieldName": "target_table", "scale": {"type": "categorical"}, "displayName": "target_table"}}, "frame": {"showTitle": true, "title": "Threshold Mismatches"}}}, "position": {"x": 3, "y": 37, "width": 3, "height": 6}}, {"widget": {"name": "7d60be10", "queries": [{"name": "main_query", "query": {"datasetName": "07f3d7d5", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "source_column", "expression": "`source_column`"}, {"name": "source_datatype", "expression": "`source_datatype`"}, {"name": "databricks_column", "expression": "`databricks_column`"}, {"name": "databricks_datatype", "expression": "`databricks_datatype`"}, {"name": "is_valid", "expression": "`is_valid`"}], "disaggregated": true}}], "spec": {"version": 1, "widgetType": "table", "encodings": {"columns": [{"fieldName": "recon_id", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 0, "title": "recon_id", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "cellFormat": {"default": {"foregroundColor": null}, "rules": [{"if": {"column": "is_valid", "fn": "=", "literal": "false"}, "value": {"foregroundColor": "#E92828"}}, {"if": {"column": "is_valid", "fn": "=", "literal": "true"}, "value": {"foregroundColor": "#3BD973"}}]}, "displayName": "recon_id"}, {"fieldName": "source_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 4, "title": "source_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_table"}, {"fieldName": "target_table", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 8, "title": "target_table", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "target_table"}, {"fieldName": "source_column", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 9, "title": "source_column", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_column"}, {"fieldName": "source_datatype", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 10, "title": "source_datatype", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "source_datatype"}, {"fieldName": "databricks_column", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 11, "title": "databricks_column", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "databricks_column"}, {"fieldName": "databricks_datatype", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 12, "title": "databricks_datatype", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "databricks_datatype"}, {"fieldName": "is_valid", "booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "type": "string", "displayAs": "string", "visible": true, "order": 13, "title": "is_valid", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false, "displayName": "is_valid"}]}, "invisibleColumns": [{"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_table_name", "type": "string", "displayAs": "string", "order": 1, "title": "source_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_catalog", "type": "string", "displayAs": "string", "order": 2, "title": "source_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "source_schema", "type": "string", "displayAs": "string", "order": 3, "title": "source_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_catalog", "type": "string", "displayAs": "string", "order": 5, "title": "target_catalog", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_schema", "type": "string", "displayAs": "string", "order": 6, "title": "target_schema", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}, {"booleanValues": ["false", "true"], "imageUrlTemplate": "{{ @ }}", "imageTitleTemplate": "{{ @ }}", "imageWidth": "", "imageHeight": "", "linkUrlTemplate": "{{ @ }}", "linkTextTemplate": "{{ @ }}", "linkTitleTemplate": "{{ @ }}", "linkOpenInNewTab": true, "name": "target_table_name", "type": "string", "displayAs": "string", "order": 7, "title": "target_table_name", "allowSearch": false, "alignContent": "left", "allowHTML": false, "highlightLinks": false, "useMonospaceFont": false, "preserveWhitespace": false}], "allowHTMLByDefault": false, "itemsPerPage": 25, "paginationSize": "default", "condensed": true, "withRowNumber": false, "frame": {"showTitle": true, "title": "Schema Details"}}}, "position": {"x": 0, "y": 19, "width": 6, "height": 6}}, {"widget": {"name": "15be1fef", "queries": [{"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_catalog", "query": {"datasetName": "088adb8a", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_catalog", "query": {"datasetName": "07f3d7d5", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_catalog", "query": {"datasetName": "aec31225", "parameters": [{"name": "catalog", "keyword": "catalog"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_catalog"}, {"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_catalog"}, {"parameterName": "catalog", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_catalog"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Catalog", "description": "\u2019remorph' is the default catalog for the metrics table; specify a different catalog name if the metrics are stored elsewhere."}}}, "position": {"x": 0, "y": 2, "width": 3, "height": 2}}, {"widget": {"name": "d1e11541", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_report_type", "query": {"datasetName": "088adb8a", "fields": [{"name": "report_type", "expression": "`report_type`"}, {"name": "report_type_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "report_type", "displayName": "report_type", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_report_type"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Report", "description": "Applied only on summary table"}}}, "position": {"x": 2, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "8e9a7f6d", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_start_ts", "query": {"datasetName": "088adb8a", "fields": [{"name": "start_ts", "expression": "`start_ts`"}, {"name": "start_ts_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-date-range-picker", "encodings": {"fields": [{"fieldName": "start_ts", "displayName": "start_ts", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_start_ts"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "started at", "description": "Applied on all tables"}}}, "position": {"x": 0, "y": 8, "width": 6, "height": 2}}, {"widget": {"name": "12568355", "textbox_spec": "# Main Reconciliation Table\n### This table provides comprehensive information on the report's status, including failure indications, schema matching status, and details on missing and mismatched records."}, "position": {"x": 0, "y": 0, "width": 6, "height": 2}}, {"widget": {"name": "72f3dc17", "textbox_spec": "# Drill Down\n### The details table contains all the sample records for mismatches and missing entries, providing users with exact details to pinpoint the issues."}, "position": {"x": 0, "y": 25, "width": 6, "height": 2}}, {"widget": {"name": "67d6c36d", "textbox_spec": "# Schema Comparison Details\n### This table provides a detailed view of schema mismatches."}, "position": {"x": 0, "y": 17, "width": 6, "height": 2}}, {"widget": {"name": "fca3d649", "textbox_spec": "# Visualization of Missing and Mismatched Records"}, "position": {"x": 0, "y": 35, "width": 6, "height": 2}}, {"widget": {"name": "71206c9b", "queries": [{"name": "main_query", "query": {"datasetName": "aec31225", "fields": [{"name": "value", "expression": "`value`"}, {"name": "key", "expression": "`key`"}, {"name": "recon_id", "expression": "`recon_id`"}, {"name": "source_table", "expression": "`source_table`"}, {"name": "target_table", "expression": "`target_table`"}, {"name": "recon_type", "expression": "`recon_type`"}, {"name": "rn", "expression": "`rn`"}], "disaggregated": false, "orders": [{"direction": "ASC", "expression": "`recon_id`"}, {"direction": "ASC", "expression": "`source_table`"}, {"direction": "ASC", "expression": "`target_table`"}, {"direction": "ASC", "expression": "`recon_type`"}, {"direction": "ASC", "expression": "`rn`"}, {"direction": "ASC", "expression": "`key`"}]}}], "spec": {"version": 3, "widgetType": "pivot", "encodings": {"rows": [{"fieldName": "recon_id", "displayName": "recon_id"}, {"fieldName": "source_table", "displayName": "source_table"}, {"fieldName": "target_table", "displayName": "target_table"}, {"fieldName": "recon_type", "displayName": "recon_type"}, {"fieldName": "rn", "displayName": "rn"}], "columns": [{"fieldName": "key", "displayName": "key"}], "cell": {"fieldName": "value", "cellType": "text", "displayName": "value"}}}}, "position": {"x": 0, "y": 29, "width": 6, "height": 6}}, {"widget": {"name": "14576173", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table", "query": {"datasetName": "aec31225", "fields": [{"name": "target_table", "expression": "`target_table`"}, {"name": "target_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "target_table", "displayName": "target_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_target_table"}]}, "frame": {"showDescription": true, "showTitle": true, "title": "Target Table Name", "description": "Applied only on details table"}}}, "position": {"x": 0, "y": 28, "width": 3, "height": 1}}, {"widget": {"name": "81426d8f", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table", "query": {"datasetName": "aec31225", "fields": [{"name": "source_table", "expression": "`source_table`"}, {"name": "source_table_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "source_table", "displayName": "source_table", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_source_table"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Source Table Name", "description": "Applied only on details table"}}}, "position": {"x": 3, "y": 28, "width": 3, "height": 1}}, {"widget": {"name": "6f02d764", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id", "query": {"datasetName": "aec31225", "fields": [{"name": "recon_id", "expression": "`recon_id`"}, {"name": "recon_id_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "recon_id", "displayName": "recon_id", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_recon_id"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Recon Id", "description": "Applied only on details table"}}}, "position": {"x": 0, "y": 27, "width": 3, "height": 1}}, {"widget": {"name": "832b10ed", "queries": [{"name": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_executed_by", "query": {"datasetName": "088adb8a", "fields": [{"name": "executed_by", "expression": "`executed_by`"}, {"name": "executed_by_associativity", "expression": "COUNT_IF(`associative_filter_predicate_group`)"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-multi-select", "encodings": {"fields": [{"fieldName": "executed_by", "displayName": "executed_by", "queryName": "dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_executed_by"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Executed by", "description": "Applied only on summary table"}}}, "position": {"x": 4, "y": 6, "width": 2, "height": 2}}, {"widget": {"name": "27298c01", "queries": [{"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_schema", "query": {"datasetName": "088adb8a", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_schema", "query": {"datasetName": "07f3d7d5", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}, {"name": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_schema", "query": {"datasetName": "aec31225", "parameters": [{"name": "schema", "keyword": "schema"}], "disaggregated": false}}], "spec": {"version": 2, "widgetType": "filter-single-select", "encodings": {"fields": [{"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257cf103a894826ffd7e2f57d_schema"}, {"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d615a59bb80b99ca72d1db_schema"}, {"parameterName": "schema", "queryName": "parameter_dashboards/01ef183257cb15078fcc146dc3d5dffc/datasets/01ef183257d91a65971719236fe20951_schema"}]}, "frame": {"showTitle": true, "showDescription": true, "title": "Schema", "description": "\u2019reconcile\u2019 is the default schema for the metrics table; specify a different schema name if the metrics are stored elsewhere."}}}, "position": {"x": 3, "y": 2, "width": 3, "height": 2}}]}]} \ No newline at end of file diff --git a/tests/resources/Test_Dashboard_No_Param.lvdash.json b/tests/resources/Test_Dashboard_No_Param.lvdash.json deleted file mode 100644 index 043202049..000000000 --- a/tests/resources/Test_Dashboard_No_Param.lvdash.json +++ /dev/null @@ -1 +0,0 @@ -{"datasets":[{"name":"91a1460a","displayName":"order_status","query":"SELECT * FROM dbsql_tpch_demo.tpch.order_status"}],"pages":[{"name":"7fdcfeb6","displayName":"New Page","layout":[{"widget":{"name":"ba98213a","queries":[{"name":"main_query","query":{"datasetName":"91a1460a","disaggregated":true}}],"spec":{"version":3,"widgetType":"bar","encodings":{}}},"position":{"x":0,"y":0,"width":3,"height":6}}]}]} \ No newline at end of file diff --git a/tests/resources/dashboards/queries/00_description.md b/tests/resources/dashboards/queries/00_description.md new file mode 100644 index 000000000..ffbb93f01 --- /dev/null +++ b/tests/resources/dashboards/queries/00_description.md @@ -0,0 +1 @@ +#Reconcile Dashboard Queries Test \ No newline at end of file diff --git a/tests/resources/dashboards/queries/01_queries.sql b/tests/resources/dashboards/queries/01_queries.sql new file mode 100644 index 000000000..6326979d6 --- /dev/null +++ b/tests/resources/dashboards/queries/01_queries.sql @@ -0,0 +1,8 @@ +SELECT +main.recon_id, +main.source_type, +main.report_type, +main.source_table.`catalog` as source_catalog, +main.source_table.`schema` as source_schema, +main.source_table.table_name as source_table_name +FROM remorph.reconcile.main main diff --git a/tests/resources/dashboards/queries/dashboard.yml b/tests/resources/dashboards/queries/dashboard.yml new file mode 100644 index 000000000..6028c8ded --- /dev/null +++ b/tests/resources/dashboards/queries/dashboard.yml @@ -0,0 +1 @@ +display_name: "Reconciliation Metrics Test" \ No newline at end of file diff --git a/tests/unit/deployment/test_dashboard.py b/tests/unit/deployment/test_dashboard.py index 0f4bc528e..46c027d49 100644 --- a/tests/unit/deployment/test_dashboard.py +++ b/tests/unit/deployment/test_dashboard.py @@ -1,102 +1,107 @@ +import json from pathlib import Path from unittest.mock import create_autospec - +import logging +import pytest from databricks.labs.blueprint.installation import MockInstallation from databricks.labs.blueprint.installer import InstallState from databricks.sdk import WorkspaceClient -from databricks.sdk.errors import InvalidParameterValue +from databricks.sdk.errors import InvalidParameterValue, NotFound from databricks.sdk.service.dashboards import Dashboard +from databricks.sdk.service.dashboards import LifecycleState +from databricks.labs.remorph.config import ReconcileMetadataConfig, ReconcileConfig, DatabaseConfig from databricks.labs.remorph.deployment.dashboard import DashboardDeployment -def test_deploy_new_dashboard(): - dashboard_file = Path(__file__).parent / Path("../../resources/Remorph-Reconciliation.lvdash.json") - ws = create_autospec(WorkspaceClient) - dashboard = Dashboard( - dashboard_id="9c1fbf4ad3449be67d6cb64c8acc730b", - display_name="Remorph-Reconciliation", - ) - ws.lakeview.create.return_value = dashboard - installation = MockInstallation(is_global=False) - install_state = InstallState.from_installation(installation) - name = "Remorph-Reconciliation" - dashboard_publisher = DashboardDeployment(ws, installation, install_state) - dashboard_publisher.deploy(name, dashboard_file) - _, kwargs = ws.lakeview.create.call_args - assert kwargs["serialized_dashboard"] == dashboard_file.read_text() - assert install_state.dashboards[name] == dashboard.dashboard_id - +def _get_dashboard_query(kwargs): + serialized_dashboard = json.loads(kwargs['serialized_dashboard']) + return serialized_dashboard['datasets'][0]['query'] -def test_deploy_new_dashboard_with_params(): - dashboard_file = Path(__file__).parent / Path("../../resources/Remorph-Reconciliation.lvdash.json") - substituted_dashboard_file = Path(__file__).parent / Path( - '../../resources/Remorph-Reconciliation-Substituted.lvdash.json' - ) +def test_deploy_dashboard(): ws = create_autospec(WorkspaceClient) + expected_query = """SELECT + main.recon_id, + main.source_type, + main.report_type, + main.source_table.`catalog` AS source_catalog, + main.source_table.`schema` AS source_schema, + main.source_table.table_name AS source_table_name\nFROM remorph.reconcile.main AS main""".strip() + + dashboard_folder = Path(__file__).parent / Path("../../resources/dashboards") dashboard = Dashboard( dashboard_id="9c1fbf4ad3449be67d6cb64c8acc730b", display_name="Remorph-Reconciliation", ) ws.lakeview.create.return_value = dashboard - installation = MockInstallation(is_global=False) install_state = InstallState.from_installation(installation) - name = "Remorph-Reconciliation" dashboard_publisher = DashboardDeployment(ws, installation, install_state) - dashboard_params = { - "catalog": "remorph1", - "schema": "reconcile1", - } - dashboard_publisher.deploy(name, dashboard_file, parameters=dashboard_params) - _, kwargs = ws.lakeview.create.call_args - assert kwargs["serialized_dashboard"] == substituted_dashboard_file.read_text() - assert install_state.dashboards[name] == dashboard.dashboard_id - - -def test_deploy_new_parameterless_dashboard_with_user_params(): - dashboard_file = Path(__file__).parent / Path("../../resources/Test_Dashboard_No_Param.lvdash.json") - ws = create_autospec(WorkspaceClient) - dashboard = Dashboard( - dashboard_id="8c1fbf4ad3449be67d6cb64c8acc730b", - display_name="Test_Dashboard_No_Param", + reconcile_config = ReconcileConfig( + data_source="oracle", + report_type="all", + secret_scope="remorph_oracle69", + database_config=DatabaseConfig( + source_schema="tpch_sf100069", + target_catalog="tpch69", + target_schema="1000gb69", + ), + metadata_config=ReconcileMetadataConfig(), ) - ws.lakeview.create.return_value = dashboard + dashboard_publisher.deploy(dashboard_folder, reconcile_config) + _, kwargs = ws.lakeview.create.call_args + query = _get_dashboard_query(kwargs) + assert query == expected_query + assert install_state.dashboards["queries"] == dashboard.dashboard_id - installation = MockInstallation(is_global=False) - install_state = InstallState.from_installation(installation) - name = "Test_Dashboard_No_Param" - dashboard_publisher = DashboardDeployment(ws, installation, install_state) - dashboard_params = { - "catalog": "remorph1", - "schema": "reconcile1", - } - dashboard_publisher.deploy(name, dashboard_file, parameters=dashboard_params) - assert install_state.dashboards[name] == dashboard.dashboard_id +@pytest.mark.parametrize("exception", [InvalidParameterValue, NotFound]) +def test_recovery_invalid_dashboard(caplog, exception): + dashboard_folder = Path(__file__).parent / Path("../../resources/dashboards") -def test_deploy_existing_dashboard(): - dashboard_file = Path(__file__).parent / Path("../../resources/Remorph-Reconciliation.lvdash.json") ws = create_autospec(WorkspaceClient) dashboard_id = "9c1fbf4ad3449be67d6cb64c8acc730b" dashboard = Dashboard( dashboard_id=dashboard_id, display_name="Remorph-Reconciliation", ) - ws.lakeview.update.return_value = dashboard - name = "Remorph-Reconciliation" - installation = MockInstallation({"state.json": {"resources": {"dashboards": {name: dashboard_id}}, "version": 1}}) + ws.lakeview.create.return_value = dashboard + ws.lakeview.get.side_effect = exception + # name = "Remorph-Reconciliation" + installation = MockInstallation( + { + "state.json": { + "resources": {"dashboards": {"queries": "8c1fbf4ad3449be67d6cb64c8acc730b"}}, + "version": 1, + }, + } + ) install_state = InstallState.from_installation(installation) dashboard_publisher = DashboardDeployment(ws, installation, install_state) - dashboard_publisher.deploy(name, dashboard_file) - _, kwargs = ws.lakeview.update.call_args - assert kwargs["serialized_dashboard"] == dashboard_file.read_text() - assert install_state.dashboards[name] == dashboard.dashboard_id + reconcile_config = ReconcileConfig( + data_source="oracle", + report_type="all", + secret_scope="remorph_oracle66", + database_config=DatabaseConfig( + source_schema="tpch_sf100066", + target_catalog="tpch66", + target_schema="1000gb66", + ), + metadata_config=ReconcileMetadataConfig(), + ) + with caplog.at_level(logging.DEBUG, logger="databricks.labs.remorph.deployment.dashboard"): + dashboard_publisher.deploy(dashboard_folder, reconcile_config) + assert "Recovering invalid dashboard" in caplog.text + assert "Deleted dangling dashboard" in caplog.text + ws.workspace.delete.assert_called() + ws.lakeview.create.assert_called() + ws.lakeview.update.assert_not_called() + +def test_recovery_trashed_dashboard(caplog): + dashboard_folder = Path(__file__).parent / Path("../../resources/dashboards") -def test_deploy_missing_dashboard(): - dashboard_file = Path(__file__).parent / Path("../../resources/Remorph-Reconciliation.lvdash.json") ws = create_autospec(WorkspaceClient) dashboard_id = "9c1fbf4ad3449be67d6cb64c8acc730b" dashboard = Dashboard( @@ -104,19 +109,30 @@ def test_deploy_missing_dashboard(): display_name="Remorph-Reconciliation", ) ws.lakeview.create.return_value = dashboard - ws.lakeview.update.side_effect = InvalidParameterValue("Dashboard not found") - name = "Remorph-Reconciliation" + ws.lakeview.get.return_value = Dashboard(lifecycle_state=LifecycleState.TRASHED) installation = MockInstallation( { "state.json": { - "resources": {"dashboards": {name: "8c1fbf4ad3449be67d6cb64c8acc730b"}}, + "resources": {"dashboards": {"queries": "8c1fbf4ad3449be67d6cb64c8acc730b"}}, "version": 1, }, } ) install_state = InstallState.from_installation(installation) dashboard_publisher = DashboardDeployment(ws, installation, install_state) - dashboard_publisher.deploy(name, dashboard_file) - _, kwargs = ws.lakeview.create.call_args - assert kwargs["serialized_dashboard"] == dashboard_file.read_text() - assert install_state.dashboards[name] == dashboard.dashboard_id + reconcile_config = ReconcileConfig( + data_source="oracle", + report_type="all", + secret_scope="remorph_oracle77", + database_config=DatabaseConfig( + source_schema="tpch_sf100077", + target_catalog="tpch77", + target_schema="1000gb77", + ), + metadata_config=ReconcileMetadataConfig(), + ) + with caplog.at_level(logging.DEBUG, logger="databricks.labs.remorph.deployment.dashboard"): + dashboard_publisher.deploy(dashboard_folder, reconcile_config) + assert "Recreating trashed dashboard" in caplog.text + ws.lakeview.create.assert_called() + ws.lakeview.update.assert_not_called() diff --git a/tests/unit/deployment/test_recon.py b/tests/unit/deployment/test_recon.py index ea90d6be0..4f8a457eb 100644 --- a/tests/unit/deployment/test_recon.py +++ b/tests/unit/deployment/test_recon.py @@ -116,9 +116,6 @@ def raise_invalid_parameter_err_for_job(jid: str): assert "Reconciliation Deprecated Job 1" not in install_state.jobs assert "Reconciliation Deprecated Job 2" not in install_state.jobs assert "Some other Job" in install_state.jobs - assert "Reconciliation Deprecated Dashboard 1" not in install_state.dashboards - assert "Reconciliation Deprecated Dashboard 2" not in install_state.dashboards - assert "Some other Dashboard" in install_state.dashboards def test_uninstall_missing_config(ws): @@ -211,5 +208,4 @@ def raise_invalid_parameter_err_for_job(jid: str): assert "Reconciliation Runner" not in install_state.jobs assert "Some other Job" in install_state.jobs - assert "Reconciliation Metrics" not in install_state.dashboards - assert "Some other Dashboard" in install_state.dashboards + assert len(install_state.dashboards.keys()) == 0