From 599d932cadb3b68880d9305cb51d7f11b1749ef3 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Thu, 28 Aug 2025 19:59:48 -0400 Subject: [PATCH 01/27] Initial commit --- docs/dqx/docs/guide/index.mdx | 12 +- docs/dqx/docs/guide/quality_checks_apply.mdx | 38 ++- docs/dqx/docs/guide/summary_metrics.mdx | 238 ++++++++++++++++++ docs/dqx/docs/reference/engine.mdx | 2 +- pyproject.toml | 2 +- src/databricks/labs/dqx/base.py | 10 +- src/databricks/labs/dqx/config.py | 3 + .../labs/dqx/contexts/workflow_context.py | 15 +- src/databricks/labs/dqx/engine.py | 139 +++++++--- .../labs/dqx/installer/config_provider.py | 74 ++++++ src/databricks/labs/dqx/observer.py | 58 +++++ .../quality_checker/quality_checker_runner.py | 3 + .../quality_checker_workflow.py | 1 + tests/integration/conftest.py | 45 ++++ tests/integration/test_metrics_workflow.py | 106 ++++++++ tests/integration/test_summary_metrics.py | 217 ++++++++++++++++ tests/unit/test_observer.py | 106 ++++++++ 17 files changed, 1026 insertions(+), 43 deletions(-) create mode 100644 docs/dqx/docs/guide/summary_metrics.mdx create mode 100644 src/databricks/labs/dqx/observer.py create mode 100644 tests/integration/test_metrics_workflow.py create mode 100644 tests/integration/test_summary_metrics.py create mode 100644 tests/unit/test_observer.py diff --git a/docs/dqx/docs/guide/index.mdx b/docs/dqx/docs/guide/index.mdx index 53fcd5f30..4db1dc094 100644 --- a/docs/dqx/docs/guide/index.mdx +++ b/docs/dqx/docs/guide/index.mdx @@ -45,4 +45,14 @@ Quality rules can be defined in the following ways: Additionally, quality rule candidates can be auto-generated using the DQX profiler. -For more details, see the [Quality Checks Definition Guide](/docs/guide/quality_checks_definition). \ No newline at end of file +For more details, see the [Quality Checks Definition Guide](/docs/guide/quality_checks_definition). + +## Summary metrics and monitoring + +DQX can capture and store data summary metrics about your data quality across multiple tables and runs. Metrics are computed lazily and accessible after checked datasets are counted, displayed, or written to a table or files. Users can: + +- Capture quality metrics for each checked dataset +- Track both default (e.g. input/error/warning/valid counts) and custom quality metrics +- Store quality metrics in Delta tables for historical analysis and alerting + +For more details, see the [Summary Metrics Guide](/docs/guide/summary_metrics). \ No newline at end of file diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index 34f294fe8..a89ac9fac 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -51,10 +51,11 @@ You can customize the reporting columns as described in the [additional configur from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule from databricks.labs.dqx.config import InputConfig, OutputConfig + from databricks.labs.dqx.observer import DQObserver from databricks.sdk import WorkspaceClient - dq_engine = DQEngine(WorkspaceClient()) + dq_engine = DQEngine(WorkspaceClient(), observer=DQObserver()) checks = [ DQRowRule( @@ -79,14 +80,14 @@ You can customize the reporting columns as described in the [additional configur input_df = spark.read.table("catalog.schema.input") # Option 1: apply quality checks on the DataFrame and output results to a single DataFrame - valid_and_invalid_df = dq_engine.apply_checks(input_df, checks) + valid_and_invalid_df, metrics = dq_engine.apply_checks(input_df, checks) dq_engine.save_results_in_table( output_df=valid_and_invalid_df, output_config=OutputConfig(location="catalog.schema.output"), ) # Option 2: apply quality checks on the DataFrame and provide valid and invalid (quarantined) DataFrames - valid_df, invalid_df = dq_engine.apply_checks_and_split(input_df, checks) + valid_df, invalid_df, metrics = dq_engine.apply_checks_and_split(input_df, checks) dq_engine.save_results_in_table( output_df=valid_df, quarantine_df=invalid_df, @@ -108,6 +109,15 @@ You can customize the reporting columns as described in the [additional configur input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.output"), ) + + # Option 5: End-to-End approach with summary data: apply quality checks to a table and save the results and summary metrics to output tables + dq_engine_with_metrics.apply_checks_and_save_in_table( + checks=checks, + input_config=InputConfig(location="catalog.schema.input"), + output_config=OutputConfig(location="catalog.schema.valid"), + quarantine_config=OutputConfig(location="catalog.schema.quarantine"), + metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), # Store metrics + ) ``` @@ -478,6 +488,7 @@ The following fields from the [configuration file](/docs/installation/#configura - `input_config`: configuration for the input data. - `output_config`: configuration for the output data. - `quarantine_config`: (optional) configuration for the quarantine data. +- `metrics_config`: (optional) configuration for storing summary metrics. - `checks_location`: location of the quality checks in storage. - `serverless_clusters`: whether to use serverless clusters for running the workflow (default: `true`). Using serverless clusters is recommended as it allows for automated cluster management and scaling. - `e2e_spark_conf`: (optional) spark configuration to use for the e2e workflow, only applicable if `serverless_clusters` is set to `false`. @@ -492,6 +503,7 @@ The following fields from the [configuration file](/docs/installation/#configura - `sample_seed`: seed for reproducible sampling. - `limit`: maximum number of records to analyze. - `extra_params`: (optional) extra parameters to pass to the jobs such as result column names and user_metadata +- `custom_metrics`: (optional) list of Spark SQL expressions for capturing custom summary metrics. - `custom_check_functions`: (optional) custom check functions defined in Python files that can be used in the quality checks. - `reference_tables`: (optional) reference tables that can be used in the quality checks. @@ -519,6 +531,10 @@ Example of the configuration file (relevant fields only): #checkpointLocation: /Volumes/catalog/schema/volume/checkpoint # only applicable if input_config.is_streaming is enabled #trigger: # streaming trigger, only applicable if input_config.is_streaming is enabled # availableNow: true + metrics_config: # optional - summary metrics storage + format: delta + location: main.nytaxi.dq_metrics + mode: append profiler_config: limit: 1000 sample_fraction: 0.3 @@ -532,8 +548,24 @@ Example of the configuration file (relevant fields only): input_config: format: delta location: main.nytaxi.ref + # Global custom metrics for summary statistics (optional) + custom_metrics: + - "sum(array_size(_warnings)) as total_warnings" + - "sum(array_size(_errors)) as total_errors" ``` +## Summary Metrics + +DQX can automatically capture and store summary metrics about your data quality checking processes. When enabled, the system collects both default metrics (input count, error count, warning count, valid count) and any custom metrics you define. + +To enable summary metrics in workflows: + +1. **During Installation**: When prompted, choose to store summary metrics and configure the metrics table location +2. **Configuration File**: Add `metrics_config` to your run configuration and `custom_metrics` at the workspace level +3. **Programmatic Usage**: Create a `DQObserver` with custom metrics when initializing the `DQEngine` + +For detailed information about summary metrics, including examples and best practices, see the [Summary Metrics Guide](/docs/guide/summary_metrics). + ## Quality checking results Quality check results are added as additional columns to the output or quarantine (if defined) DataFrame or tables (if saved). These columns capture the outcomes of the checks performed on the input data. diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx new file mode 100644 index 000000000..05748ccd7 --- /dev/null +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -0,0 +1,238 @@ +--- +sidebar_position: 8 +--- + +import Admonition from '@theme/Admonition'; +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + +# Summary Metrics + +DQX provides comprehensive functionality to capture and store aggregate statistics about your data quality. This allows you to track data quality trends over time, monitor the health of your data pipelines, and gain insights into the overall quality of your datasets. + +## Overview + +Summary metrics in DQX capture both **built-in metrics** (automatically calculated) and **custom metrics** (user-defined SQL expressions) during data quality checking. These metrics are collected using Spark's built-in [Observation](https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/api/pyspark.sql.Observation.html) functionality and can be persisted to tables for historical analysis. + +### Built-in Metrics + +DQX automatically captures the following built-in metrics for every data quality check execution: + +| Metric Name | Data Type | Description | +|-------------|------------------------------------------------|-------------------------------------------------------| +| `input_count` | `int` | Total number of input records processed | +| `error_count` | `int` | Number of records that failed error-level checks | +| `warning_count` | `int` | Number of records that triggered warning-level checks | +| `valid_count` | `int` | Number of records that passed all checks | + +### Custom Metrics + +Users can define custom metrics with Spark SQL expressions. These metrics will be collected in addition to DQX's built-in metrics. + + +Summary metrics are calculated on all records processed by DQX. Complex aggregations can degrade performance when processing large datasets. Be cautious with operations like `DISTINCT` on high-cardinality columns + + +```sql +/* EXAMPLE: Custom data quality summary metrics */ +sum(array_size(_errors)) as total_errors +avg(array_size(_errors)) as errors_per_row +count(case when array_size(_errors) > 1) as count_multiple_errors +``` + +## Usage Examples + +### Basic Usage with Built-in Metrics + +Engine methods (e.g. `apply_checks`) return a Spark `Observation` with 1 or more output `DataFrames`. Data quality metrics can be accessed after any action is performed on the output `DataFrames`. + + + + ```python + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.observer import DQObserver + + # Create observer + observer = DQObserver(name="basic_metrics") + + # Create the engine with the observer + dq_engine = DQEngine(WorkspaceClient(), observer=observer) + + # Apply checks and get metrics + checked_df, observation = dq_engine.apply_checks_by_metadata(df, checks) + + # Trigger action to populate metrics + row_count = checked_df.count() + + # Access metrics + metrics = observation.get + print(f"Input count: {metrics['input_count']}") + print(f"Error count: {metrics['error_count']}") + print(f"Warning count: {metrics['warning_count']}") + print(f"Valid count: {metrics['valid_count']}") + ``` + + + +### Advanced Usage with Custom Metrics + +Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. These metrics will be accessible by name in the returned `Observation`. + + + + ```python + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.observer import DQObserver + from databricks.labs.dqx.config import InputConfig, OutputConfig + + # Define custom metrics + custom_metrics = [ + "sum(array_size(_errors)) as total_errors", + "sum(array_size(_warnings)) as total_warnings", + ] + + # Create the observer with custom metrics + observer = DQObserver( + name="business_metrics", + custom_metrics=custom_metrics + ) + + # Create the engine with the observer + dq_engine = DQEngine(WorkspaceClient(), observer=observer) + + # End-to-end processing with metrics persistence + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=my_checks, + input_config=InputConfig(location="main.raw.customer_data"), + output_config=OutputConfig(location="main.clean.customer_data"), + quarantine_config=OutputConfig(location="main.quarantine.customer_data"), + metrics_config=OutputConfig(location="main.analytics.dq_metrics") + ) + ``` + + + +### Custom Result Column Names + +If you're using custom column names for errors and warnings, pass the column mapping when creating the `DQObserver`. + + + + ```python + from databricks.labs.dqx.config import ExtraParams + + # Configure custom column names + extra_params = ExtraParams( + result_column_names={ + "errors_column": "data_issues", + "warnings_column": "data_alerts" + } + ) + + # Create observer with custom metrics and column names + observer = DQObserver( + name="custom_columns", + result_columns=extra_params.result_column_names, + custom_metrics=["sum(array_size(data_issues)) as total_errors"] + ) + + dq_engine = DQEngine( + workspace_client=WorkspaceClient(), + extra_params=extra_params, + observer=observer + ) + ``` + + + +## Workflow Integration + +### No-Code Approach (Workflows) + +When using DQX workflows, summary metrics are automatically configured based on your installation settings: + +1. **Installation Configuration**: During installation, specify metrics table and custom metrics +2. **Automatic Observer Creation**: Workflows automatically create `DQObserver` when metrics are configured +3. **Metrics Persistence**: Metrics are automatically saved to the configured table after each workflow run + +```bash +# Run quality checker workflow with metrics enabled +databricks labs dqx apply-checks --run-config "production" + +# Run end-to-end workflow with metrics enabled +databricks labs dqx e2e --run-config "production" +``` + +### Configuration File Example + +Metrics will be defined in the `metrics_config` section of your configuration file. + +```yaml +run_configs: +- name: production + input_config: + location: main.raw.sales_data + format: delta + output_config: + location: main.clean.sales_data + format: delta + mode: append + quarantine_config: + location: main.quarantine.sales_data + format: delta + mode: append + metrics_config: # Summary metrics configuration + location: main.analytics.dq_metrics + format: delta + mode: append + checks_location: main.config.quality_checks + +# Global custom metrics (applied to all run configs) +custom_metrics: + - "avg(amount) as average_transaction_amount" + - "sum(case when region = 'US' then amount else 0 end) as us_revenue" + - "count(distinct customer_id) as unique_customers" +``` + +## Metrics Table Schema + +Metrics can be written automatically by specifying a metrics table. The metrics table will contain the following fields: + +| Column Name | Column Type | Description | +|-------------|---------------------|-----------------------------------------------------------| +| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | +| `input_table` | `STRING` | Location of the input dataset | +| `metric_key` | `STRING` | Name of the metric | +| `metric_value` | `STRING` | Value of the metric (as a string) | +| `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | + +## Best Practices + +### Performance Considerations + +1. **Batch Metrics Collection**: Collect metrics during regular data processing after data is written +2. **Partition Metrics Tables**: Consider partitioning metrics tables by date for better query performance +3. **Monitor Metrics Overhead**: Complex custom metrics may impact processing performance + +### Monitoring and Alerting + +1. **Track Trends**: Monitor metrics over time to identify data quality degradation +2. **Set Thresholds**: Establish acceptable ranges for error rates and warning counts +3. **Alert on Anomalies**: Set up alerts when metrics deviate significantly from historical patterns + +The example query below shows how you can analyze metrics persisted to a table. +```sql +/* EXAMPLE: Identify quality degradation */ +SELECT + date_trunc('day', run_ts) as run_date, + avg(error_count * 100.0 / input_count) as avg_error_rate, + avg(warning_count * 100.0 / input_count) as avg_warning_rate +FROM + main.analytics.dq_metrics +WHERE + run_date >= current_date - INTERVAL 30 DAYS +GROUP BY + date_trunc('day', run_date) +ORDER BY + run_date DESC +``` diff --git a/docs/dqx/docs/reference/engine.mdx b/docs/dqx/docs/reference/engine.mdx index 8fa01b6f1..621fb13f7 100644 --- a/docs/dqx/docs/reference/engine.mdx +++ b/docs/dqx/docs/reference/engine.mdx @@ -56,7 +56,7 @@ The following table outlines the available methods of the `DQEngine` and their f | `apply_checks_and_save_in_table` | Applies quality checks using DQRule objects and writes results to valid and invalid Delta table(s) with result columns. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | | `apply_checks_by_metadata` | Applies quality checks defined as a dictionary to the DataFrame and returns a DataFrame with result columns. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | | `apply_checks_by_metadata_and_split` | Applies quality checks defined as a dictionary and returns valid and invalid (quarantine) DataFrames. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary and writes results to valid and invalid Delta table(s) with result columns. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | +| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary and writes results to valid and invalid Delta table(s) with result columns. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | | `validate_checks` | Validates the provided quality checks to ensure they conform to the expected structure and types. | `checks`: List of checks to validate; `custom_check_functions`: (optional) dictionary of custom check functions that can be used; `validate_custom_check_functions`: (optional) if set to True, validates custom check functions (defaults to True). | Yes | | `get_invalid` | Retrieves records from the DataFrame that violate data quality checks (records with warnings and errors). | `df`: Input DataFrame. | Yes | | `get_valid` | Retrieves records from the DataFrame that pass all data quality checks. | `df`: Input DataFrame. | Yes | diff --git a/pyproject.toml b/pyproject.toml index af0e43703..2f6503974 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -475,7 +475,7 @@ max-bool-expr = 5 max-branches = 20 # Maximum number of locals for function / method body. -max-locals = 21 +max-locals = 22 # Maximum number of parents for a class (see R0901). max-parents = 7 diff --git a/src/databricks/labs/dqx/base.py b/src/databricks/labs/dqx/base.py index 4bf3b88e9..56ca6cd1b 100644 --- a/src/databricks/labs/dqx/base.py +++ b/src/databricks/labs/dqx/base.py @@ -2,7 +2,7 @@ from collections.abc import Callable from functools import cached_property from typing import final -from pyspark.sql import DataFrame, SparkSession +from pyspark.sql import DataFrame, Observation, SparkSession from databricks.labs.dqx.checks_validator import ChecksValidationStatus from databricks.labs.dqx.rule import DQRule @@ -53,7 +53,7 @@ class DQEngineCoreBase(DQEngineBase): @abc.abstractmethod def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame. Args: @@ -68,7 +68,7 @@ def apply_checks( @abc.abstractmethod def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -89,7 +89,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """ Apply data quality checks defined as metadata to the given DataFrame. @@ -114,7 +114,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). diff --git a/src/databricks/labs/dqx/config.py b/src/databricks/labs/dqx/config.py index 4eaf67a83..747cf2252 100644 --- a/src/databricks/labs/dqx/config.py +++ b/src/databricks/labs/dqx/config.py @@ -58,6 +58,7 @@ class RunConfig: input_config: InputConfig | None = None output_config: OutputConfig | None = None quarantine_config: OutputConfig | None = None # quarantined data table + metrics_config: OutputConfig | None = None # summary metrics table checks_location: str = ( "checks.yml" # absolute or relative workspace file path or table containing quality rules / checks ) @@ -106,6 +107,8 @@ class WorkspaceConfig: quality_checker_spark_conf: dict[str, str] | None = field(default_factory=dict) e2e_spark_conf: dict[str, str] | None = field(default_factory=dict) + custom_metrics: list[str] | None = None + def get_run_config(self, run_config_name: str | None = "default") -> RunConfig: """Get the run configuration for a given run name, or the default configuration if no run name is provided. diff --git a/src/databricks/labs/dqx/contexts/workflow_context.py b/src/databricks/labs/dqx/contexts/workflow_context.py index 4adb1f28a..528396e63 100644 --- a/src/databricks/labs/dqx/contexts/workflow_context.py +++ b/src/databricks/labs/dqx/contexts/workflow_context.py @@ -9,6 +9,7 @@ from databricks.labs.dqx.config import WorkspaceConfig, RunConfig from databricks.labs.dqx.__about__ import __version__ from databricks.labs.dqx.engine import DQEngine +from databricks.labs.dqx.observer import DQObserver from databricks.labs.dqx.profiler.generator import DQGenerator from databricks.labs.dqx.profiler.profiler import DQProfiler from databricks.labs.dqx.profiler.profiler_runner import ProfilerRunner @@ -88,7 +89,19 @@ def profiler(self) -> ProfilerRunner: @cached_property def quality_checker(self) -> QualityCheckerRunner: """Returns the QualityCheckerRunner instance.""" + # Create observer if metrics are configured + observer = None + if self.run_config.metrics_config: + observer = DQObserver( + name="dqx", + custom_metrics=self.config.custom_metrics, + result_columns=self.config.extra_params.result_column_names if self.config.extra_params else None, + ) + dq_engine = DQEngine( - workspace_client=self.workspace_client, spark=self.spark, extra_params=self.config.extra_params + workspace_client=self.workspace_client, + spark=self.spark, + extra_params=self.config.extra_params, + observer=observer, ) return QualityCheckerRunner(self.spark, dq_engine) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 5734f99ac..53dca7e94 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -4,7 +4,7 @@ from datetime import datetime import pyspark.sql.functions as F -from pyspark.sql import DataFrame, SparkSession +from pyspark.sql import DataFrame, Observation, SparkSession from databricks.labs.dqx.base import DQEngineBase, DQEngineCoreBase from databricks.labs.dqx.checks_serializer import deserialize_checks @@ -35,9 +35,13 @@ from databricks.labs.dqx.checks_validator import ChecksValidator, ChecksValidationStatus from databricks.labs.dqx.schema import dq_result_schema from databricks.labs.dqx.utils import read_input_data, save_dataframe_as_table +from databricks.labs.dqx.observer import DQObserver from databricks.sdk import WorkspaceClient logger = logging.getLogger(__name__) +OBSERVATION_TABLE_SCHEMA = ( + "run_ts timestamp, input_table string, metric_name string, metric_value string, user_metadata map" +) class DQEngineCore(DQEngineCoreBase): @@ -47,6 +51,7 @@ class DQEngineCore(DQEngineCoreBase): workspace_client: WorkspaceClient instance used to access the workspace. spark: Optional SparkSession to use. If not provided, the active session is used. extra_params: Optional extra parameters for the engine, such as result column names and run metadata. + observer: Optional DQObserver for tracking data quality summary metrics. """ def __init__( @@ -54,6 +59,7 @@ def __init__( workspace_client: WorkspaceClient, spark: SparkSession | None = None, extra_params: ExtraParams | None = None, + observer: DQObserver | None = None, ): super().__init__(workspace_client) @@ -71,10 +77,11 @@ def __init__( self.spark = SparkSession.builder.getOrCreate() if spark is None else spark self.run_time = datetime.fromisoformat(extra_params.run_time) self.engine_user_metadata = extra_params.user_metadata + self.observer = observer def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame. Args: @@ -83,10 +90,10 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns. + A DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. """ if not checks: - return self._append_empty_checks(df) + return self._append_empty_checks(df), None if not DQEngineCore._all_are_dq_rules(checks): raise TypeError( @@ -102,12 +109,13 @@ def apply_checks( result_df = self._create_results_array( result_df, warning_checks, self._result_column_names[ColumnArguments.WARNINGS], ref_dfs ) + observed_df, observation = self._observe_metrics(result_df) - return result_df + return observed_df, observation def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -117,23 +125,24 @@ def apply_checks_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and - "bad" (rows with errors or warnings and the corresponding result columns). + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and a dictionary with data quality + summary metrics. """ if not checks: - return df, self._append_empty_checks(df).limit(0) + return df, self._append_empty_checks(df).limit(0), None if not DQEngineCore._all_are_dq_rules(checks): raise TypeError( "All elements in the 'checks' list must be instances of DQRule. Use 'apply_checks_by_metadata_and_split' to pass checks as list of dicts instead." ) - checked_df = self.apply_checks(df, checks, ref_dfs) + checked_df, observation = self.apply_checks(df, checks, ref_dfs) good_df = self.get_valid(checked_df) bad_df = self.get_invalid(checked_df) - return good_df, bad_df + return good_df, bad_df, observation def apply_checks_by_metadata( self, @@ -141,7 +150,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """Apply data quality checks defined as metadata to the given DataFrame. Args: @@ -155,7 +164,7 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns. + DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) @@ -167,7 +176,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -182,12 +191,14 @@ def apply_checks_by_metadata_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame that includes errors and warnings result columns. + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and a dictionary with data quality + summary metrics. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) - good_df, bad_df = self.apply_checks_and_split(df, dq_rule_checks, ref_dfs) - return good_df, bad_df + good_df, bad_df, observation = self.apply_checks_and_split(df, dq_rule_checks, ref_dfs) + return good_df, bad_df, observation @staticmethod def validate_checks( @@ -356,6 +367,25 @@ def _create_results_array( # Ensure the result DataFrame has the same columns as the input DataFrame + the new result column return result_df.select(*df.columns, dest_col) + def _observe_metrics(self, df: DataFrame) -> tuple[DataFrame, Observation | None]: + """ + Adds Spark observable metrics to the input DataFrame. + + Args: + df: Input DataFrame + + Returns: + The unmodified DataFrame with observed metrics and the corresponding Spark Observation + """ + if not self.observer: + return df, None + + observation = self.observer.observation + return ( + df.observe(observation, *[F.expr(metric_statement) for metric_statement in self.observer.metrics]), + observation, + ) + class DQEngine(DQEngineBase): """High-level engine to apply data quality checks and manage IO. @@ -372,11 +402,12 @@ def __init__( extra_params: ExtraParams | None = None, checks_handler_factory: BaseChecksStorageHandlerFactory | None = None, run_config_loader: RunConfigLoader | None = None, + observer: DQObserver | None = None, ): super().__init__(workspace_client) self.spark = SparkSession.builder.getOrCreate() if spark is None else spark - self._engine = engine or DQEngineCore(workspace_client, spark, extra_params) + self._engine = engine or DQEngineCore(workspace_client, spark, extra_params, observer) self._run_config_loader = run_config_loader or RunConfigLoader(workspace_client) self._checks_handler_factory: BaseChecksStorageHandlerFactory = ( checks_handler_factory or ChecksStorageHandlerFactory(self.ws, self.spark) @@ -384,7 +415,7 @@ def __init__( def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame. Args: @@ -393,13 +424,13 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns. + DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. """ return self._engine.apply_checks(df, checks, ref_dfs) def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -409,8 +440,9 @@ def apply_checks_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and - "bad" (rows with errors or warnings and the corresponding result columns). + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and a dictionary with data quality + summary metrics. """ return self._engine.apply_checks_and_split(df, checks, ref_dfs) @@ -420,7 +452,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> DataFrame: + ) -> tuple[DataFrame, Observation | None]: """Apply data quality checks defined as metadata to the given DataFrame. Args: @@ -434,7 +466,7 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns. + DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. """ return self._engine.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) @@ -444,7 +476,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame]: + ) -> tuple[DataFrame, DataFrame, Observation | None]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -459,7 +491,9 @@ def apply_checks_by_metadata_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame that includes errors and warnings result columns. + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and a dictionary with data quality + summary metrics. """ return self._engine.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) @@ -469,6 +503,7 @@ def apply_checks_and_save_in_table( input_config: InputConfig, output_config: OutputConfig, quarantine_config: OutputConfig | None = None, + metrics_config: OutputConfig | None = None, ref_dfs: dict[str, DataFrame] | None = None, ) -> None: """ @@ -485,6 +520,7 @@ def apply_checks_and_save_in_table( input_config: Input configuration (e.g., table/view or file location and read options). output_config: Output configuration (e.g., table name, mode, and write options). quarantine_config: Optional configuration for writing invalid records. + metrics_config: Optional configuration for writing summary metrics. ref_dfs: Optional reference DataFrames used by checks. """ # Read data from the specified table @@ -492,20 +528,39 @@ def apply_checks_and_save_in_table( if quarantine_config: # Split data into good and bad records - good_df, bad_df = self.apply_checks_and_split(df, checks, ref_dfs) + good_df, bad_df, observation = self.apply_checks_and_split(df, checks, ref_dfs) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df = self.apply_checks(df, checks, ref_dfs) + checked_df, observation = self.apply_checks(df, checks, ref_dfs) save_dataframe_as_table(checked_df, output_config) + if observation and metrics_config: + # Create DataFrame with observation metrics - keys as column names, values as data + metrics = observation.get + metrics_df = self.spark.createDataFrame( + [ + [ + self._engine.run_time if isinstance(self._engine, DQEngineCore) else None, + input_config.location, + metric_key, + metric_value, + self._engine.engine_user_metadata if isinstance(self._engine, DQEngineCore) else None, + ] + for metric_key, metric_value in metrics.items() + ], + schema=OBSERVATION_TABLE_SCHEMA, + ) + save_dataframe_as_table(metrics_df, metrics_config) + def apply_checks_by_metadata_and_save_in_table( self, checks: list[dict], input_config: InputConfig, output_config: OutputConfig, quarantine_config: OutputConfig | None = None, + metrics_config: OutputConfig | None = None, custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, ) -> None: @@ -527,6 +582,7 @@ def apply_checks_by_metadata_and_save_in_table( input_config: Input configuration (e.g., table/view or file location and read options). output_config: Output configuration (e.g., table name, mode, and write options). quarantine_config: Optional configuration for writing invalid records. + metrics_config: Optional configuration for writing summary metrics. custom_check_functions: Optional mapping of custom check function names to callables/modules (e.g., globals()). ref_dfs: Optional reference DataFrames used by checks. @@ -536,14 +592,35 @@ def apply_checks_by_metadata_and_save_in_table( if quarantine_config: # Split data into good and bad records - good_df, bad_df = self.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) + good_df, bad_df, observation = self.apply_checks_by_metadata_and_split( + df, checks, custom_check_functions, ref_dfs + ) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + checked_df, observation = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) save_dataframe_as_table(checked_df, output_config) + if observation and metrics_config: + # Create DataFrame with observation metrics - keys as column names, values as data + metrics = observation.get + metrics_df = self.spark.createDataFrame( + [ + [ + self._engine.run_time if isinstance(self._engine, DQEngineCore) else None, + input_config.location, + metric_key, + metric_value, + self._engine.engine_user_metadata if isinstance(self._engine, DQEngineCore) else None, + ] + for metric_key, metric_value in metrics.items() + ], + schema=OBSERVATION_TABLE_SCHEMA, + ) + save_dataframe_as_table(metrics_df, metrics_config) + save_dataframe_as_table(metrics_df, metrics_config) + @staticmethod def validate_checks( checks: list[dict], diff --git a/src/databricks/labs/dqx/installer/config_provider.py b/src/databricks/labs/dqx/installer/config_provider.py index 9be82f6e5..29423bf34 100644 --- a/src/databricks/labs/dqx/installer/config_provider.py +++ b/src/databricks/labs/dqx/installer/config_provider.py @@ -29,6 +29,16 @@ def prompt_new_installation(self) -> WorkspaceConfig: output_config = self._prompt_output_config(is_streaming) quarantine_config = self._prompt_quarantine_config(is_streaming) + store_summary_metrics = self._prompts.confirm( + "Do you want to store summary metrics from data quality checking?" + ) + metrics_config = None + custom_metrics = [] + + if store_summary_metrics: + metrics_config = self._prompt_metrics_config(is_streaming) + custom_metrics = self._prompt_custom_metrics() + checks_location = self._prompts.question( "Provide location of the quality checks definitions, either:\n" "- a filename for storing data quality rules (e.g. checks.yml),\n" @@ -95,6 +105,7 @@ def prompt_new_installation(self) -> WorkspaceConfig: input_config=input_config, output_config=output_config, quarantine_config=quarantine_config, + metrics_config=metrics_config, checks_location=checks_location, warehouse_id=warehouse_id, profiler_config=profiler_config, @@ -109,6 +120,7 @@ def prompt_new_installation(self) -> WorkspaceConfig: quality_checker_override_clusters=quality_checker_override_clusters, e2e_spark_conf=e2e_spark_conf, e2e_override_clusters=e2e_override_clusters, + custom_metrics=custom_metrics, ) def _prompt_clusters_configs(self): @@ -319,3 +331,65 @@ def _prompt_quarantine_config(self, is_streaming: bool) -> OutputConfig | None: trigger=quarantine_trigger_options, ) return None + + def _prompt_metrics_config(self, is_streaming: bool) -> OutputConfig: + """Prompt user for metrics configuration.""" + metrics_table = self._prompts.question( + "Provide table for storing summary metrics in the fully qualified format `catalog.schema.table` or `schema.table`", + valid_regex=r"^([\w]+(?:\.[\w]+){1,2})$", + ) + + metrics_write_mode = self._prompts.question( + "Provide write mode for metrics table (e.g. 'append' or 'overwrite')", + default="append", + valid_regex=r"^(append|overwrite)$", + ) + + metrics_format = self._prompts.question( + "Provide format for the metrics data (e.g. delta, parquet)", + default="delta", + valid_regex=r"^\w.+$", + ) + + metrics_write_options = json.loads( + self._prompts.question( + "Provide additional options for writing the metrics data (e.g. {\"mergeSchema\": \"true\"})", + default="{}", + valid_regex=r"^.*$", + ) + ) + + metrics_trigger_options = {} + if is_streaming: + metrics_trigger_options = json.loads( + self._prompts.question( + "Provide additional options for writing the metrics data using streaming " + "(e.g. {\"availableNow\": true})", + default="{}", + valid_regex=r"^.*$", + ) + ) + + return OutputConfig( + location=metrics_table, + mode=metrics_write_mode, + format=metrics_format, + options=metrics_write_options, + trigger=metrics_trigger_options, + ) + + def _prompt_custom_metrics(self) -> list[str]: + """Prompt user for custom metrics as Spark SQL expressions.""" + custom_metrics_input = self._prompts.question( + "Provide custom metrics as Spark SQL expressions separated by semicolons " + "(e.g. \"count(case when age > 65 then 1 end) as senior_count; avg(salary) as avg_salary\"). " + "Leave blank if no custom metrics are needed.", + default="", + valid_regex=r"^.*$", + ) + + if custom_metrics_input.strip(): + # Split by semicolon and clean up whitespace + custom_metrics = [metric.strip() for metric in custom_metrics_input.split(";") if metric.strip()] + return custom_metrics + return [] diff --git a/src/databricks/labs/dqx/observer.py b/src/databricks/labs/dqx/observer.py new file mode 100644 index 000000000..afa4ba1de --- /dev/null +++ b/src/databricks/labs/dqx/observer.py @@ -0,0 +1,58 @@ +from dataclasses import dataclass, field + +from pyspark.sql import Observation +from databricks.labs.dqx.rule import ColumnArguments, DefaultColumnNames + + +@dataclass +class DQObserver: + """ + Observation class used to track summary metrics about data quality when validating datasets with DQX + + Args: + name: Name of the observations which will be displayed in listener metrics (default is 'dqx'). + custom_metrics: Optional list of SQL expressions defining custom, dataset-level quality metrics + result_columns: Optional dictionary of result column names (e.g. 'errors_column' or 'warnings_column') for + tracking summary metrics (defaults are '_errors' and '_warnings') + """ + + name: str = "dqx" + custom_metrics: list[str] | None = None + result_columns: dict[str, str] | None = field(default_factory=dict) + metrics: list[str] = field(default_factory=list) + + def __post_init__(self) -> None: + self.metrics.extend(self.default_metrics) + if self.custom_metrics: + self.metrics.extend(self.custom_metrics) + + @property + def default_metrics(self) -> list[str]: + """ + Default metrics tracked by the DQObservation. + + Returns: + A list of Spark SQL expressions as strings + """ + result_columns = self.result_columns or {} + errors_column = result_columns.get(ColumnArguments.ERRORS.value, DefaultColumnNames.ERRORS.value) + warnings_column = result_columns.get(ColumnArguments.WARNINGS.value, DefaultColumnNames.WARNINGS.value) + return [ + "count(1) as input_count", + f"count(case when {errors_column} is not null then 1 end) as error_count", + f"count(case when {warnings_column} is not null then 1 end) as warning_count", + f"count(case when {errors_column} is null and {warnings_column} is null then 1 end) as valid_count", + ] + + @property + def observation(self) -> Observation: + """ + Spark `Observation` which can be attached to a `DataFrame` to track summary metrics. Metrics will be collected + when the 1st action is triggered on the attached `DataFrame`. Subsequent operations on the attached `DataFrame` + will not update the observed metrics. See: https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/api/pyspark.sql.Observation.html + for complete documentation. + + Returns: + A Spark `Observation` instance + """ + return Observation(name=self.name) diff --git a/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py b/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py index 79d5511e3..b9419b29f 100644 --- a/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py +++ b/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py @@ -22,6 +22,7 @@ def run( input_config: InputConfig, output_config: OutputConfig, quarantine_config: OutputConfig | None, + metrics_config: OutputConfig | None = None, custom_check_functions: dict[str, str] | None = None, reference_tables: dict[str, InputConfig] | None = None, ) -> None: @@ -33,6 +34,7 @@ def run( input_config: Input data configuration (e.g. table name or file location, read options). output_config: Output data configuration (e.g. table name or file location, write options). quarantine_config: Quarantine data configuration (e.g. table name or file location, write options). + metrics_config: Summary metrics data configuration (e.g. table name or file location, write options). custom_check_functions: A mapping where each key is the name of a function (e.g., "my_func") and each value is the file path to the Python module that defines it. The path can be absolute or relative to the installation folder, and may refer to a local filesystem location, a @@ -50,6 +52,7 @@ def run( input_config=input_config, output_config=output_config, quarantine_config=quarantine_config, + metrics_config=metrics_config, custom_check_functions=custom_check_functions_resolved, ref_dfs=ref_dfs, ) diff --git a/src/databricks/labs/dqx/quality_checker/quality_checker_workflow.py b/src/databricks/labs/dqx/quality_checker/quality_checker_workflow.py index fe51803ad..800102c57 100644 --- a/src/databricks/labs/dqx/quality_checker/quality_checker_workflow.py +++ b/src/databricks/labs/dqx/quality_checker/quality_checker_workflow.py @@ -44,6 +44,7 @@ def apply_checks(self, ctx: WorkflowContext): run_config.input_config, run_config.output_config, run_config.quarantine_config, + run_config.metrics_config, custom_check_functions, run_config.reference_tables, ) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index d833aa199..42addc373 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -91,6 +91,51 @@ def delete(resource) -> None: yield from factory("workflows", lambda **kw: create(spark, **kw), delete) +@pytest.fixture +def setup_workflows_with_metrics(ws, spark, installation_ctx, make_schema, make_table, make_random): + """Set up workflows with metrics configuration for testing.""" + + def create(_spark, **kwargs): + installation_ctx.installation_service.run() + + run_config = _setup_workflows_deps( + installation_ctx, + make_schema, + make_table, + make_random, + checks_location=None, + quarantine=kwargs.get("quarantine", False), + ) + + if kwargs.get("metrics"): + catalog_name = "main" + schema_name = run_config.output_config.location.split('.')[1] + metrics_table_name = f"{catalog_name}.{schema_name}.metrics_{make_random(6).lower()}" + run_config.metrics_config = OutputConfig(location=metrics_table_name) + + custom_metrics = kwargs.get("custom_metrics") + if custom_metrics: + config = installation_ctx.config + config.custom_metrics = custom_metrics + installation_ctx.installation.save(config) + + checks_location = _setup_quality_checks(installation_ctx, _spark, ws) + run_config.checks_location = checks_location + installation_ctx.installation.save(installation_ctx.config) + + return installation_ctx, run_config + + def delete(resource): + ctx, run_config = resource + checks_location = f"{ctx.installation.install_folder()}/{run_config.checks_location}" + try: + ws.workspace.delete(checks_location) + except Exception: + pass + + yield from factory("workflows_with_metrics", lambda **kw: create(spark, **kw), delete) + + def _setup_workflows_deps( ctx, make_schema, diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py new file mode 100644 index 000000000..b9561afaa --- /dev/null +++ b/tests/integration/test_metrics_workflow.py @@ -0,0 +1,106 @@ +def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metrics): + """Test that quality checker workflow saves metrics when configured.""" + ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=["avg(id) as avg_id"]) + + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) + output_df = spark.table(run_config.output_config.location) + output_count = output_df.count() + + expected_metrics = { + "input_count": output_count, + "error_count": 0, + "warning_count": 0, + "valid_count": output_count, + "avg_id": 0, + } + + metrics_rows = spark.table(run_config.metrics_config.location).collect() + actual_metrics = metrics_rows[0].asDict() + assert actual_metrics == expected_metrics + + +def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workflows_with_metrics): + """Test workflow with both quarantine and metrics configurations.""" + ctx, run_config = setup_workflows_with_metrics( + quarantine=True, metrics=True, custom_metrics=["count(distinct name) as unique_names"] + ) + + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) + output_df = spark.table(run_config.output_config.location) + output_count = output_df.count() + quarantine_df = spark.table(run_config.quarantine_config.location) + quarantine_count = quarantine_df.count() + + expected_metrics = { + "input_count": output_count + quarantine_count, + "error_count": quarantine_count, + "warning_count": 0, + "valid_count": output_count, + "unique_names": 0, + } + + metrics_rows = spark.table(run_config.metrics_config.location).collect() + actual_metrics = metrics_rows[0].asDict() + assert actual_metrics == expected_metrics + + +def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): + """Test that e2e workflow generates checks and applies them with metrics.""" + ctx, run_config = setup_workflows_with_metrics( + metrics=True, custom_metrics=["max(id) as max_id", "min(id) as min_id"] + ) + + ctx.deployed_workflows.run_workflow("e2e", run_config.name) + output_df = spark.table(run_config.output_config.location) + output_count = output_df.count() + + expected_metrics = { + "input_count": output_count, + "error_count": 0, + "warning_count": 0, + "valid_count": output_count, + "max_id": 1, + "min_id": 0, + } + + metrics_rows = spark.table(run_config.metrics_config.location).collect() + actual_metrics = metrics_rows[0].asDict() + assert actual_metrics == expected_metrics + + +def test_multiple_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): + """Test workflow with multiple custom metrics.""" + custom_metrics = [ + "avg(id) as average_id", + "sum(id) as total_id", + "count(distinct name) as unique_names", + "max(id) - min(id) as id_range", + ] + + ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=custom_metrics) + + expected_metrics = { + "input_count": 0, + "error_count": 0, + "warning_count": 0, + "valid_count": 0, + "average_id": 0, + "total_id": 0, + "unique_names": 0, + "id_range": 0, + } + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) + metrics_df = spark.table(run_config.metrics_config.location) + metrics_rows = metrics_df.collect() + actual_metrics = metrics_rows[0].asDict() + + assert actual_metrics == expected_metrics + + +def test_quality_checker_workflow_without_metrics_config(ws, setup_workflows_with_metrics): + """Test that workflow works normally when metrics config is not provided.""" + ctx, run_config = setup_workflows_with_metrics(metrics=False) + assert run_config.metrics_config is None + + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) + assert not ws.tables.exists(run_config.metrics_config.location).table_exists diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py new file mode 100644 index 000000000..c6209d630 --- /dev/null +++ b/tests/integration/test_summary_metrics.py @@ -0,0 +1,217 @@ +from datetime import datetime, timezone +from pyspark.sql.types import StructType, StructField, IntegerType, StringType + +from databricks.labs.dqx.config import InputConfig, OutputConfig, ExtraParams +from databricks.labs.dqx.engine import DQEngine +from databricks.labs.dqx.observer import DQObserver + + +RUN_TIME = datetime(2025, 1, 1, 0, 0, 0, 0, tzinfo=timezone.utc) +EXTRA_PARAMS = ExtraParams(run_time=RUN_TIME.isoformat()) +TEST_SCHEMA = StructType( + [ + StructField("id", IntegerType(), True), + StructField("name", StringType(), True), + StructField("age", IntegerType(), True), + StructField("salary", IntegerType(), True), + ] +) +TEST_CHECKS = [ + { + "name": "id_is_not_null", + "criticality": "error", + "check": {"function": "is_not_null", "arguments": {"column": "id"}}, + }, + { + "name": "name_is_not_null_and_not_empty", + "criticality": "warn", + "check": {"function": "is_not_null_and_not_empty", "arguments": {"column": "name"}}, + }, +] + + +def test_engine_with_observer_before_action(ws, spark): + """Test that summary metrics are empty before running a Spark action.""" + custom_metrics = ["avg(age) as avg_age", "sum(salary) as total_salary"] + observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], # This will trigger an error + [4, None, 28, 55000], # This will trigger a warning + ], + TEST_SCHEMA, + ) + _, observation = dq_engine.apply_checks_by_metadata(test_df, TEST_CHECKS) + + actual_metrics = observation.get + assert actual_metrics == {} + + +def test_engine_with_observer_after_action(ws, spark): + """Test that summary metrics can be accessed after running a Spark action like df.count().""" + custom_metrics = [ + "avg(case when _errors is not null then age else null end) as avg_error_age", + "sum(case when _warnings is not null then salary else null end) as total_warning_salary", + ] + observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + checked_df, observation = dq_engine.apply_checks_by_metadata(test_df, TEST_CHECKS) + checked_df.count() # Trigger an action to get the metrics + + expected_metrics = { + "input_count": 4, + "error_count": 1, + "warning_count": 1, + "valid_count": 2, + "avg_error_age": 35.0, + "total_warning_salary": 55000, + } + actual_metrics = observation.get + assert actual_metrics == expected_metrics + + +def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): + """Test that summary metrics are written to the table defined in metrics_config.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + + custom_metrics = [ + "avg(case when _errors is not null then age else null end) as avg_error_age", + "sum(case when _warnings is not null then salary else null end) as total_warning_salary", + ] + observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + + test_df.write.saveAsTable(input_table_name) + input_config = InputConfig(location=input_table_name) + output_config = OutputConfig(location=output_table_name) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config + ) + + expected_metrics = { + "input_count": 4, + "error_count": 1, + "warning_count": 1, + "valid_count": 2, + "avg_error_age": 35.0, + "total_warning_salary": 55000, + } + metrics_rows = spark.table(metrics_table_name).collect() + actual_metrics = metrics_rows[0].asDict() + + assert actual_metrics == expected_metrics + + +def test_engine_metrics_with_quarantine_and_metrics(ws, spark, make_schema, make_random): + """Test that metrics work correctly when using both quarantine and metrics configs.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + quarantine_table_name = f"{catalog_name}.{schema.name}.quarantine_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + + custom_metrics = [ + "avg(case when _errors is not null then age else null end) as avg_error_age", + "sum(case when _warnings is not null then salary else null end) as total_warning_salary", + ] + observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + + test_df.write.saveAsTable(input_table_name) + input_config = InputConfig(location=input_table_name) + output_config = OutputConfig(location=output_table_name) + quarantine_config = OutputConfig(location=quarantine_table_name) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=TEST_CHECKS, + input_config=input_config, + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) + + expected_metrics = { + "input_count": 4, + "error_count": 1, + "warning_count": 1, + "valid_count": 2, + "avg_error_age": 35.0, + "total_warning_salary": 55000, + } + metrics_rows = spark.table(metrics_table_name).collect() + actual_metrics = metrics_rows[0].asDict() + + assert actual_metrics == expected_metrics + + +def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_random): + """Test that no metrics are saved when observer is not configured.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + + dq_engine = DQEngine(workspace_client=ws, spark=spark, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + ], + TEST_SCHEMA, + ) + + test_df.write.saveAsTable(input_table_name) + + input_config = InputConfig(location=input_table_name) + output_config = OutputConfig(location=output_table_name) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config + ) + + assert not ws.tables.exists(metrics_table_name).table_exists diff --git a/tests/unit/test_observer.py b/tests/unit/test_observer.py new file mode 100644 index 000000000..def6db99d --- /dev/null +++ b/tests/unit/test_observer.py @@ -0,0 +1,106 @@ +"""Unit tests for DQObserver class.""" + +from pyspark.sql import Observation +from databricks.labs.dqx.observer import DQObserver +from databricks.labs.dqx.rule import DefaultColumnNames + + +def test_dq_observer_default_initialization(): + """Test DQObserver default initialization.""" + observer = DQObserver() + assert observer.name == "dqx" + assert observer.custom_metrics is None + assert observer.result_columns == {} + + expected_default_metrics = [ + "count(1) as input_count", + "count(case when _errors is not null then 1 end) as error_count", + "count(case when _warnings is not null then 1 end) as warning_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + ] + assert observer.metrics == expected_default_metrics + + +def test_dq_observer_with_custom_metrics(): + """Test DQObserver with custom metrics.""" + custom_metrics = ["avg(age) as avg_age", "count(case when age > 65 then 1 end) as senior_count"] + + observer = DQObserver(name="custom_observer", custom_metrics=custom_metrics) + assert observer.name == "custom_observer" + assert observer.custom_metrics == custom_metrics + + expected_metrics = [ + "count(1) as input_count", + "count(case when _errors is not null then 1 end) as error_count", + "count(case when _warnings is not null then 1 end) as warning_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + "avg(age) as avg_age", + "count(case when age > 65 then 1 end) as senior_count", + ] + assert observer.metrics == expected_metrics + + +def test_dq_observer_with_custom_result_columns(): + """Test DQObserver with custom result column names.""" + custom_columns = {"errors_column": "custom_errors", "warnings_column": "custom_warnings"} + observer = DQObserver(result_columns=custom_columns) + + expected_default_metrics = [ + "count(1) as input_count", + "count(case when custom_errors is not null then 1 end) as error_count", + "count(case when custom_warnings is not null then 1 end) as warning_count", + "count(case when custom_errors is null and custom_warnings is null then 1 end) as valid_count", + ] + assert observer.metrics == expected_default_metrics + + +def test_dq_observer_with_custom_metrics_and_columns(): + """Test DQObserver with both custom metrics and custom result columns.""" + custom_metrics = ["max(salary) as max_salary"] + custom_columns = {"errors_column": "issues", "warnings_column": "alerts"} + + observer = DQObserver(custom_metrics=custom_metrics, result_columns=custom_columns) + + expected_metrics = [ + "count(1) as input_count", + "count(case when issues is not null then 1 end) as error_count", + "count(case when alerts is not null then 1 end) as warning_count", + "count(case when issues is null and alerts is null then 1 end) as valid_count", + "max(salary) as max_salary", + ] + assert observer.metrics == expected_metrics + + +def test_dq_observer_empty_custom_metrics(): + """Test DQObserver with empty custom metrics list.""" + observer = DQObserver(custom_metrics=[]) + + expected_default_metrics = [ + "count(1) as input_count", + "count(case when _errors is not null then 1 end) as error_count", + "count(case when _warnings is not null then 1 end) as warning_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + ] + assert observer.metrics == expected_default_metrics + + +def test_dq_observer_default_column_names(): + """Test that DQObserver uses correct default column names.""" + observer = DQObserver() + errors_column = DefaultColumnNames.ERRORS.value + warnings_column = DefaultColumnNames.WARNINGS.value + + assert f"count(case when {errors_column} is not null then 1 end) as error_count" in observer.metrics + assert f"count(case when {warnings_column} is not null then 1 end) as warning_count" in observer.metrics + assert ( + f"count(case when {errors_column} is null and {warnings_column} is null then 1 end) as valid_count" + in observer.metrics + ) + + +def test_dq_observer_observation_property(): + """Test that the observation property creates a Spark Observation.""" + observer = DQObserver(name="test_obs") + observation = observer.observation + assert isinstance(observation, Observation) + assert observation is not None From b767a0758cdf5db0df8f9f0e22acd4b6962f6865 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:35:01 +0200 Subject: [PATCH 02/27] Update docs/dqx/docs/guide/quality_checks_apply.mdx --- docs/dqx/docs/guide/quality_checks_apply.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index a89ac9fac..3069f288e 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -116,7 +116,7 @@ You can customize the reporting columns as described in the [additional configur input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.valid"), quarantine_config=OutputConfig(location="catalog.schema.quarantine"), - metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), # Store metrics + metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), ) ``` From 1dd380670cd61bc5d923306f6eb6f4eab8fb58c3 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:35:09 +0200 Subject: [PATCH 03/27] Update docs/dqx/docs/guide/quality_checks_apply.mdx --- docs/dqx/docs/guide/quality_checks_apply.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index 3069f288e..75d66cde3 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -111,7 +111,7 @@ You can customize the reporting columns as described in the [additional configur ) # Option 5: End-to-End approach with summary data: apply quality checks to a table and save the results and summary metrics to output tables - dq_engine_with_metrics.apply_checks_and_save_in_table( + dq_engine.apply_checks_and_save_in_table( checks=checks, input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.valid"), From 71c0178d9adad7ff949c3e2dc8743fe8938743f1 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:35:53 +0200 Subject: [PATCH 04/27] Update docs/dqx/docs/guide/summary_metrics.mdx --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 05748ccd7..d66b8a83e 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -30,7 +30,7 @@ DQX automatically captures the following built-in metrics for every data quality Users can define custom metrics with Spark SQL expressions. These metrics will be collected in addition to DQX's built-in metrics. -Summary metrics are calculated on all records processed by DQX. Complex aggregations can degrade performance when processing large datasets. Be cautious with operations like `DISTINCT` on high-cardinality columns +Summary metrics are calculated on all records processed by DQX. Complex aggregations can degrade performance when processing large datasets. Be cautious with operations like `DISTINCT` on high-cardinality columns. ```sql From 843316f7711113ce5e7695626949d8aa2859fd89 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:36:15 +0200 Subject: [PATCH 05/27] Update docs/dqx/docs/guide/summary_metrics.mdx updated docs --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index d66b8a83e..11d9568ff 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -36,7 +36,7 @@ Summary metrics are calculated on all records processed by DQX. Complex aggregat ```sql /* EXAMPLE: Custom data quality summary metrics */ sum(array_size(_errors)) as total_errors -avg(array_size(_errors)) as errors_per_row +avg(array_size(_errors)) as errors_avg count(case when array_size(_errors) > 1) as count_multiple_errors ``` From a252923430dfdbfbf2ba2fce5197e335a5e0bb90 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:36:35 +0200 Subject: [PATCH 06/27] Update docs/dqx/docs/guide/summary_metrics.mdx updated docs --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 11d9568ff..907ba98aa 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -44,7 +44,7 @@ count(case when array_size(_errors) > 1) as count_multiple_errors ### Basic Usage with Built-in Metrics -Engine methods (e.g. `apply_checks`) return a Spark `Observation` with 1 or more output `DataFrames`. Data quality metrics can be accessed after any action is performed on the output `DataFrames`. +Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark `Observation` with 1 or more output `DataFrames`. Data quality metrics can be accessed after any action is performed on the output `DataFrames`. From fb440820edc6218f00321c1e2b4798fa29dc188b Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:37:06 +0200 Subject: [PATCH 07/27] Update docs/dqx/docs/guide/summary_metrics.mdx updated comment --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 907ba98aa..71fada0ae 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -61,7 +61,7 @@ Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark # Apply checks and get metrics checked_df, observation = dq_engine.apply_checks_by_metadata(df, checks) - # Trigger action to populate metrics + # Trigger an action to populate metrics (e.g. count, save to a table) row_count = checked_df.count() # Access metrics From ae18de4effb75b0b6e49ad5b49bb7a49ce32638b Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:37:20 +0200 Subject: [PATCH 08/27] Update docs/dqx/docs/guide/summary_metrics.mdx --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 71fada0ae..daa51535a 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -76,7 +76,7 @@ Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark ### Advanced Usage with Custom Metrics -Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. These metrics will be accessible by name in the returned `Observation`. +Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. Default and these custom metrics will be accessible by name in the returned `Observation`. From 4bd4cc440fee4cd0b161b08a3675e33d88da2cb5 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:37:31 +0200 Subject: [PATCH 09/27] Update docs/dqx/docs/guide/summary_metrics.mdx --- docs/dqx/docs/guide/summary_metrics.mdx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index daa51535a..2b785360d 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -149,7 +149,7 @@ If you're using custom column names for errors and warnings, pass the column map ### No-Code Approach (Workflows) -When using DQX workflows, summary metrics are automatically configured based on your installation settings: +When using DQX workflows, summary metrics are automatically configured based on your installation [configuration file](/docs/installation/#configuration-file): 1. **Installation Configuration**: During installation, specify metrics table and custom metrics 2. **Automatic Observer Creation**: Workflows automatically create `DQObserver` when metrics are configured From 152f28f48db2bfa6b81952ef80b97f023ee59158 Mon Sep 17 00:00:00 2001 From: Marcin Wojtyczka Date: Fri, 29 Aug 2025 12:38:31 +0200 Subject: [PATCH 10/27] Update src/databricks/labs/dqx/engine.py removed duplicate --- src/databricks/labs/dqx/engine.py | 1 - 1 file changed, 1 deletion(-) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 53dca7e94..f2faee296 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -619,7 +619,6 @@ def apply_checks_by_metadata_and_save_in_table( schema=OBSERVATION_TABLE_SCHEMA, ) save_dataframe_as_table(metrics_df, metrics_config) - save_dataframe_as_table(metrics_df, metrics_config) @staticmethod def validate_checks( From cede7f0f8a2caf9c1c0bb5e2d60c8d5e0f3b1148 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Mon, 1 Sep 2025 13:10:17 -0400 Subject: [PATCH 11/27] Update engine methods, docs, and tests --- docs/dqx/docs/guide/index.mdx | 1 + docs/dqx/docs/guide/quality_checks_apply.mdx | 56 +++++++++-- docs/dqx/docs/reference/engine.mdx | 12 +-- src/databricks/labs/dqx/base.py | 16 +-- .../labs/dqx/contexts/workflow_context.py | 1 - src/databricks/labs/dqx/engine.py | 97 ++++++++++++------- src/databricks/labs/dqx/observer.py | 2 +- tests/integration/test_metrics_workflow.py | 5 +- tests/integration/test_summary_metrics.py | 3 +- 9 files changed, 128 insertions(+), 65 deletions(-) diff --git a/docs/dqx/docs/guide/index.mdx b/docs/dqx/docs/guide/index.mdx index 4db1dc094..0abfd4ebb 100644 --- a/docs/dqx/docs/guide/index.mdx +++ b/docs/dqx/docs/guide/index.mdx @@ -54,5 +54,6 @@ DQX can capture and store data summary metrics about your data quality across mu - Capture quality metrics for each checked dataset - Track both default (e.g. input/error/warning/valid counts) and custom quality metrics - Store quality metrics in Delta tables for historical analysis and alerting +- Centralize quality metrics across datasets, jobs, or job runs in a unified data quality history table For more details, see the [Summary Metrics Guide](/docs/guide/summary_metrics). \ No newline at end of file diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index 75d66cde3..2f9fdf940 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -165,14 +165,14 @@ In the example below, checks are defined in YAML syntax for convenience and then input_df = spark.read.table("catalog.schema.input") # Option 1: apply quality checks on the DataFrame and output results as a single DataFrame - valid_and_invalid_df = dq_engine.apply_checks_by_metadata(input_df, checks) + valid_and_invalid_df, metrics = dq_engine.apply_checks_by_metadata(input_df, checks) dq_engine.save_results_in_table( output_df=valid_and_invalid_df, output_config=OutputConfig(location="catalog.schema.output"), ) # Option 2: apply quality checks on the DataFrame and provide valid and invalid (quarantined) DataFrames - valid_df, invalid_df = dq_engine.apply_checks_by_metadata_and_split(input_df, checks) + valid_df, invalid_df, metrics = dq_engine.apply_checks_by_metadata_and_split(input_df, checks) dq_engine.save_results_in_table( output_df=valid_df, quarantine_df=invalid_df, @@ -194,6 +194,16 @@ In the example below, checks are defined in YAML syntax for convenience and then input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.output"), ) + + # Option 5: End-to-End approach with summary data: apply quality checks to a table and save the results and summary metrics to output tables + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=checks, + input_config=InputConfig(location="catalog.schema.input"), + output_config=OutputConfig(location="catalog.schema.valid"), + quarantine_config=OutputConfig(location="catalog.schema.quarantine"), + metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), + ) + ``` ``` @@ -503,7 +513,7 @@ The following fields from the [configuration file](/docs/installation/#configura - `sample_seed`: seed for reproducible sampling. - `limit`: maximum number of records to analyze. - `extra_params`: (optional) extra parameters to pass to the jobs such as result column names and user_metadata -- `custom_metrics`: (optional) list of Spark SQL expressions for capturing custom summary metrics. +- `custom_metrics`: (optional) list of Spark SQL expressions for capturing custom summary metrics. If no summary metrics are provided, the default metrics (number of input, valid, warning, and error rows) will be tracked. - `custom_check_functions`: (optional) custom check functions defined in Python files that can be used in the quality checks. - `reference_tables`: (optional) reference tables that can be used in the quality checks. @@ -556,13 +566,43 @@ Example of the configuration file (relevant fields only): ## Summary Metrics -DQX can automatically capture and store summary metrics about your data quality checking processes. When enabled, the system collects both default metrics (input count, error count, warning count, valid count) and any custom metrics you define. +DQX can automatically capture and store summary metrics about your data quality checking processes. When enabled, the system collects both default metrics (input count, error count, warning count, valid count) and any custom metrics you define. Metrics can be configured programmatically or via a configuration file when installing DQX using the Databricks CLI. + +### Enabling summary metrics programmatically + +To enable summary metrics programmatically, create and pass a `DQObserver` when initializing the `DQEngine`: + + + + ```python + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.observer import DQObserver + + # set up a DQObserver with default name and metrics + dq_observer = DQObserver() + dq_engine = DQEngine(observer=dq_observer) + + # set up a DQObserver with a custom name (this name will be appended to the result table) + dq_observer = DQObserver(name="my_dqx_run") + dq_engine = DQEngine(observer=dq_observer) + + # set up a DQObserver with custom metrics + custom_metrics = [ + "count(case when array_size(_errors) > 1 then 1 end) as rows_with_multiple_errors", + "count(case when array_size(_warnings) > 1 then 1 end) as rows_with_multiple_warnings" + ] + dq_observer = DQObserver(metrics=custom_metrics) + dq_engine = DQEngine(observer=dq_observer) + ``` + + + +### Enabling summary metrics in DQX workflows -To enable summary metrics in workflows: +Summary metrics can also be enabled in DQX workflows installed using the Databricks CLI. Metrics are configured: -1. **During Installation**: When prompted, choose to store summary metrics and configure the metrics table location -2. **Configuration File**: Add `metrics_config` to your run configuration and `custom_metrics` at the workspace level -3. **Programmatic Usage**: Create a `DQObserver` with custom metrics when initializing the `DQEngine` +1. **During Installation**: When prompted, choose to store summary metrics and configure the default metrics table location +2. **Configuration File**: 2. **Configuration File**: Add `custom_metrics` and `metrics_config` to the [configuration file](/docs/installation/#configuration-file) For detailed information about summary metrics, including examples and best practices, see the [Summary Metrics Guide](/docs/guide/summary_metrics). diff --git a/docs/dqx/docs/reference/engine.mdx b/docs/dqx/docs/reference/engine.mdx index 621fb13f7..3ba6f14d7 100644 --- a/docs/dqx/docs/reference/engine.mdx +++ b/docs/dqx/docs/reference/engine.mdx @@ -51,12 +51,12 @@ The following table outlines the available methods of the `DQEngine` and their f **Available DQX engine methods** | Method | Description | Arguments | Supports local execution | | ---------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ----------------------------------------- | -| `apply_checks` | Applies quality checks to the DataFrame and returns a DataFrame with result columns. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_and_split` | Applies quality checks to the DataFrame and returns valid and invalid (quarantine) DataFrames with result columns. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_and_save_in_table` | Applies quality checks using DQRule objects and writes results to valid and invalid Delta table(s) with result columns. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | -| `apply_checks_by_metadata` | Applies quality checks defined as a dictionary to the DataFrame and returns a DataFrame with result columns. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_by_metadata_and_split` | Applies quality checks defined as a dictionary and returns valid and invalid (quarantine) DataFrames. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary and writes results to valid and invalid Delta table(s) with result columns. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | +| `apply_checks` | Applies quality checks to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_and_split` | Applies quality checks to the DataFrame; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_and_save_in_table` | Applies quality checks using DQRule objects, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | +| `apply_checks_by_metadata` | Applies quality checks defined as a dictionary to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_by_metadata_and_split` | Applies quality checks defined as a dictionary; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | | `validate_checks` | Validates the provided quality checks to ensure they conform to the expected structure and types. | `checks`: List of checks to validate; `custom_check_functions`: (optional) dictionary of custom check functions that can be used; `validate_custom_check_functions`: (optional) if set to True, validates custom check functions (defaults to True). | Yes | | `get_invalid` | Retrieves records from the DataFrame that violate data quality checks (records with warnings and errors). | `df`: Input DataFrame. | Yes | | `get_valid` | Retrieves records from the DataFrame that pass all data quality checks. | `df`: Input DataFrame. | Yes | diff --git a/src/databricks/labs/dqx/base.py b/src/databricks/labs/dqx/base.py index 56ca6cd1b..f5d2edf0a 100644 --- a/src/databricks/labs/dqx/base.py +++ b/src/databricks/labs/dqx/base.py @@ -62,7 +62,8 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame that includes errors and warnings result columns. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ @abc.abstractmethod @@ -78,8 +79,9 @@ def apply_checks_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and - "bad" (rows with errors or warnings and the corresponding result columns). + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality + summary metrics. """ @abc.abstractmethod @@ -104,7 +106,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame that includes errors and warnings result columns. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ @abc.abstractmethod @@ -129,8 +132,9 @@ def apply_checks_by_metadata_and_split( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and - "bad" (rows with errors or warnings and the corresponding result columns). + A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality + summary metrics. """ @staticmethod diff --git a/src/databricks/labs/dqx/contexts/workflow_context.py b/src/databricks/labs/dqx/contexts/workflow_context.py index 528396e63..e95e1f5ea 100644 --- a/src/databricks/labs/dqx/contexts/workflow_context.py +++ b/src/databricks/labs/dqx/contexts/workflow_context.py @@ -93,7 +93,6 @@ def quality_checker(self) -> QualityCheckerRunner: observer = None if self.run_config.metrics_config: observer = DQObserver( - name="dqx", custom_metrics=self.config.custom_metrics, result_columns=self.config.extra_params.result_column_names if self.config.extra_params else None, ) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index f2faee296..773c274ce 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -2,6 +2,7 @@ import warnings from collections.abc import Callable from datetime import datetime +from functools import cached_property import pyspark.sql.functions as F from pyspark.sql import DataFrame, Observation, SparkSession @@ -40,7 +41,9 @@ logger = logging.getLogger(__name__) OBSERVATION_TABLE_SCHEMA = ( - "run_ts timestamp, input_table string, metric_name string, metric_value string, user_metadata map" + "run_name string, input_location string, output_location string, quarantine_location string, " + "checks_location string, metric_name string, metric_value string, run_ts timestamp, error_column_name string, " + "warning_column_name string, user_metadata map" ) @@ -79,6 +82,10 @@ def __init__( self.engine_user_metadata = extra_params.user_metadata self.observer = observer + @cached_property + def result_column_names(self) -> dict[ColumnArguments, str]: + return self._result_column_names + def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None ) -> tuple[DataFrame, Observation | None]: @@ -90,7 +97,8 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ if not checks: return self._append_empty_checks(df), None @@ -126,7 +134,7 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and a dictionary with data quality + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality summary metrics. """ if not checks: @@ -164,7 +172,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) @@ -192,7 +201,7 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and a dictionary with data quality + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality summary metrics. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) @@ -424,7 +433,8 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ return self._engine.apply_checks(df, checks, ref_dfs) @@ -441,7 +451,7 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and a dictionary with data quality + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality summary metrics. """ return self._engine.apply_checks_and_split(df, checks, ref_dfs) @@ -466,7 +476,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - DataFrame with errors and warnings result columns and a dictionary with data quality summary metrics. + A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary + metrics. """ return self._engine.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) @@ -492,7 +503,7 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and a dictionary with data quality + with errors or warnings and the corresponding result columns) and an Observation which tracks data quality summary metrics. """ return self._engine.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) @@ -538,20 +549,7 @@ def apply_checks_and_save_in_table( if observation and metrics_config: # Create DataFrame with observation metrics - keys as column names, values as data - metrics = observation.get - metrics_df = self.spark.createDataFrame( - [ - [ - self._engine.run_time if isinstance(self._engine, DQEngineCore) else None, - input_config.location, - metric_key, - metric_value, - self._engine.engine_user_metadata if isinstance(self._engine, DQEngineCore) else None, - ] - for metric_key, metric_value in metrics.items() - ], - schema=OBSERVATION_TABLE_SCHEMA, - ) + metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) save_dataframe_as_table(metrics_df, metrics_config) def apply_checks_by_metadata_and_save_in_table( @@ -604,20 +602,7 @@ def apply_checks_by_metadata_and_save_in_table( if observation and metrics_config: # Create DataFrame with observation metrics - keys as column names, values as data - metrics = observation.get - metrics_df = self.spark.createDataFrame( - [ - [ - self._engine.run_time if isinstance(self._engine, DQEngineCore) else None, - input_config.location, - metric_key, - metric_value, - self._engine.engine_user_metadata if isinstance(self._engine, DQEngineCore) else None, - ] - for metric_key, metric_value in metrics.items() - ], - schema=OBSERVATION_TABLE_SCHEMA, - ) + metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) save_dataframe_as_table(metrics_df, metrics_config) @staticmethod @@ -984,3 +969,41 @@ def load_run_config( return RunConfigLoader(self.ws).load_run_config( run_config_name=run_config_name, assume_user=assume_user, product_name=product_name ) + + def _build_metrics_df( + self, + input_config: InputConfig, + output_config: OutputConfig, + quarantine_config: OutputConfig | None, + checks_config: FileChecksStorageConfig | TableChecksStorageConfig | None, + ) -> DataFrame: + engine = self._engine + + if not isinstance(engine, DQEngineCore) or not engine.observer: + raise ValueError("Property 'observer' must be provided to DQEngine to track summary metrics") + + result_column_names = engine.result_column_names or { + ColumnArguments.ERRORS: DefaultColumnNames.ERRORS.value, + ColumnArguments.WARNINGS: DefaultColumnNames.WARNINGS.value, + } + observation = engine.observer.observation + metrics = observation.get + return self.spark.createDataFrame( + [ + [ + engine.observer.name, + input_config.location, + output_config.location, + None if not quarantine_config else quarantine_config.location, + None if not checks_config else checks_config.location, + metric_key, + metric_value, + engine.run_time, + result_column_names.get(ColumnArguments.ERRORS), + result_column_names.get(ColumnArguments.WARNINGS), + engine.engine_user_metadata, + ] + for metric_key, metric_value in metrics.items() + ], + schema=OBSERVATION_TABLE_SCHEMA, + ) diff --git a/src/databricks/labs/dqx/observer.py b/src/databricks/labs/dqx/observer.py index afa4ba1de..dadc5d8d9 100644 --- a/src/databricks/labs/dqx/observer.py +++ b/src/databricks/labs/dqx/observer.py @@ -49,7 +49,7 @@ def observation(self) -> Observation: """ Spark `Observation` which can be attached to a `DataFrame` to track summary metrics. Metrics will be collected when the 1st action is triggered on the attached `DataFrame`. Subsequent operations on the attached `DataFrame` - will not update the observed metrics. See: https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/api/pyspark.sql.Observation.html + will not update the observed metrics. See: [PySpark Observation](https://spark.apache.org/docs/latest/api/python/reference/pyspark.sql/api/pyspark.sql.Observation.html) for complete documentation. Returns: diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index b9561afaa..92ed893b6 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -99,8 +99,5 @@ def test_multiple_custom_metrics_in_workflow(spark, setup_workflows_with_metrics def test_quality_checker_workflow_without_metrics_config(ws, setup_workflows_with_metrics): """Test that workflow works normally when metrics config is not provided.""" - ctx, run_config = setup_workflows_with_metrics(metrics=False) + _, run_config = setup_workflows_with_metrics(metrics=False) assert run_config.metrics_config is None - - ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) - assert not ws.tables.exists(run_config.metrics_config.location).table_exists diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index c6209d630..54be8b295 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -32,8 +32,7 @@ def test_engine_with_observer_before_action(ws, spark): """Test that summary metrics are empty before running a Spark action.""" - custom_metrics = ["avg(age) as avg_age", "sum(salary) as total_salary"] - observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + observer = DQObserver(name="test_observer") dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) test_df = spark.createDataFrame( From fd12138356417e5d809a8ad92bdadfcc9f47023c Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Fri, 5 Sep 2025 10:22:26 -0400 Subject: [PATCH 12/27] Add streaming support --- docs/dqx/docs/guide/quality_checks_apply.mdx | 20 +-- docs/dqx/docs/guide/summary_metrics.mdx | 32 +++-- .../labs/dqx/contexts/workflow_context.py | 4 +- src/databricks/labs/dqx/engine.py | 48 +++++-- src/databricks/labs/dqx/metrics_listener.py | 66 +++++++++ .../dqx/{observer.py => metrics_observer.py} | 15 ++- tests/integration/test_summary_metrics.py | 127 ++++++++++++++++-- tests/unit/test_observer.py | 16 +-- 8 files changed, 273 insertions(+), 55 deletions(-) create mode 100644 src/databricks/labs/dqx/metrics_listener.py rename src/databricks/labs/dqx/{observer.py => metrics_observer.py} (89%) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index c86a79e72..d804f53a7 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -51,11 +51,11 @@ You can customize the reporting columns as described in the [additional configur from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule from databricks.labs.dqx.config import InputConfig, OutputConfig - from databricks.labs.dqx.observer import DQObserver + from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.sdk import WorkspaceClient - dq_engine = DQEngine(WorkspaceClient(), observer=DQObserver()) + dq_engine = DQEngine(WorkspaceClient(), observer=DQMetricsObserver()) checks = [ DQRowRule( @@ -572,28 +572,28 @@ DQX can automatically capture and store summary metrics about your data quality ### Enabling summary metrics programmatically -To enable summary metrics programmatically, create and pass a `DQObserver` when initializing the `DQEngine`: +To enable summary metrics programmatically, create and pass a `DQMetricsObserver` when initializing the `DQEngine`: ```python from databricks.labs.dqx.engine import DQEngine - from databricks.labs.dqx.observer import DQObserver + from databricks.labs.dqx.metrics_observer import DQMetricsObserver - # set up a DQObserver with default name and metrics - dq_observer = DQObserver() + # set up a DQMetricsObserver with default name and metrics + dq_observer = DQMetricsObserver() dq_engine = DQEngine(observer=dq_observer) - # set up a DQObserver with a custom name (this name will be appended to the result table) - dq_observer = DQObserver(name="my_dqx_run") + # set up a DQMetricsObserver with a custom name (this name will be appended to the result table) + dq_observer = DQMetricsObserver(name="my_dqx_run") dq_engine = DQEngine(observer=dq_observer) - # set up a DQObserver with custom metrics + # set up a DQMetricsObserver with custom metrics custom_metrics = [ "count(case when array_size(_errors) > 1 then 1 end) as rows_with_multiple_errors", "count(case when array_size(_warnings) > 1 then 1 end) as rows_with_multiple_warnings" ] - dq_observer = DQObserver(metrics=custom_metrics) + dq_observer = DQMetricsObserver(custom_metrics=custom_metrics) dq_engine = DQEngine(observer=dq_observer) ``` diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 2b785360d..fe17833c9 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -44,22 +44,26 @@ count(case when array_size(_errors) > 1) as count_multiple_errors ### Basic Usage with Built-in Metrics -Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark `Observation` with 1 or more output `DataFrames`. Data quality metrics can be accessed after any action is performed on the output `DataFrames`. +Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark Observation with 1 or more output DataFrames. Data quality metrics can be accessed after any action is performed on the output DataFrames. + + +Metrics are not directly accessible from the returned Spark Observation when data is processed with streaming. Use DQX's built-in methods to persist streaming metrics to an output table or implement a custom [StreamingQueryListener](https://spark.apache.org/docs/latest/api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryListener.html) to process the metrics. + ```python from databricks.labs.dqx.engine import DQEngine - from databricks.labs.dqx.observer import DQObserver + from databricks.labs.dqx.metrics_observer import DQMetricsObserver # Create observer - observer = DQObserver(name="basic_metrics") + observer = DQMetricsObserver(name="basic_metrics") # Create the engine with the observer - dq_engine = DQEngine(WorkspaceClient(), observer=observer) + engine = DQEngine(WorkspaceClient(), observer=observer) # Apply checks and get metrics - checked_df, observation = dq_engine.apply_checks_by_metadata(df, checks) + checked_df, observation = engine.apply_checks_by_metadata(df, checks) # Trigger an action to populate metrics (e.g. count, save to a table) row_count = checked_df.count() @@ -76,13 +80,13 @@ Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark ### Advanced Usage with Custom Metrics -Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. Default and these custom metrics will be accessible by name in the returned `Observation`. +Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserver`. Custom metrics should be defined as Spark SQL expressions with column aliases and will be accessible by their alias. ```python from databricks.labs.dqx.engine import DQEngine - from databricks.labs.dqx.observer import DQObserver + from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.labs.dqx.config import InputConfig, OutputConfig # Define custom metrics @@ -92,16 +96,16 @@ Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. Def ] # Create the observer with custom metrics - observer = DQObserver( + observer = DQMetricsObserver( name="business_metrics", custom_metrics=custom_metrics ) # Create the engine with the observer - dq_engine = DQEngine(WorkspaceClient(), observer=observer) + engine = DQEngine(WorkspaceClient(), observer=observer) # End-to-end processing with metrics persistence - dq_engine.apply_checks_by_metadata_and_save_in_table( + engine.apply_checks_by_metadata_and_save_in_table( checks=my_checks, input_config=InputConfig(location="main.raw.customer_data"), output_config=OutputConfig(location="main.clean.customer_data"), @@ -114,7 +118,7 @@ Pass custom metrics as Spark SQL expressions when creating the `DQObserver`. Def ### Custom Result Column Names -If you're using custom column names for errors and warnings, pass the column mapping when creating the `DQObserver`. +When using custom column names for errors and warnings, pass the column mapping when creating the `DQMetricsObserver`. @@ -130,13 +134,13 @@ If you're using custom column names for errors and warnings, pass the column map ) # Create observer with custom metrics and column names - observer = DQObserver( + observer = DQMetricsObserver( name="custom_columns", result_columns=extra_params.result_column_names, custom_metrics=["sum(array_size(data_issues)) as total_errors"] ) - dq_engine = DQEngine( + engine = DQEngine( workspace_client=WorkspaceClient(), extra_params=extra_params, observer=observer @@ -152,7 +156,7 @@ If you're using custom column names for errors and warnings, pass the column map When using DQX workflows, summary metrics are automatically configured based on your installation [configuration file](/docs/installation/#configuration-file): 1. **Installation Configuration**: During installation, specify metrics table and custom metrics -2. **Automatic Observer Creation**: Workflows automatically create `DQObserver` when metrics are configured +2. **Automatic Observer Creation**: Workflows automatically create `DQMetricsObserver` when metrics are configured 3. **Metrics Persistence**: Metrics are automatically saved to the configured table after each workflow run ```bash diff --git a/src/databricks/labs/dqx/contexts/workflow_context.py b/src/databricks/labs/dqx/contexts/workflow_context.py index e95e1f5ea..f8322cafd 100644 --- a/src/databricks/labs/dqx/contexts/workflow_context.py +++ b/src/databricks/labs/dqx/contexts/workflow_context.py @@ -9,7 +9,7 @@ from databricks.labs.dqx.config import WorkspaceConfig, RunConfig from databricks.labs.dqx.__about__ import __version__ from databricks.labs.dqx.engine import DQEngine -from databricks.labs.dqx.observer import DQObserver +from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.labs.dqx.profiler.generator import DQGenerator from databricks.labs.dqx.profiler.profiler import DQProfiler from databricks.labs.dqx.profiler.profiler_runner import ProfilerRunner @@ -92,7 +92,7 @@ def quality_checker(self) -> QualityCheckerRunner: # Create observer if metrics are configured observer = None if self.run_config.metrics_config: - observer = DQObserver( + observer = DQMetricsObserver( custom_metrics=self.config.custom_metrics, result_columns=self.config.extra_params.result_column_names if self.config.extra_params else None, ) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 773c274ce..f546d6740 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -36,7 +36,8 @@ from databricks.labs.dqx.checks_validator import ChecksValidator, ChecksValidationStatus from databricks.labs.dqx.schema import dq_result_schema from databricks.labs.dqx.utils import read_input_data, save_dataframe_as_table -from databricks.labs.dqx.observer import DQObserver +from databricks.labs.dqx.metrics_observer import DQMetricsObserver +from databricks.labs.dqx.metrics_listener import StreamingMetricsListener from databricks.sdk import WorkspaceClient logger = logging.getLogger(__name__) @@ -54,7 +55,7 @@ class DQEngineCore(DQEngineCoreBase): workspace_client: WorkspaceClient instance used to access the workspace. spark: Optional SparkSession to use. If not provided, the active session is used. extra_params: Optional extra parameters for the engine, such as result column names and run metadata. - observer: Optional DQObserver for tracking data quality summary metrics. + observer: Optional DQMetricsObserver for tracking data quality summary metrics. """ def __init__( @@ -62,7 +63,7 @@ def __init__( workspace_client: WorkspaceClient, spark: SparkSession | None = None, extra_params: ExtraParams | None = None, - observer: DQObserver | None = None, + observer: DQMetricsObserver | None = None, ): super().__init__(workspace_client) @@ -390,8 +391,9 @@ def _observe_metrics(self, df: DataFrame) -> tuple[DataFrame, Observation | None return df, None observation = self.observer.observation + observer_id = self.observer.id return ( - df.observe(observation, *[F.expr(metric_statement) for metric_statement in self.observer.metrics]), + df.observe(observer_id, *[F.expr(metric_statement) for metric_statement in self.observer.metrics]), observation, ) @@ -411,7 +413,7 @@ def __init__( extra_params: ExtraParams | None = None, checks_handler_factory: BaseChecksStorageHandlerFactory | None = None, run_config_loader: RunConfigLoader | None = None, - observer: DQObserver | None = None, + observer: DQMetricsObserver | None = None, ): super().__init__(workspace_client) @@ -537,17 +539,21 @@ def apply_checks_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) + listener = self._get_metrics_listener(input_config, output_config, quarantine_config, metrics_config) + if listener: + self.spark.streams.addListener(listener) + if quarantine_config: # Split data into good and bad records - good_df, bad_df, observation = self.apply_checks_and_split(df, checks, ref_dfs) + good_df, bad_df, _ = self.apply_checks_and_split(df, checks, ref_dfs) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df, observation = self.apply_checks(df, checks, ref_dfs) + checked_df, _ = self.apply_checks(df, checks, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if observation and metrics_config: + if metrics_config and not listener: # Create DataFrame with observation metrics - keys as column names, values as data metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) save_dataframe_as_table(metrics_df, metrics_config) @@ -588,6 +594,10 @@ def apply_checks_by_metadata_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) + listener = self._get_metrics_listener(input_config, output_config, quarantine_config, metrics_config) + if listener: + self.spark.streams.addListener(listener) + if quarantine_config: # Split data into good and bad records good_df, bad_df, observation = self.apply_checks_by_metadata_and_split( @@ -600,7 +610,7 @@ def apply_checks_by_metadata_and_save_in_table( checked_df, observation = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if observation and metrics_config: + if observation and metrics_config and not listener: # Create DataFrame with observation metrics - keys as column names, values as data metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) save_dataframe_as_table(metrics_df, metrics_config) @@ -973,7 +983,7 @@ def load_run_config( def _build_metrics_df( self, input_config: InputConfig, - output_config: OutputConfig, + output_config: OutputConfig | None, quarantine_config: OutputConfig | None, checks_config: FileChecksStorageConfig | TableChecksStorageConfig | None, ) -> DataFrame: @@ -993,7 +1003,7 @@ def _build_metrics_df( [ engine.observer.name, input_config.location, - output_config.location, + None if not output_config else output_config.location, None if not quarantine_config else quarantine_config.location, None if not checks_config else checks_config.location, metric_key, @@ -1007,3 +1017,19 @@ def _build_metrics_df( ], schema=OBSERVATION_TABLE_SCHEMA, ) + + def _get_metrics_listener( + self, + input_config: InputConfig, + output_config: OutputConfig | None, + quarantine_config: OutputConfig | None, + metrics_config: OutputConfig | None, + ) -> StreamingMetricsListener | None: + if input_config.is_streaming and metrics_config: + return StreamingMetricsListener( + save_dataframe_as_table( + self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None), + metrics_config, + ) + ) + return None diff --git a/src/databricks/labs/dqx/metrics_listener.py b/src/databricks/labs/dqx/metrics_listener.py new file mode 100644 index 000000000..d6cae1842 --- /dev/null +++ b/src/databricks/labs/dqx/metrics_listener.py @@ -0,0 +1,66 @@ +import logging +from collections.abc import Callable +from pyspark.sql.streaming.listener import ( + StreamingQueryListener, + QueryStartedEvent, + QueryProgressEvent, + QueryIdleEvent, + QueryTerminatedEvent, +) + + +logger = logging.getLogger(__name__) + + +class StreamingMetricsListener(StreamingQueryListener): + """ + Implements a Spark `StreamingQueryListener` for writing data quality summary metrics to an output destination. See + the [Spark documentation](https://spark.apache.org/docs/latest/api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryListener.html) + for detailed information about `StreamingQueryListener`. + + Args: + handler: Python `Callable` which handles writing metrics to an output destination. Called for every + micro-batch processed by the streaming query. + """ + + def __init__(self, handler: Callable) -> None: + self._handler = handler + + def onQueryStarted(self, event: QueryStartedEvent) -> None: + """ + Writes a message to the standard output logs when a streaming query starts. + + Args: + event: A `QueryStartedEvent` with details about the streaming query + """ + logger.debug(f"Streaming query '{event.name}' started run ID '{event.runId}'") + + def onQueryProgress(self, event: QueryProgressEvent) -> None: + """ + Writes the custom metrics from the DQObserver to the output destination. + + Args: + event: A `QueryProgressEvent` with details about the last processed micro-batch + """ + self._handler() + + def onQueryIdle(self, event: QueryIdleEvent) -> None: + """ + Writes a message to the standard output logs when a streaming query is idle. + + Args: + event: A `QueryIdleEvent` with details about the streaming query + """ + logger.debug(f"Streaming query run '{event.runId}' was reported idle") + + def onQueryTerminated(self, event: QueryTerminatedEvent) -> None: + """ + Writes a message to the standard output logs when a streaming query stops due to cancellation or failure. + + Args: + event: A `QueryTerminatedEvent` with details about the streaming query + """ + if event.exception: + logger.debug(f"Streaming query run '{event.runId}' failed with error: {event.exception}") + + logger.debug(f"Streaming query run '{event.runId}' stopped") diff --git a/src/databricks/labs/dqx/observer.py b/src/databricks/labs/dqx/metrics_observer.py similarity index 89% rename from src/databricks/labs/dqx/observer.py rename to src/databricks/labs/dqx/metrics_observer.py index dadc5d8d9..91454e0af 100644 --- a/src/databricks/labs/dqx/observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -1,11 +1,12 @@ from dataclasses import dataclass, field +from uuid import uuid4 from pyspark.sql import Observation from databricks.labs.dqx.rule import ColumnArguments, DefaultColumnNames @dataclass -class DQObserver: +class DQMetricsObserver: """ Observation class used to track summary metrics about data quality when validating datasets with DQX @@ -20,12 +21,24 @@ class DQObserver: custom_metrics: list[str] | None = None result_columns: dict[str, str] | None = field(default_factory=dict) metrics: list[str] = field(default_factory=list) + _id: str = field(default_factory=str) def __post_init__(self) -> None: + self._id = str(uuid4()) self.metrics.extend(self.default_metrics) if self.custom_metrics: self.metrics.extend(self.custom_metrics) + @property + def id(self) -> str: + """ + Observer ID. + + Returns: + Unique observation ID + """ + return self._id + @property def default_metrics(self) -> list[str]: """ diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 54be8b295..d53a1b986 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -3,7 +3,7 @@ from databricks.labs.dqx.config import InputConfig, OutputConfig, ExtraParams from databricks.labs.dqx.engine import DQEngine -from databricks.labs.dqx.observer import DQObserver +from databricks.labs.dqx.metrics_observer import DQMetricsObserver RUN_TIME = datetime(2025, 1, 1, 0, 0, 0, 0, tzinfo=timezone.utc) @@ -32,7 +32,7 @@ def test_engine_with_observer_before_action(ws, spark): """Test that summary metrics are empty before running a Spark action.""" - observer = DQObserver(name="test_observer") + observer = DQMetricsObserver(name="test_observer") dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) test_df = spark.createDataFrame( @@ -56,7 +56,7 @@ def test_engine_with_observer_after_action(ws, spark): "avg(case when _errors is not null then age else null end) as avg_error_age", "sum(case when _warnings is not null then salary else null end) as total_warning_salary", ] - observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + observer = DQMetricsObserver(name="test_observer", custom_metrics=custom_metrics) dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) test_df = spark.createDataFrame( @@ -95,7 +95,7 @@ def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): "avg(case when _errors is not null then age else null end) as avg_error_age", "sum(case when _warnings is not null then salary else null end) as total_warning_salary", ] - observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + observer = DQMetricsObserver(name="test_observer", custom_metrics=custom_metrics) dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) test_df = spark.createDataFrame( @@ -125,8 +125,7 @@ def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): "avg_error_age": 35.0, "total_warning_salary": 55000, } - metrics_rows = spark.table(metrics_table_name).collect() - actual_metrics = metrics_rows[0].asDict() + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() assert actual_metrics == expected_metrics @@ -144,7 +143,7 @@ def test_engine_metrics_with_quarantine_and_metrics(ws, spark, make_schema, make "avg(case when _errors is not null then age else null end) as avg_error_age", "sum(case when _warnings is not null then salary else null end) as total_warning_salary", ] - observer = DQObserver(name="test_observer", custom_metrics=custom_metrics) + observer = DQMetricsObserver(name="test_observer", custom_metrics=custom_metrics) dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) test_df = spark.createDataFrame( @@ -179,8 +178,7 @@ def test_engine_metrics_with_quarantine_and_metrics(ws, spark, make_schema, make "avg_error_age": 35.0, "total_warning_salary": 55000, } - metrics_rows = spark.table(metrics_table_name).collect() - actual_metrics = metrics_rows[0].asDict() + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() assert actual_metrics == expected_metrics @@ -214,3 +212,114 @@ def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_r ) assert not ws.tables.exists(metrics_table_name).table_exists + + +def test_engine_streaming_metrics_saved_to_table(ws, spark, make_schema, make_random, make_volume): + """Test that summary metrics are written to the table when using streaming with metrics_config.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + volume = make_volume(catalog_name=catalog_name, schema_name=schema.name) + checkpoint_location = f"/Volumes/{volume.catalog_name}/{volume.schema_name}/{volume.name}/{make_random(8).lower()}" + + custom_metrics = [ + "avg(case when _errors is not null then age else null end) as avg_error_age", + "sum(case when _warnings is not null then salary else null end) as total_warning_salary", + ] + observer = DQMetricsObserver(name="test_streaming_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + + test_df.write.saveAsTable(input_table_name) + input_config = InputConfig(location=input_table_name, is_streaming=True) + output_config = OutputConfig( + location=output_table_name, options={"checkPointLocation": checkpoint_location}, trigger={"availableNow": True} + ) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config + ) + + expected_metrics = { + "input_count": 4, + "error_count": 1, + "warning_count": 1, + "valid_count": 2, + "avg_error_age": 35.0, + "total_warning_salary": 55000, + } + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + assert actual_metrics == expected_metrics + + +def test_engine_streaming_with_quarantine_and_metrics(ws, spark, make_schema, make_random, make_volume): + """Test that streaming metrics work correctly when using both quarantine and metrics configs.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + quarantine_table_name = f"{catalog_name}.{schema.name}.quarantine_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + volume = make_volume(catalog_name=catalog_name, schema_name=schema.name) + + custom_metrics = [ + "avg(case when _errors is not null then age else null end) as avg_error_age", + "sum(case when _warnings is not null then salary else null end) as total_warning_salary", + ] + observer = DQMetricsObserver(name="test_streaming_quarantine_observer", custom_metrics=custom_metrics) + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + + test_df.write.saveAsTable(input_table_name) + input_config = InputConfig(location=input_table_name, is_streaming=True) + output_config = OutputConfig( + location=output_table_name, + options={"checkPointLocation": f"/Volumes/{volume.catalog_name}/{volume.schema_name}/{volume.name}/output"}, + trigger={"availableNow": True}, + ) + quarantine_config = OutputConfig( + location=quarantine_table_name, + options={"checkPointLocation": f"/Volumes/{volume.catalog_name}/{volume.schema_name}/{volume.name}/quarantine"}, + trigger={"availableNow": True}, + ) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.apply_checks_by_metadata_and_save_in_table( + checks=TEST_CHECKS, + input_config=input_config, + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) + + expected_metrics = { + "input_count": 4, + "error_count": 1, + "warning_count": 1, + "valid_count": 2, + "avg_error_age": 35.0, + "total_warning_salary": 55000, + } + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + assert actual_metrics == expected_metrics diff --git a/tests/unit/test_observer.py b/tests/unit/test_observer.py index def6db99d..cc784066b 100644 --- a/tests/unit/test_observer.py +++ b/tests/unit/test_observer.py @@ -1,13 +1,13 @@ """Unit tests for DQObserver class.""" from pyspark.sql import Observation -from databricks.labs.dqx.observer import DQObserver +from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.labs.dqx.rule import DefaultColumnNames def test_dq_observer_default_initialization(): """Test DQObserver default initialization.""" - observer = DQObserver() + observer = DQMetricsObserver() assert observer.name == "dqx" assert observer.custom_metrics is None assert observer.result_columns == {} @@ -25,7 +25,7 @@ def test_dq_observer_with_custom_metrics(): """Test DQObserver with custom metrics.""" custom_metrics = ["avg(age) as avg_age", "count(case when age > 65 then 1 end) as senior_count"] - observer = DQObserver(name="custom_observer", custom_metrics=custom_metrics) + observer = DQMetricsObserver(name="custom_observer", custom_metrics=custom_metrics) assert observer.name == "custom_observer" assert observer.custom_metrics == custom_metrics @@ -43,7 +43,7 @@ def test_dq_observer_with_custom_metrics(): def test_dq_observer_with_custom_result_columns(): """Test DQObserver with custom result column names.""" custom_columns = {"errors_column": "custom_errors", "warnings_column": "custom_warnings"} - observer = DQObserver(result_columns=custom_columns) + observer = DQMetricsObserver(result_columns=custom_columns) expected_default_metrics = [ "count(1) as input_count", @@ -59,7 +59,7 @@ def test_dq_observer_with_custom_metrics_and_columns(): custom_metrics = ["max(salary) as max_salary"] custom_columns = {"errors_column": "issues", "warnings_column": "alerts"} - observer = DQObserver(custom_metrics=custom_metrics, result_columns=custom_columns) + observer = DQMetricsObserver(custom_metrics=custom_metrics, result_columns=custom_columns) expected_metrics = [ "count(1) as input_count", @@ -73,7 +73,7 @@ def test_dq_observer_with_custom_metrics_and_columns(): def test_dq_observer_empty_custom_metrics(): """Test DQObserver with empty custom metrics list.""" - observer = DQObserver(custom_metrics=[]) + observer = DQMetricsObserver(custom_metrics=[]) expected_default_metrics = [ "count(1) as input_count", @@ -86,7 +86,7 @@ def test_dq_observer_empty_custom_metrics(): def test_dq_observer_default_column_names(): """Test that DQObserver uses correct default column names.""" - observer = DQObserver() + observer = DQMetricsObserver() errors_column = DefaultColumnNames.ERRORS.value warnings_column = DefaultColumnNames.WARNINGS.value @@ -100,7 +100,7 @@ def test_dq_observer_default_column_names(): def test_dq_observer_observation_property(): """Test that the observation property creates a Spark Observation.""" - observer = DQObserver(name="test_obs") + observer = DQMetricsObserver(name="test_obs") observation = observer.observation assert isinstance(observation, Observation) assert observation is not None From 9916b0fd19412831fcf3c72a0fbe05fa996d3897 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Sun, 21 Sep 2025 15:21:30 -0400 Subject: [PATCH 13/27] Refactor --- docs/dqx/docs/guide/quality_checks_apply.mdx | 16 +++--- docs/dqx/docs/guide/summary_metrics.mdx | 57 +++++++++++-------- src/databricks/labs/dqx/base.py | 18 +++--- src/databricks/labs/dqx/config.py | 8 ++- .../labs/dqx/contexts/workflow_context.py | 5 +- src/databricks/labs/dqx/engine.py | 27 +++++---- .../labs/dqx/installer/config_provider.py | 52 ++++++++--------- src/databricks/labs/dqx/metrics_listener.py | 18 ++---- src/databricks/labs/dqx/metrics_observer.py | 48 +++++++++------- tests/integration/test_metrics_workflow.py | 16 ++++-- .../test_save_and_load_checks_from_table.py | 2 + tests/unit/test_observer.py | 32 ----------- 12 files changed, 143 insertions(+), 156 deletions(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index ba4eea33f..dfce25a04 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -55,7 +55,7 @@ You can customize the reporting columns as described in the [additional configur from databricks.sdk import WorkspaceClient - dq_engine = DQEngine(WorkspaceClient(), observer=DQMetricsObserver()) + dq_engine = DQEngine(WorkspaceClient()) checks = [ DQRowRule( @@ -80,14 +80,14 @@ You can customize the reporting columns as described in the [additional configur input_df = spark.read.table("catalog.schema.input") # Option 1: apply quality checks on the DataFrame and output results to a single DataFrame - valid_and_invalid_df, metrics = dq_engine.apply_checks(input_df, checks) + valid_and_invalid_df = dq_engine.apply_checks(input_df, checks) dq_engine.save_results_in_table( output_df=valid_and_invalid_df, output_config=OutputConfig(location="catalog.schema.output"), ) # Option 2: apply quality checks on the DataFrame and provide valid and invalid (quarantined) DataFrames - valid_df, invalid_df, metrics = dq_engine.apply_checks_and_split(input_df, checks) + valid_df, invalid_df = dq_engine.apply_checks_and_split(input_df, checks) dq_engine.save_results_in_table( output_df=valid_df, quarantine_df=invalid_df, @@ -515,7 +515,7 @@ The following fields from the [configuration file](/docs/installation/#configura - `sample_seed`: seed for reproducible sampling. - `limit`: maximum number of records to analyze. - `extra_params`: (optional) extra parameters to pass to the jobs such as result column names and user_metadata -- `custom_metrics`: (optional) list of Spark SQL expressions for capturing custom summary metrics. If no summary metrics are provided, the default metrics (number of input, valid, warning, and error rows) will be tracked. +- `custom_metrics`: (optional) list of Spark SQL expressions for capturing custom summary metrics. By default, the number of input, warning, and error rows will be tracked. When custom metrics are defined, they will be tracked in addition to the default metrics. - `custom_check_functions`: (optional) custom check functions defined in Python files that can be used in the quality checks. - `reference_tables`: (optional) reference tables that can be used in the quality checks. @@ -568,7 +568,7 @@ Example of the configuration file (relevant fields only): ## Summary Metrics -DQX can automatically capture and store summary metrics about your data quality checking processes. When enabled, the system collects both default metrics (input count, error count, warning count, valid count) and any custom metrics you define. Metrics can be configured programmatically or via a configuration file when installing DQX using the Databricks CLI. +DQX can automatically capture and store summary metrics about your data quality checking processes. When enabled, the system collects both default metrics (input count, error count, warning count, valid count) and any custom metrics you define. Metrics can be configured programmatically or via a configuration file when installing DQX as a tool in the workspace. ### Enabling summary metrics programmatically @@ -601,10 +601,10 @@ To enable summary metrics programmatically, create and pass a `DQMetricsObserver ### Enabling summary metrics in DQX workflows -Summary metrics can also be enabled in DQX workflows installed using the Databricks CLI. Metrics are configured: +Summary metrics can also be enabled in DQX workflows. Metrics are configured: -1. **During Installation**: When prompted, choose to store summary metrics and configure the default metrics table location -2. **Configuration File**: 2. **Configuration File**: Add `custom_metrics` and `metrics_config` to the [configuration file](/docs/installation/#configuration-file) +1. **During installation for default run config**: When prompted, choose to store summary metrics and configure the default metrics table location +2. **Configuration file**: Add `custom_metrics` and `metrics_config` to the [configuration file](/docs/installation/#configuration-file) For detailed information about summary metrics, including examples and best practices, see the [Summary Metrics Guide](/docs/guide/summary_metrics). diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index fe17833c9..a1e64a681 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -20,10 +20,10 @@ DQX automatically captures the following built-in metrics for every data quality | Metric Name | Data Type | Description | |-------------|------------------------------------------------|-------------------------------------------------------| -| `input_count` | `int` | Total number of input records processed | -| `error_count` | `int` | Number of records that failed error-level checks | -| `warning_count` | `int` | Number of records that triggered warning-level checks | -| `valid_count` | `int` | Number of records that passed all checks | +| `input_row_count` | `int` | Total number of input records processed | +| `error_row_count` | `int` | Number of records that failed error-level checks | +| `warning_row_count` | `int` | Number of records that triggered warning-level checks | +| `valid_row_count` | `int` | Number of records that passed all checks | ### Custom Metrics @@ -33,8 +33,8 @@ Users can define custom metrics with Spark SQL expressions. These metrics will b Summary metrics are calculated on all records processed by DQX. Complex aggregations can degrade performance when processing large datasets. Be cautious with operations like `DISTINCT` on high-cardinality columns. +Example of custom data quality summary metrics: ```sql -/* EXAMPLE: Custom data quality summary metrics */ sum(array_size(_errors)) as total_errors avg(array_size(_errors)) as errors_avg count(case when array_size(_errors) > 1) as count_multiple_errors @@ -59,21 +59,22 @@ Metrics are not directly accessible from the returned Spark Observation when dat # Create observer observer = DQMetricsObserver(name="basic_metrics") - # Create the engine with the observer + # Create the engine with the optional observer engine = DQEngine(WorkspaceClient(), observer=observer) # Apply checks and get metrics checked_df, observation = engine.apply_checks_by_metadata(df, checks) # Trigger an action to populate metrics (e.g. count, save to a table) + # Metrics are only populated if an action is triggered on the DataFrame such as count or saving to a table. row_count = checked_df.count() # Access metrics metrics = observation.get - print(f"Input count: {metrics['input_count']}") - print(f"Error count: {metrics['error_count']}") - print(f"Warning count: {metrics['warning_count']}") - print(f"Valid count: {metrics['valid_count']}") + print(f"Input row count: {metrics['input_row_count']}") + print(f"Error row count: {metrics['error_row_count']}") + print(f"Warning row count: {metrics['warning_row_count']}") + print(f"Valid row count: {metrics['valid_row_count']}") ``` @@ -91,8 +92,8 @@ Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserve # Define custom metrics custom_metrics = [ - "sum(array_size(_errors)) as total_errors", - "sum(array_size(_warnings)) as total_warnings", + "sum(array_size(_errors)) as total_check_errors", + "sum(array_size(_warnings)) as total_check_warnings", ] # Create the observer with custom metrics @@ -101,17 +102,24 @@ Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserve custom_metrics=custom_metrics ) - # Create the engine with the observer + # Create the engine with the optional observer engine = DQEngine(WorkspaceClient(), observer=observer) - # End-to-end processing with metrics persistence - engine.apply_checks_by_metadata_and_save_in_table( - checks=my_checks, - input_config=InputConfig(location="main.raw.customer_data"), - output_config=OutputConfig(location="main.clean.customer_data"), - quarantine_config=OutputConfig(location="main.quarantine.customer_data"), - metrics_config=OutputConfig(location="main.analytics.dq_metrics") - ) + # Apply checks and get metrics + checked_df, observation = engine.apply_checks_by_metadata(df, checks) + + # Trigger an action to populate metrics (e.g. count, save to a table) + # Metrics are only populated if an action is triggered on the DataFrame such as count or saving to a table. + row_count = checked_df.count() + + # Access metrics + metrics = observation.get + print(f"Input row count: {metrics['input_row_count']}") + print(f"Error row count: {metrics['error_row_count']}") + print(f"Warning row count: {metrics['warning_row_count']}") + print(f"Valid row count: {metrics['valid_row_count']}") + print(f"Total check errors: {metrics['total_check_errors']}") + print(f"Total check warnings: {metrics['total_check_warnings']}") ``` @@ -200,7 +208,7 @@ custom_metrics: ## Metrics Table Schema -Metrics can be written automatically by specifying a metrics table. The metrics table will contain the following fields: +Metrics can be written automatically and centralized by specifying a metrics table. The metrics table will contain the following fields: | Column Name | Column Type | Description | |-------------|---------------------|-----------------------------------------------------------| @@ -215,14 +223,13 @@ Metrics can be written automatically by specifying a metrics table. The metrics ### Performance Considerations 1. **Batch Metrics Collection**: Collect metrics during regular data processing after data is written -2. **Partition Metrics Tables**: Consider partitioning metrics tables by date for better query performance -3. **Monitor Metrics Overhead**: Complex custom metrics may impact processing performance +2. **Monitor Metrics Overhead**: Complex custom metrics may impact processing performance ### Monitoring and Alerting 1. **Track Trends**: Monitor metrics over time to identify data quality degradation 2. **Set Thresholds**: Establish acceptable ranges for error rates and warning counts -3. **Alert on Anomalies**: Set up alerts when metrics deviate significantly from historical patterns +3. **Alert on Anomalies**: Set up alerts when metrics deviate significantly from historical patterns, e.g. by using Databricks SQL Alerts The example query below shows how you can analyze metrics persisted to a table. ```sql diff --git a/src/databricks/labs/dqx/base.py b/src/databricks/labs/dqx/base.py index c5d427f55..741df435c 100644 --- a/src/databricks/labs/dqx/base.py +++ b/src/databricks/labs/dqx/base.py @@ -2,7 +2,7 @@ from collections.abc import Callable from functools import cached_property from typing import final -from pyspark.sql import DataFrame, Observation, SparkSession +from pyspark.sql import DataFrame, Observation from databricks.labs.dqx.checks_validator import ChecksValidationStatus from databricks.labs.dqx.rule import DQRule from databricks.sdk import WorkspaceClient @@ -52,8 +52,8 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. """ @abc.abstractmethod @@ -70,8 +70,8 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. """ @abc.abstractmethod @@ -96,8 +96,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. """ @abc.abstractmethod @@ -123,8 +123,8 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. """ @staticmethod diff --git a/src/databricks/labs/dqx/config.py b/src/databricks/labs/dqx/config.py index 929dad197..acd24e213 100644 --- a/src/databricks/labs/dqx/config.py +++ b/src/databricks/labs/dqx/config.py @@ -137,7 +137,13 @@ def get_run_config(self, run_config_name: str | None = "default") -> RunConfig: @dataclass class BaseChecksStorageConfig(abc.ABC): - """Marker base class for storage configuration.""" + """Marker base class for storage configuration. + + Args: + location: The file path or table name where checks are stored. + """ + + location: str @dataclass diff --git a/src/databricks/labs/dqx/contexts/workflow_context.py b/src/databricks/labs/dqx/contexts/workflow_context.py index 89b7815ed..e695e7cc9 100644 --- a/src/databricks/labs/dqx/contexts/workflow_context.py +++ b/src/databricks/labs/dqx/contexts/workflow_context.py @@ -93,10 +93,7 @@ def quality_checker(self) -> QualityCheckerRunner: # Create observer if metrics are configured observer = None if self.run_config.metrics_config: - observer = DQMetricsObserver( - custom_metrics=self.config.custom_metrics, - result_columns=self.config.extra_params.result_column_names if self.config.extra_params else None, - ) + observer = DQMetricsObserver(custom_metrics=self.config.custom_metrics) dq_engine = DQEngine( workspace_client=self.workspace_client, diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index b6e35931f..d13cd5ab5 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -82,7 +82,12 @@ def __init__( self.spark = SparkSession.builder.getOrCreate() if spark is None else spark self.run_time = datetime.fromisoformat(extra_params.run_time) self.engine_user_metadata = extra_params.user_metadata - self.observer = observer + if observer: + self.observer = observer + self.observer._set_column_names( + error_column_name=self._result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._result_column_names[ColumnArguments.WARNINGS], + ) @cached_property def result_column_names(self) -> dict[ColumnArguments, str]: @@ -393,9 +398,12 @@ def _observe_metrics(self, df: DataFrame) -> tuple[DataFrame, Observation | None return df, None observation = self.observer.observation - observer_id = self.observer.id + observer_id = self.observer.observation_id return ( - df.observe(observer_id, *[F.expr(metric_statement) for metric_statement in self.observer.metrics]), + df.observe( + observer_id, + *[F.expr(metric_statement) for metric_statement in self.observer.metrics], + ), observation, ) @@ -1006,17 +1014,14 @@ def _build_metrics_df( input_config: InputConfig, output_config: OutputConfig | None, quarantine_config: OutputConfig | None, - checks_config: FileChecksStorageConfig | TableChecksStorageConfig | None, + checks_config: BaseChecksStorageConfig | None, ) -> DataFrame: engine = self._engine if not isinstance(engine, DQEngineCore) or not engine.observer: raise ValueError("Property 'observer' must be provided to DQEngine to track summary metrics") - result_column_names = engine.result_column_names or { - ColumnArguments.ERRORS: DefaultColumnNames.ERRORS.value, - ColumnArguments.WARNINGS: DefaultColumnNames.WARNINGS.value, - } + result_column_names = engine.result_column_names observation = engine.observer.observation metrics = observation.get return self.spark.createDataFrame( @@ -1024,9 +1029,9 @@ def _build_metrics_df( [ engine.observer.name, input_config.location, - None if not output_config else output_config.location, - None if not quarantine_config else quarantine_config.location, - None if not checks_config else checks_config.location, + output_config.location if output_config else None, + quarantine_config.location if quarantine_config else None, + checks_config.location if checks_config else None, metric_key, metric_value, engine.run_time, diff --git a/src/databricks/labs/dqx/installer/config_provider.py b/src/databricks/labs/dqx/installer/config_provider.py index 239cadd6f..65858179f 100644 --- a/src/databricks/labs/dqx/installer/config_provider.py +++ b/src/databricks/labs/dqx/installer/config_provider.py @@ -37,15 +37,7 @@ def prompt_new_installation(self, install_folder: str | None = None) -> Workspac output_config = self._prompt_output_config(is_streaming) quarantine_config = self._prompt_quarantine_config(is_streaming) - store_summary_metrics = self._prompts.confirm( - "Do you want to store summary metrics from data quality checking?" - ) - metrics_config = None - custom_metrics = [] - - if store_summary_metrics: - metrics_config = self._prompt_metrics_config(is_streaming) - custom_metrics = self._prompt_custom_metrics() + metrics_config, custom_metrics = self._prompt_metrics() checks_location = self._prompts.question( "Provide location of the quality checks definitions, either:\n" @@ -340,7 +332,20 @@ def _prompt_quarantine_config(self, is_streaming: bool) -> OutputConfig | None: ) return None - def _prompt_metrics_config(self, is_streaming: bool) -> OutputConfig: + def _prompt_metrics(self) -> tuple[OutputConfig | None, list[str] | None]: + store_summary_metrics = self._prompts.confirm( + "Do you want to store summary metrics from data quality checking in a table?" + ) + metrics_config = None + custom_metrics = None + + if store_summary_metrics: + metrics_config = self._prompt_metrics_config() + custom_metrics = self._prompt_custom_metrics() + + return metrics_config, custom_metrics + + def _prompt_metrics_config(self) -> OutputConfig: """Prompt user for metrics configuration.""" metrics_table = self._prompts.question( "Provide table for storing summary metrics in the fully qualified format `catalog.schema.table` or `schema.table`", @@ -367,37 +372,28 @@ def _prompt_metrics_config(self, is_streaming: bool) -> OutputConfig: ) ) - metrics_trigger_options = {} - if is_streaming: - metrics_trigger_options = json.loads( - self._prompts.question( - "Provide additional options for writing the metrics data using streaming " - "(e.g. {\"availableNow\": true})", - default="{}", - valid_regex=r"^.*$", - ) - ) - return OutputConfig( location=metrics_table, mode=metrics_write_mode, format=metrics_format, options=metrics_write_options, - trigger=metrics_trigger_options, ) def _prompt_custom_metrics(self) -> list[str]: """Prompt user for custom metrics as Spark SQL expressions.""" custom_metrics_input = self._prompts.question( - "Provide custom metrics as Spark SQL expressions separated by semicolons " - "(e.g. \"count(case when age > 65 then 1 end) as senior_count; avg(salary) as avg_salary\"). " - "Leave blank if no custom metrics are needed.", - default="", + "Provide custom metrics as a list of Spark SQL expressions " + "(e.g. \"[\'count(case when age > 65 then 1 end) as senior_count\', \'avg(salary) as avg_salary\']\"). " + "Leave blank to track the default data quality metrics.", + default="[]", valid_regex=r"^.*$", ) if custom_metrics_input.strip(): - # Split by semicolon and clean up whitespace - custom_metrics = [metric.strip() for metric in custom_metrics_input.split(";") if metric.strip()] + custom_metrics = json.loads(custom_metrics_input) + if not isinstance(custom_metrics, list): + raise ValueError( + "Custom metrics must be provided as a list of Spark SQL expressions (e.g. ['count(case when age > 65 then 1 end) as senior_count']" + ) return custom_metrics return [] diff --git a/src/databricks/labs/dqx/metrics_listener.py b/src/databricks/labs/dqx/metrics_listener.py index d6cae1842..a272c5a64 100644 --- a/src/databricks/labs/dqx/metrics_listener.py +++ b/src/databricks/labs/dqx/metrics_listener.py @@ -1,18 +1,12 @@ import logging from collections.abc import Callable -from pyspark.sql.streaming.listener import ( - StreamingQueryListener, - QueryStartedEvent, - QueryProgressEvent, - QueryIdleEvent, - QueryTerminatedEvent, -) +from pyspark.sql.streaming import listener logger = logging.getLogger(__name__) -class StreamingMetricsListener(StreamingQueryListener): +class StreamingMetricsListener(listener.StreamingQueryListener): """ Implements a Spark `StreamingQueryListener` for writing data quality summary metrics to an output destination. See the [Spark documentation](https://spark.apache.org/docs/latest/api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryListener.html) @@ -26,7 +20,7 @@ class StreamingMetricsListener(StreamingQueryListener): def __init__(self, handler: Callable) -> None: self._handler = handler - def onQueryStarted(self, event: QueryStartedEvent) -> None: + def onQueryStarted(self, event: listener.QueryStartedEvent) -> None: """ Writes a message to the standard output logs when a streaming query starts. @@ -35,7 +29,7 @@ def onQueryStarted(self, event: QueryStartedEvent) -> None: """ logger.debug(f"Streaming query '{event.name}' started run ID '{event.runId}'") - def onQueryProgress(self, event: QueryProgressEvent) -> None: + def onQueryProgress(self, event: listener.QueryProgressEvent) -> None: """ Writes the custom metrics from the DQObserver to the output destination. @@ -44,7 +38,7 @@ def onQueryProgress(self, event: QueryProgressEvent) -> None: """ self._handler() - def onQueryIdle(self, event: QueryIdleEvent) -> None: + def onQueryIdle(self, event: listener.QueryIdleEvent) -> None: """ Writes a message to the standard output logs when a streaming query is idle. @@ -53,7 +47,7 @@ def onQueryIdle(self, event: QueryIdleEvent) -> None: """ logger.debug(f"Streaming query run '{event.runId}' was reported idle") - def onQueryTerminated(self, event: QueryTerminatedEvent) -> None: + def onQueryTerminated(self, event: listener.QueryTerminatedEvent) -> None: """ Writes a message to the standard output logs when a streaming query stops due to cancellation or failure. diff --git a/src/databricks/labs/dqx/metrics_observer.py b/src/databricks/labs/dqx/metrics_observer.py index 91454e0af..9682653be 100644 --- a/src/databricks/labs/dqx/metrics_observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -1,8 +1,8 @@ from dataclasses import dataclass, field +from functools import cached_property from uuid import uuid4 from pyspark.sql import Observation -from databricks.labs.dqx.rule import ColumnArguments, DefaultColumnNames @dataclass @@ -13,24 +13,19 @@ class DQMetricsObserver: Args: name: Name of the observations which will be displayed in listener metrics (default is 'dqx'). custom_metrics: Optional list of SQL expressions defining custom, dataset-level quality metrics - result_columns: Optional dictionary of result column names (e.g. 'errors_column' or 'warnings_column') for - tracking summary metrics (defaults are '_errors' and '_warnings') """ name: str = "dqx" custom_metrics: list[str] | None = None - result_columns: dict[str, str] | None = field(default_factory=dict) - metrics: list[str] = field(default_factory=list) _id: str = field(default_factory=str) + _error_column_name: str = "_errors" + _warning_column_name: str = "_warnings" def __post_init__(self) -> None: self._id = str(uuid4()) - self.metrics.extend(self.default_metrics) - if self.custom_metrics: - self.metrics.extend(self.custom_metrics) - @property - def id(self) -> str: + @cached_property + def observation_id(self) -> str: """ Observer ID. @@ -39,23 +34,23 @@ def id(self) -> str: """ return self._id - @property - def default_metrics(self) -> list[str]: + @cached_property + def metrics(self) -> list[str]: """ - Default metrics tracked by the DQObservation. + Gets the observer metrics as Spark SQL expressions. Returns: - A list of Spark SQL expressions as strings + A list of Spark SQL expressions defining the observer metrics (both default and custom). """ - result_columns = self.result_columns or {} - errors_column = result_columns.get(ColumnArguments.ERRORS.value, DefaultColumnNames.ERRORS.value) - warnings_column = result_columns.get(ColumnArguments.WARNINGS.value, DefaultColumnNames.WARNINGS.value) - return [ + default_metrics = [ "count(1) as input_count", - f"count(case when {errors_column} is not null then 1 end) as error_count", - f"count(case when {warnings_column} is not null then 1 end) as warning_count", - f"count(case when {errors_column} is null and {warnings_column} is null then 1 end) as valid_count", + f"count(case when {self._error_column_name} is not null then 1 end) as error_count", + f"count(case when {self._warning_column_name} is not null then 1 end) as warning_count", + f"count(case when {self._error_column_name} is null and {self._warning_column_name} is null then 1 end) as valid_count", ] + if self.custom_metrics: + default_metrics.extend(self.custom_metrics) + return default_metrics @property def observation(self) -> Observation: @@ -69,3 +64,14 @@ def observation(self) -> Observation: A Spark `Observation` instance """ return Observation(name=self.name) + + def _set_column_names(self, error_column_name: str, warning_column_name: str) -> None: + """ + Sets the default column names (e.g. `_errors` and `_warnings`) for monitoring summary metrics. + + Args: + error_column_name: Error column name + warning_column_name: Warning column name + """ + self._error_column_name = error_column_name + self._warning_column_name = warning_column_name diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index 92ed893b6..784afbae4 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -1,6 +1,6 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metrics): """Test that quality checker workflow saves metrics when configured.""" - ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=["avg(id) as avg_id"]) + ctx, run_config = setup_workflows_with_metrics(metrics=True) ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) output_df = spark.table(run_config.output_config.location) @@ -11,10 +11,11 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri "error_count": 0, "warning_count": 0, "valid_count": output_count, - "avg_id": 0, } metrics_rows = spark.table(run_config.metrics_config.location).collect() + assert len(metrics_rows) == 1 + actual_metrics = metrics_rows[0].asDict() assert actual_metrics == expected_metrics @@ -40,6 +41,8 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf } metrics_rows = spark.table(run_config.metrics_config.location).collect() + assert len(metrics_rows) == 1 + actual_metrics = metrics_rows[0].asDict() assert actual_metrics == expected_metrics @@ -64,12 +67,14 @@ def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): } metrics_rows = spark.table(run_config.metrics_config.location).collect() + assert len(metrics_rows) == 1 + actual_metrics = metrics_rows[0].asDict() assert actual_metrics == expected_metrics -def test_multiple_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): - """Test workflow with multiple custom metrics.""" +def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): + """Test workflow with custom metrics.""" custom_metrics = [ "avg(id) as average_id", "sum(id) as total_id", @@ -92,8 +97,9 @@ def test_multiple_custom_metrics_in_workflow(spark, setup_workflows_with_metrics ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) metrics_df = spark.table(run_config.metrics_config.location) metrics_rows = metrics_df.collect() - actual_metrics = metrics_rows[0].asDict() + assert len(metrics_rows) == 1 + actual_metrics = metrics_rows[0].asDict() assert actual_metrics == expected_metrics diff --git a/tests/integration/test_save_and_load_checks_from_table.py b/tests/integration/test_save_and_load_checks_from_table.py index 391c8c3ed..facfbc84a 100644 --- a/tests/integration/test_save_and_load_checks_from_table.py +++ b/tests/integration/test_save_and_load_checks_from_table.py @@ -430,6 +430,8 @@ def test_save_load_checks_from_table_in_user_installation(ws, installation_ctx, class ChecksDummyStorageConfig(BaseChecksStorageConfig): """Dummy storage config for testing unsupported storage type.""" + location: str = "test_location" + def test_load_checks_invalid_storage_config(ws, spark): engine = DQEngine(ws, spark) diff --git a/tests/unit/test_observer.py b/tests/unit/test_observer.py index cc784066b..d01b9dc4c 100644 --- a/tests/unit/test_observer.py +++ b/tests/unit/test_observer.py @@ -10,7 +10,6 @@ def test_dq_observer_default_initialization(): observer = DQMetricsObserver() assert observer.name == "dqx" assert observer.custom_metrics is None - assert observer.result_columns == {} expected_default_metrics = [ "count(1) as input_count", @@ -40,37 +39,6 @@ def test_dq_observer_with_custom_metrics(): assert observer.metrics == expected_metrics -def test_dq_observer_with_custom_result_columns(): - """Test DQObserver with custom result column names.""" - custom_columns = {"errors_column": "custom_errors", "warnings_column": "custom_warnings"} - observer = DQMetricsObserver(result_columns=custom_columns) - - expected_default_metrics = [ - "count(1) as input_count", - "count(case when custom_errors is not null then 1 end) as error_count", - "count(case when custom_warnings is not null then 1 end) as warning_count", - "count(case when custom_errors is null and custom_warnings is null then 1 end) as valid_count", - ] - assert observer.metrics == expected_default_metrics - - -def test_dq_observer_with_custom_metrics_and_columns(): - """Test DQObserver with both custom metrics and custom result columns.""" - custom_metrics = ["max(salary) as max_salary"] - custom_columns = {"errors_column": "issues", "warnings_column": "alerts"} - - observer = DQMetricsObserver(custom_metrics=custom_metrics, result_columns=custom_columns) - - expected_metrics = [ - "count(1) as input_count", - "count(case when issues is not null then 1 end) as error_count", - "count(case when alerts is not null then 1 end) as warning_count", - "count(case when issues is null and alerts is null then 1 end) as valid_count", - "max(salary) as max_salary", - ] - assert observer.metrics == expected_metrics - - def test_dq_observer_empty_custom_metrics(): """Test DQObserver with empty custom metrics list.""" observer = DQMetricsObserver(custom_metrics=[]) From d367ef339f6fa3f7ea108931545d09b6e0951d49 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Sun, 21 Sep 2025 15:36:31 -0400 Subject: [PATCH 14/27] Update docs and tests --- docs/dqx/docs/guide/quality_checks_apply.mdx | 5 +- src/databricks/labs/dqx/metrics_observer.py | 8 +-- tests/integration/test_metrics_workflow.py | 32 ++++----- tests/integration/test_summary_metrics.py | 69 ++++++++++++++------ 4 files changed, 70 insertions(+), 44 deletions(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index dfce25a04..dee3be810 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -165,14 +165,14 @@ In the example below, checks are defined in YAML syntax for convenience and then input_df = spark.read.table("catalog.schema.input") # Option 1: apply quality checks on the DataFrame and output results as a single DataFrame - valid_and_invalid_df, metrics = dq_engine.apply_checks_by_metadata(input_df, checks) + valid_and_invalid_df = dq_engine.apply_checks_by_metadata(input_df, checks) dq_engine.save_results_in_table( output_df=valid_and_invalid_df, output_config=OutputConfig(location="catalog.schema.output"), ) # Option 2: apply quality checks on the DataFrame and provide valid and invalid (quarantined) DataFrames - valid_df, invalid_df, metrics = dq_engine.apply_checks_by_metadata_and_split(input_df, checks) + valid_df, invalid_df = dq_engine.apply_checks_by_metadata_and_split(input_df, checks) dq_engine.save_results_in_table( output_df=valid_df, quarantine_df=invalid_df, @@ -204,7 +204,6 @@ In the example below, checks are defined in YAML syntax for convenience and then metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), ) ``` - ``` diff --git a/src/databricks/labs/dqx/metrics_observer.py b/src/databricks/labs/dqx/metrics_observer.py index 9682653be..b2a4d5050 100644 --- a/src/databricks/labs/dqx/metrics_observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -43,10 +43,10 @@ def metrics(self) -> list[str]: A list of Spark SQL expressions defining the observer metrics (both default and custom). """ default_metrics = [ - "count(1) as input_count", - f"count(case when {self._error_column_name} is not null then 1 end) as error_count", - f"count(case when {self._warning_column_name} is not null then 1 end) as warning_count", - f"count(case when {self._error_column_name} is null and {self._warning_column_name} is null then 1 end) as valid_count", + "count(1) as input_row_count", + f"count(case when {self._error_column_name} is not null then 1 end) as error_row_count", + f"count(case when {self._warning_column_name} is not null then 1 end) as warning_row_count", + f"count(case when {self._error_column_name} is null and {self._warning_column_name} is null then 1 end) as valid_row_count", ] if self.custom_metrics: default_metrics.extend(self.custom_metrics) diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index 784afbae4..23fea515b 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -7,10 +7,10 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri output_count = output_df.count() expected_metrics = { - "input_count": output_count, - "error_count": 0, - "warning_count": 0, - "valid_count": output_count, + "input_row_count": output_count, + "error_row_count": 0, + "warning_row_count": 0, + "valid_row_count": output_count, } metrics_rows = spark.table(run_config.metrics_config.location).collect() @@ -33,10 +33,10 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf quarantine_count = quarantine_df.count() expected_metrics = { - "input_count": output_count + quarantine_count, - "error_count": quarantine_count, - "warning_count": 0, - "valid_count": output_count, + "input_row_count": output_count + quarantine_count, + "error_row_count": quarantine_count, + "warning_row_count": 0, + "valid_row_count": output_count, "unique_names": 0, } @@ -58,10 +58,10 @@ def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): output_count = output_df.count() expected_metrics = { - "input_count": output_count, - "error_count": 0, - "warning_count": 0, - "valid_count": output_count, + "input_row_count": output_count, + "error_row_count": 0, + "warning_row_count": 0, + "valid_row_count": output_count, "max_id": 1, "min_id": 0, } @@ -85,10 +85,10 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=custom_metrics) expected_metrics = { - "input_count": 0, - "error_count": 0, - "warning_count": 0, - "valid_count": 0, + "input_row_count": 0, + "error_row_count": 0, + "warning_row_count": 0, + "valid_row_count": 0, "average_id": 0, "total_id": 0, "unique_names": 0, diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index d53a1b986..63b23629c 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -50,7 +50,34 @@ def test_engine_with_observer_before_action(ws, spark): assert actual_metrics == {} -def test_engine_with_observer_after_action(ws, spark): +def test_engine_with_observer(ws, spark): + """Test that summary metrics can be accessed after running a Spark action like df.count().""" + observer = DQMetricsObserver(name="test_observer") + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], + [4, None, 28, 55000], + ], + TEST_SCHEMA, + ) + checked_df, observation = dq_engine.apply_checks_by_metadata(test_df, TEST_CHECKS) + checked_df.count() # Trigger an action to get the metrics + + expected_metrics = { + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, + } + actual_metrics = observation.get + assert actual_metrics == expected_metrics + + +def test_engine_with_observer_custom_metrics(ws, spark): """Test that summary metrics can be accessed after running a Spark action like df.count().""" custom_metrics = [ "avg(case when _errors is not null then age else null end) as avg_error_age", @@ -72,10 +99,10 @@ def test_engine_with_observer_after_action(ws, spark): checked_df.count() # Trigger an action to get the metrics expected_metrics = { - "input_count": 4, - "error_count": 1, - "warning_count": 1, - "valid_count": 2, + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, "avg_error_age": 35.0, "total_warning_salary": 55000, } @@ -118,10 +145,10 @@ def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): ) expected_metrics = { - "input_count": 4, - "error_count": 1, - "warning_count": 1, - "valid_count": 2, + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, "avg_error_age": 35.0, "total_warning_salary": 55000, } @@ -171,10 +198,10 @@ def test_engine_metrics_with_quarantine_and_metrics(ws, spark, make_schema, make ) expected_metrics = { - "input_count": 4, - "error_count": 1, - "warning_count": 1, - "valid_count": 2, + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, "avg_error_age": 35.0, "total_warning_salary": 55000, } @@ -253,10 +280,10 @@ def test_engine_streaming_metrics_saved_to_table(ws, spark, make_schema, make_ra ) expected_metrics = { - "input_count": 4, - "error_count": 1, - "warning_count": 1, - "valid_count": 2, + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, "avg_error_age": 35.0, "total_warning_salary": 55000, } @@ -314,10 +341,10 @@ def test_engine_streaming_with_quarantine_and_metrics(ws, spark, make_schema, ma ) expected_metrics = { - "input_count": 4, - "error_count": 1, - "warning_count": 1, - "valid_count": 2, + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, "avg_error_age": 35.0, "total_warning_salary": 55000, } From 461b90f69e9c047e9f8b684772dd7c926777800f Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Sun, 21 Sep 2025 15:49:32 -0400 Subject: [PATCH 15/27] Update tests --- tests/integration/test_summary_metrics.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 63b23629c..4253f50e2 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -30,7 +30,7 @@ ] -def test_engine_with_observer_before_action(ws, spark): +def test_observer_metrics_before_action(ws, spark): """Test that summary metrics are empty before running a Spark action.""" observer = DQMetricsObserver(name="test_observer") dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) @@ -50,7 +50,7 @@ def test_engine_with_observer_before_action(ws, spark): assert actual_metrics == {} -def test_engine_with_observer(ws, spark): +def test_observer_metrics(ws, spark): """Test that summary metrics can be accessed after running a Spark action like df.count().""" observer = DQMetricsObserver(name="test_observer") dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) @@ -77,7 +77,7 @@ def test_engine_with_observer(ws, spark): assert actual_metrics == expected_metrics -def test_engine_with_observer_custom_metrics(ws, spark): +def test_observer_custom_metrics(ws, spark): """Test that summary metrics can be accessed after running a Spark action like df.count().""" custom_metrics = [ "avg(case when _errors is not null then age else null end) as avg_error_age", @@ -110,7 +110,7 @@ def test_engine_with_observer_custom_metrics(ws, spark): assert actual_metrics == expected_metrics -def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): +def test_observer_metrics_output(ws, spark, make_schema, make_random): """Test that summary metrics are written to the table defined in metrics_config.""" catalog_name = "main" schema = make_schema(catalog_name=catalog_name) @@ -157,7 +157,7 @@ def test_engine_metrics_saved_to_table(ws, spark, make_schema, make_random): assert actual_metrics == expected_metrics -def test_engine_metrics_with_quarantine_and_metrics(ws, spark, make_schema, make_random): +def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random): """Test that metrics work correctly when using both quarantine and metrics configs.""" catalog_name = "main" schema = make_schema(catalog_name=catalog_name) @@ -241,7 +241,7 @@ def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_r assert not ws.tables.exists(metrics_table_name).table_exists -def test_engine_streaming_metrics_saved_to_table(ws, spark, make_schema, make_random, make_volume): +def test_streaming_observer_metrics_output(ws, spark, make_schema, make_random, make_volume): """Test that summary metrics are written to the table when using streaming with metrics_config.""" catalog_name = "main" schema = make_schema(catalog_name=catalog_name) @@ -291,7 +291,7 @@ def test_engine_streaming_metrics_saved_to_table(ws, spark, make_schema, make_ra assert actual_metrics == expected_metrics -def test_engine_streaming_with_quarantine_and_metrics(ws, spark, make_schema, make_random, make_volume): +def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random, make_volume): """Test that streaming metrics work correctly when using both quarantine and metrics configs.""" catalog_name = "main" schema = make_schema(catalog_name=catalog_name) From f349b08b085b85219963acd4ca432bf580bce7dc Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Wed, 24 Sep 2025 18:22:46 -0400 Subject: [PATCH 16/27] Refactor --- docs/dqx/docs/guide/quality_checks_apply.mdx | 52 +++--- docs/dqx/docs/guide/summary_metrics.mdx | 182 ++++++++++++++----- src/databricks/labs/dqx/config.py | 8 +- src/databricks/labs/dqx/engine.py | 164 +++++++++++------ src/databricks/labs/dqx/metrics_listener.py | 52 +++++- src/databricks/labs/dqx/metrics_observer.py | 82 ++++++++- 6 files changed, 388 insertions(+), 152 deletions(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index dee3be810..58f5dc803 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -51,7 +51,6 @@ You can customize the reporting columns as described in the [additional configur from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule from databricks.labs.dqx.config import InputConfig, OutputConfig - from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.sdk import WorkspaceClient @@ -109,15 +108,6 @@ You can customize the reporting columns as described in the [additional configur input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.output"), ) - - # Option 5: End-to-End approach with summary data: apply quality checks to a table and save the results and summary metrics to output tables - dq_engine.apply_checks_and_save_in_table( - checks=checks, - input_config=InputConfig(location="catalog.schema.input"), - output_config=OutputConfig(location="catalog.schema.valid"), - quarantine_config=OutputConfig(location="catalog.schema.quarantine"), - metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), - ) ``` @@ -194,15 +184,6 @@ In the example below, checks are defined in YAML syntax for convenience and then input_config=InputConfig(location="catalog.schema.input"), output_config=OutputConfig(location="catalog.schema.output"), ) - - # Option 5: End-to-End approach with summary data: apply quality checks to a table and save the results and summary metrics to output tables - dq_engine.apply_checks_by_metadata_and_save_in_table( - checks=checks, - input_config=InputConfig(location="catalog.schema.input"), - output_config=OutputConfig(location="catalog.schema.valid"), - quarantine_config=OutputConfig(location="catalog.schema.quarantine"), - metrics_config=OutputConfig(location="catalog.schema.dq_metrics"), - ) ``` @@ -583,17 +564,30 @@ To enable summary metrics programmatically, create and pass a `DQMetricsObserver dq_observer = DQMetricsObserver() dq_engine = DQEngine(observer=dq_observer) - # set up a DQMetricsObserver with a custom name (this name will be appended to the result table) - dq_observer = DQMetricsObserver(name="my_dqx_run") - dq_engine = DQEngine(observer=dq_observer) + # Option 1: apply quality checks, provide a single result DataFrame, and return a metrics observation + valid_and_invalid_df, metrics_observation = dq_engine.apply_checks(input_df, checks) + print(metrics_observation.get) - # set up a DQMetricsObserver with custom metrics - custom_metrics = [ - "count(case when array_size(_errors) > 1 then 1 end) as rows_with_multiple_errors", - "count(case when array_size(_warnings) > 1 then 1 end) as rows_with_multiple_warnings" - ] - dq_observer = DQMetricsObserver(custom_metrics=custom_metrics) - dq_engine = DQEngine(observer=dq_observer) + # Option 2: apply quality checks on the DataFrame, provide valid and invalid (quarantined) DataFrames, and return a metrics observation + valid_df, invalid_df = dq_engine.apply_checks_and_split(input_df, checks) + print(metrics_observation.get) + + # Option 3 End-to-End approach: apply quality checks to a table, save results to valid and invalid (quarantined) tables, and save metrics to a metrics table + dq_engine.apply_checks_and_save_in_table( + checks=checks, + input_config=InputConfig(location="catalog.schema.input"), + output_config=OutputConfig(location="catalog.schema.valid"), + quarantine_config=OutputConfig(location="catalog.schema.quarantine"), + metrics_config=OutputConfig(location="catalog.schema.metrics"), + ) + + # Option 4 End-to-End approach: apply quality checks to a table, save results to an output table, and save metrics to a metrics table + dq_engine.apply_checks_and_save_in_table( + checks=checks, + input_config=InputConfig(location="catalog.schema.input"), + output_config=OutputConfig(location="catalog.schema.output"), + metrics_config=OutputConfig(location="catalog.schema.metrics"), + ) ``` diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index a1e64a681..2b44aa4eb 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -18,12 +18,12 @@ Summary metrics in DQX capture both **built-in metrics** (automatically calculat DQX automatically captures the following built-in metrics for every data quality check execution: -| Metric Name | Data Type | Description | -|-------------|------------------------------------------------|-------------------------------------------------------| -| `input_row_count` | `int` | Total number of input records processed | -| `error_row_count` | `int` | Number of records that failed error-level checks | -| `warning_row_count` | `int` | Number of records that triggered warning-level checks | -| `valid_row_count` | `int` | Number of records that passed all checks | +| Metric Name | Data Type | Description | +|--------------------------|-------------------------------------------------|----------------------------------------------------| +| `input_row_count` | `int` | Total number of input rows processed | +| `error_row_count` | `int` | Number of rows that failed error-level checks | +| `warning_row_count` | `int` | Number of rows that triggered warning-level checks | +| `valid_row_count` | `int` | Number of rows that passed all checks | ### Custom Metrics @@ -79,6 +79,129 @@ Metrics are not directly accessible from the returned Spark Observation when dat +### Writing Metrics to a Table in Batch + +Summary metrics can be written to a table when calling `DQEngine` methods to apply checks and write output data. When the input data is read as a batch source, metrics will be collected and written in batch. + + + + ```python + from databricks.labs.dqx import check_funcs + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.metrics_observer import DQMetricsObserver + from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule + from databricks.labs.dqx.config import InputConfig, OutputConfig + from databricks.sdk import WorkspaceClient + + # Define the checks + checks = [ + DQRowRule( + criticality="warn", + check_func=check_funcs.is_not_null, + column="col3", + ), + DQDatasetRule( + criticality="error", + check_func=check_funcs.is_unique, + columns=["col1", "col2"], + ), + DQRowRule( + name="email_invalid_format", + criticality="error", + check_func=check_funcs.regex_match, + column="email", + check_func_kwargs={"regex": "^[a-zA-Z0-9._%+-]+@[a-zA-Z0-9.-]+\.[a-zA-Z]{2,}$"}, + ), + ] + + # Create the observer + observer = DQMetricsObserver(name="basic_metrics") + + # Create the engine with the metrics observer + engine = DQEngine(WorkspaceClient(), observer=observer) + + # Create the input config for a batch data source + input_config = InputConfig("main.demo.input_data") + + # Create the output, quarantine, and metrics configs + output_config = OutputConfig("main.demo.valid_data") + quarantine_config = OutputConfig("main.demo.quarantine_data") + metrics_config = OutputConfig("main.demo.metrics_data") + + # Read the data, apply the checks, write data to valid and quarantine tables, and write metrics to the metrics table + engine.apply_checks_and_save_in_table( + checks, + input_config, + output_config, + quarantine_config, + metrics_config + ) + ``` + + + +### Writing Metrics to a Table with Streaming + +Summary metrics can also be written in streaming. When the input data is read as a streaming source, metrics will be written for each streaming micro-batch: + + + + ```python + from databricks.labs.dqx import check_funcs + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.metrics_observer import DQMetricsObserver + from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule + from databricks.labs.dqx.config import InputConfig, OutputConfig + from databricks.sdk import WorkspaceClient + + # Define the checks + checks = [ + DQRowRule( + criticality="warn", + check_func=check_funcs.is_not_null, + column="col3", + ), + DQDatasetRule( + criticality="error", + check_func=check_funcs.is_unique, + columns=["col1", "col2"], + ), + DQRowRule( + name="email_invalid_format", + criticality="error", + check_func=check_funcs.regex_match, + column="email", + check_func_kwargs={"regex": "^[a-zA-Z0-9._%+-]+@[a-zA-Z0-9.-]+\.[a-zA-Z]{2,}$"}, + ), + ] + + # Create the observer + observer = DQMetricsObserver(name="basic_metrics") + + # Create the engine with the metrics observer + engine = DQEngine(WorkspaceClient(), observer=observer) + + # Create the input config for a streaming data source + input_config = InputConfig("main.demo.input_data", is_streaming=True) + + # Create the output, quarantine, and metrics configs + output_config = OutputConfig("main.demo.valid_data") + quarantine_config = OutputConfig("main.demo.quarantine_data") + metrics_config = OutputConfig("main.demo.metrics_data") + + # Read the data, apply the checks, write data to valid and quarantine tables, and write metrics to the metrics table + # Output and quarantine data will be written in streaming and summary metrics will be written for each micro-batch + engine.apply_checks_and_save_in_table( + checks, + input_config, + output_config, + quarantine_config, + metrics_config + ) + ``` + + + ### Advanced Usage with Custom Metrics Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserver`. Custom metrics should be defined as Spark SQL expressions with column aliases and will be accessible by their alias. @@ -124,39 +247,6 @@ Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserve -### Custom Result Column Names - -When using custom column names for errors and warnings, pass the column mapping when creating the `DQMetricsObserver`. - - - - ```python - from databricks.labs.dqx.config import ExtraParams - - # Configure custom column names - extra_params = ExtraParams( - result_column_names={ - "errors_column": "data_issues", - "warnings_column": "data_alerts" - } - ) - - # Create observer with custom metrics and column names - observer = DQMetricsObserver( - name="custom_columns", - result_columns=extra_params.result_column_names, - custom_metrics=["sum(array_size(data_issues)) as total_errors"] - ) - - engine = DQEngine( - workspace_client=WorkspaceClient(), - extra_params=extra_params, - observer=observer - ) - ``` - - - ## Workflow Integration ### No-Code Approach (Workflows) @@ -175,7 +265,7 @@ databricks labs dqx apply-checks --run-config "production" databricks labs dqx e2e --run-config "production" ``` -### Configuration File Example +#### Configuration File Example Metrics will be defined in the `metrics_config` section of your configuration file. @@ -210,12 +300,12 @@ custom_metrics: Metrics can be written automatically and centralized by specifying a metrics table. The metrics table will contain the following fields: -| Column Name | Column Type | Description | -|-------------|---------------------|-----------------------------------------------------------| -| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | -| `input_table` | `STRING` | Location of the input dataset | -| `metric_key` | `STRING` | Name of the metric | -| `metric_value` | `STRING` | Value of the metric (as a string) | +| Column Name | Column Type | Description | +|-----------------|-----------------------|-----------------------------------------------------------| +| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | +| `input_table` | `STRING` | Location of the input dataset | +| `metric_key` | `STRING` | Name of the metric | +| `metric_value` | `STRING` | Value of the metric (as a string) | | `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | ## Best Practices diff --git a/src/databricks/labs/dqx/config.py b/src/databricks/labs/dqx/config.py index acd24e213..929dad197 100644 --- a/src/databricks/labs/dqx/config.py +++ b/src/databricks/labs/dqx/config.py @@ -137,13 +137,7 @@ def get_run_config(self, run_config_name: str | None = "default") -> RunConfig: @dataclass class BaseChecksStorageConfig(abc.ABC): - """Marker base class for storage configuration. - - Args: - location: The file path or table name where checks are stored. - """ - - location: str + """Marker base class for storage configuration.""" @dataclass diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index d13cd5ab5..fabde57ab 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -36,17 +36,12 @@ from databricks.labs.dqx.checks_validator import ChecksValidator, ChecksValidationStatus from databricks.labs.dqx.schema import dq_result_schema from databricks.labs.dqx.utils import read_input_data, save_dataframe_as_table -from databricks.labs.dqx.metrics_observer import DQMetricsObserver +from databricks.labs.dqx.metrics_observer import DQMetricsObservation, DQMetricsObserver from databricks.labs.dqx.metrics_listener import StreamingMetricsListener from databricks.labs.dqx.telemetry import telemetry_logger, log_telemetry from databricks.sdk import WorkspaceClient logger = logging.getLogger(__name__) -OBSERVATION_TABLE_SCHEMA = ( - "run_name string, input_location string, output_location string, quarantine_location string, " - "checks_location string, metric_name string, metric_value string, run_ts timestamp, error_column_name string, " - "warning_column_name string, user_metadata map" -) class DQEngineCore(DQEngineCoreBase): @@ -84,7 +79,7 @@ def __init__( self.engine_user_metadata = extra_params.user_metadata if observer: self.observer = observer - self.observer._set_column_names( + self.observer.set_column_names( error_column_name=self._result_column_names[ColumnArguments.ERRORS], warning_column_name=self._result_column_names[ColumnArguments.WARNINGS], ) @@ -554,23 +549,38 @@ def apply_checks_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) - listener = self._get_metrics_listener(input_config, output_config, quarantine_config, metrics_config) - if listener: + if metrics_config and isinstance(self._engine, DQEngineCore) and df.isStreaming: + listener = self._get_streaming_metrics_listener( + input_config=input_config, + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) self.spark.streams.addListener(listener) if quarantine_config: # Split data into good and bad records - good_df, bad_df, _ = self.apply_checks_and_split(df, checks, ref_dfs) + good_df, bad_df, observation = self.apply_checks_and_split(df, checks, ref_dfs) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df, _ = self.apply_checks(df, checks, ref_dfs) + checked_df, observation = self.apply_checks(df, checks, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if metrics_config and not listener: + if metrics_config and isinstance(self._engine, DQEngineCore) and not df.isStreaming: # Create DataFrame with observation metrics - keys as column names, values as data - metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) + metrics_observation = DQMetricsObservation( + observer_name=self._engine.observer.name, + observed_metrics=observation.get, + error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], + input_location=input_config.location if input_config else None, + output_location=output_config.location if output_config else None, + quarantine_location=quarantine_config.location if quarantine_config else None, + user_metadata=self._engine.engine_user_metadata, + ) + metrics_df = self._engine.observer.build_metrics_df(self.spark, metrics_observation) save_dataframe_as_table(metrics_df, metrics_config) @telemetry_logger("engine", "apply_checks_by_metadata_and_save_in_table") @@ -610,8 +620,13 @@ def apply_checks_by_metadata_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) - listener = self._get_metrics_listener(input_config, output_config, quarantine_config, metrics_config) - if listener: + if metrics_config and isinstance(self._engine, DQEngineCore) and self._engine.observer and df.isStreaming: + listener = self._get_streaming_metrics_listener( + input_config=input_config, + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) self.spark.streams.addListener(listener) if quarantine_config: @@ -626,9 +641,19 @@ def apply_checks_by_metadata_and_save_in_table( checked_df, observation = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if observation and metrics_config and not listener: + if metrics_config and not df.isStreaming and isinstance(self._engine, DQEngineCore): # Create DataFrame with observation metrics - keys as column names, values as data - metrics_df = self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None) + metrics_observation = DQMetricsObservation( + observer_name=self._engine.observer.name, + observed_metrics=observation.get, + error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], + input_location=input_config.location if input_config else None, + output_location=output_config.location if output_config else None, + quarantine_location=quarantine_config.location if quarantine_config else None, + user_metadata=self._engine.engine_user_metadata, + ) + metrics_df = self._engine.observer.build_metrics_df(self.spark, metrics_observation) save_dataframe_as_table(metrics_df, metrics_config) @staticmethod @@ -1009,53 +1034,76 @@ def load_run_config( run_config_name=run_config_name, assume_user=assume_user, product_name=product_name ) - def _build_metrics_df( + def _get_batch_metrics_df( self, + observation: Observation, input_config: InputConfig, output_config: OutputConfig | None, quarantine_config: OutputConfig | None, checks_config: BaseChecksStorageConfig | None, ) -> DataFrame: - engine = self._engine - - if not isinstance(engine, DQEngineCore) or not engine.observer: - raise ValueError("Property 'observer' must be provided to DQEngine to track summary metrics") - - result_column_names = engine.result_column_names - observation = engine.observer.observation - metrics = observation.get - return self.spark.createDataFrame( - [ - [ - engine.observer.name, - input_config.location, - output_config.location if output_config else None, - quarantine_config.location if quarantine_config else None, - checks_config.location if checks_config else None, - metric_key, - metric_value, - engine.run_time, - result_column_names.get(ColumnArguments.ERRORS), - result_column_names.get(ColumnArguments.WARNINGS), - engine.engine_user_metadata, - ] - for metric_key, metric_value in metrics.items() - ], - schema=OBSERVATION_TABLE_SCHEMA, + """ + Gets a Spark `DataFrame` with data quality summary metrics. + + Args: + observation: Spark `Observation` + input_config: Optional configuration (e.g., table/view or file location and read options) for reading data. + output_config: Optional configuration (e.g., table name, mode, and write options) for writing records. + quarantine_config: Optional configuration for writing invalid records. + checks_config: Optional configuration for loading quality checks. + """ + if not isinstance(self._engine, DQEngineCore): + raise ValueError(f"Metrics cannot be collected for engine with type '{self._engine.__class__.__name__}'") + + if not self._engine.observer: + raise ValueError("Metrics cannot be collected for engine with no observer") + + metrics_observation = DQMetricsObservation( + observer_name=self._engine.observer.name, + observed_metrics=observation.get, + error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], + input_location=input_config.location if input_config else None, + output_location=output_config.location if output_config else None, + quarantine_location=quarantine_config.location if quarantine_config else None, + checks_location=checks_config.location if checks_config else None, + user_metadata=self._engine.engine_user_metadata, ) + return self._engine.observer.build_metrics_df(self.spark, metrics_observation) - def _get_metrics_listener( + def _get_streaming_metrics_listener( self, - input_config: InputConfig, - output_config: OutputConfig | None, - quarantine_config: OutputConfig | None, - metrics_config: OutputConfig | None, - ) -> StreamingMetricsListener | None: - if input_config.is_streaming and metrics_config: - return StreamingMetricsListener( - save_dataframe_as_table( - self._build_metrics_df(input_config, output_config, quarantine_config, checks_config=None), - metrics_config, - ) - ) - return None + metrics_config: OutputConfig, + input_config: InputConfig | None = None, + output_config: OutputConfig | None = None, + quarantine_config: OutputConfig | None = None, + checks_config: BaseChecksStorageConfig | None = None, + ) -> StreamingMetricsListener: + """ + Gets a `StreamingMetricsListener` object for writing metrics to an output table. + + Args: + input_config: Optional configuration (e.g., table/view or file location and read options) for reading data. + output_config: Optional configuration (e.g., table name, mode, and write options) for writing records. + quarantine_config: Optional configuration for writing invalid records. + checks_config: Optional configuration for loading quality checks. + metrics_config: Optional configuration for writing summary metrics. + """ + + if not isinstance(self._engine, DQEngineCore): + raise ValueError(f"Metrics cannot be collected for engine with type '{self._engine.__class__.__name__}'") + + if not self._engine.observer: + raise ValueError("Metrics cannot be collected for engine with no observer") + + metrics_observation = DQMetricsObservation( + observer_name=self._engine.observer.name, + error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], + input_location=input_config.location if input_config else None, + output_location=output_config.location if output_config else None, + quarantine_location=quarantine_config.location if quarantine_config else None, + checks_location=checks_config.location if checks_config else None, + user_metadata=self._engine.engine_user_metadata, + ) + return StreamingMetricsListener(metrics_config, metrics_observation, self.spark) diff --git a/src/databricks/labs/dqx/metrics_listener.py b/src/databricks/labs/dqx/metrics_listener.py index a272c5a64..8794587ce 100644 --- a/src/databricks/labs/dqx/metrics_listener.py +++ b/src/databricks/labs/dqx/metrics_listener.py @@ -1,6 +1,10 @@ import logging -from collections.abc import Callable +from datetime import datetime +from pyspark.sql import SparkSession from pyspark.sql.streaming import listener +from databricks.labs.dqx.config import OutputConfig +from databricks.labs.dqx.metrics_observer import DQMetricsObservation, DQMetricsObserver +from databricks.labs.dqx.utils import save_dataframe_as_table logger = logging.getLogger(__name__) @@ -13,12 +17,21 @@ class StreamingMetricsListener(listener.StreamingQueryListener): for detailed information about `StreamingQueryListener`. Args: - handler: Python `Callable` which handles writing metrics to an output destination. Called for every - micro-batch processed by the streaming query. + metrics_config: Output configuration used for writing data quality summary metrics + metrics_observation: `DQMetricsObservation` with data quality summary information + spark: `SparkSession` for writing summary metrics """ - def __init__(self, handler: Callable) -> None: - self._handler = handler + metrics_config: OutputConfig + metrics_observation: DQMetricsObservation + spark: SparkSession + + def __init__( + self, metrics_config: OutputConfig, metrics_observation: DQMetricsObservation, spark: SparkSession + ) -> None: + self.metrics_config = metrics_config + self.metrics_observation = metrics_observation + self.spark = spark def onQueryStarted(self, event: listener.QueryStartedEvent) -> None: """ @@ -27,7 +40,7 @@ def onQueryStarted(self, event: listener.QueryStartedEvent) -> None: Args: event: A `QueryStartedEvent` with details about the streaming query """ - logger.debug(f"Streaming query '{event.name}' started run ID '{event.runId}'") + logger.debug(f"Streaming query '{event.name}' for summary metrics started run ID '{event.runId}'") def onQueryProgress(self, event: listener.QueryProgressEvent) -> None: """ @@ -36,7 +49,24 @@ def onQueryProgress(self, event: listener.QueryProgressEvent) -> None: Args: event: A `QueryProgressEvent` with details about the last processed micro-batch """ - self._handler() + observed_metrics = event.progress.observedMetrics.get(self.metrics_observation.observer_name) + if not observed_metrics: + return + + metrics_observation = DQMetricsObservation( + observer_name=self.metrics_observation.observer_name, + observed_metrics=observed_metrics.asDict(), + run_time=datetime.fromisoformat(event.progress.timestamp), + error_column_name=self.metrics_observation.error_column_name, + warning_column_name=self.metrics_observation.warning_column_name, + input_location=self.metrics_observation.input_location, + output_location=self.metrics_observation.output_location, + quarantine_location=self.metrics_observation.quarantine_location, + checks_location=self.metrics_observation.checks_location, + user_metadata=self.metrics_observation.user_metadata, + ) + metrics_df = DQMetricsObserver.build_metrics_df(self.spark, metrics_observation) + save_dataframe_as_table(metrics_df, self.metrics_config) def onQueryIdle(self, event: listener.QueryIdleEvent) -> None: """ @@ -45,7 +75,7 @@ def onQueryIdle(self, event: listener.QueryIdleEvent) -> None: Args: event: A `QueryIdleEvent` with details about the streaming query """ - logger.debug(f"Streaming query run '{event.runId}' was reported idle") + logger.debug(f"Streaming query run '{event.runId}' for summary metrics was reported idle") def onQueryTerminated(self, event: listener.QueryTerminatedEvent) -> None: """ @@ -55,6 +85,8 @@ def onQueryTerminated(self, event: listener.QueryTerminatedEvent) -> None: event: A `QueryTerminatedEvent` with details about the streaming query """ if event.exception: - logger.debug(f"Streaming query run '{event.runId}' failed with error: {event.exception}") + logger.debug( + f"Streaming query run '{event.runId}' for summary metrics failed with error: {event.exception}" + ) - logger.debug(f"Streaming query run '{event.runId}' stopped") + logger.debug(f"Streaming query run '{event.runId}' for summary metrics stopped") diff --git a/src/databricks/labs/dqx/metrics_observer.py b/src/databricks/labs/dqx/metrics_observer.py index b2a4d5050..4674ea55c 100644 --- a/src/databricks/labs/dqx/metrics_observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -1,8 +1,50 @@ from dataclasses import dataclass, field +from datetime import datetime from functools import cached_property +from typing import Any from uuid import uuid4 -from pyspark.sql import Observation +from pyspark.sql import DataFrame, Observation, SparkSession + + +OBSERVATION_TABLE_SCHEMA = ( + "run_name string, input_location string, output_location string, quarantine_location string, " + "checks_location string, metric_name string, metric_value string, run_ts timestamp, error_column_name string, " + "warning_column_name string, user_metadata map" +) + + +@dataclass(frozen=True) +class DQMetricsObservation: + """ + Observer metrics class used to persist summary metrics. + + Args: + observer_name: Name of the observations (default is 'dqx'). + observed_metrics: Dictionary of observed metrics + run_time: Run time when the data quality summary metrics were observed + error_column_name: Name of the error column when running quality checks + warning_column_name: Name of the warning column when running quality checks + input_location: Location where input data is loaded from when running quality checks (fully-qualified table + name or file path) used when running quality checks + output_location: Location where output data is persisted when running quality checks (fully-qualified table + name or file path) + quarantine_location: Location where quarantined data is persisted when running quality checks (fully-qualified + table name or file path) + checks_location: Location where checks are loaded from when running quality checks (fully-qualified table name + or file path) used + """ + + observer_name: str + error_column_name: str + warning_column_name: str + run_time: datetime | None = None + observed_metrics: dict[str, Any] | None = None + input_location: str | None = None + output_location: str | None = None + quarantine_location: str | None = None + checks_location: str | None = None + user_metadata: dict[str, str] | None = None @dataclass @@ -65,7 +107,7 @@ def observation(self) -> Observation: """ return Observation(name=self.name) - def _set_column_names(self, error_column_name: str, warning_column_name: str) -> None: + def set_column_names(self, error_column_name: str, warning_column_name: str) -> None: """ Sets the default column names (e.g. `_errors` and `_warnings`) for monitoring summary metrics. @@ -75,3 +117,39 @@ def _set_column_names(self, error_column_name: str, warning_column_name: str) -> """ self._error_column_name = error_column_name self._warning_column_name = warning_column_name + + @staticmethod + def build_metrics_df(spark: SparkSession, observation: DQMetricsObservation) -> DataFrame: + """ + Builds a Spark `DataFrame` from a `DQMetricsObservation`. + + Args: + spark: `SparkSession` used to create the `DataFrame` + observation: `DQMetricsObservation` with summary metrics + + Returns: + A Spark `DataFrame` with summary metrics + """ + + if not observation.observed_metrics: + return spark.createDataFrame([], schema=OBSERVATION_TABLE_SCHEMA) + + return spark.createDataFrame( + [ + [ + observation.observer_name, + observation.input_location, + observation.output_location, + observation.quarantine_location, + observation.checks_location, + metric_key, + metric_value, + observation.run_time, + observation.error_column_name, + observation.warning_column_name, + observation.user_metadata, + ] + for metric_key, metric_value in observation.observed_metrics.items() + ], + schema=OBSERVATION_TABLE_SCHEMA, + ) From 998490a123bb0a2aeafaece6ed84e4013b8f560d Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Wed, 1 Oct 2025 10:50:01 -0400 Subject: [PATCH 17/27] Update engine methods, tests, and docs --- docs/dqx/docs/guide/quality_checks_apply.mdx | 2 +- docs/dqx/docs/guide/summary_metrics.mdx | 88 +++++++++++++++--- docs/dqx/docs/reference/engine.mdx | 28 +++--- src/databricks/labs/dqx/config.py | 8 +- src/databricks/labs/dqx/engine.py | 89 +++++++++--------- tests/integration/test_metrics_workflow.py | 2 +- tests/integration/test_summary_metrics.py | 94 ++++++++++++++++++++ 7 files changed, 239 insertions(+), 72 deletions(-) diff --git a/docs/dqx/docs/guide/quality_checks_apply.mdx b/docs/dqx/docs/guide/quality_checks_apply.mdx index 58f5dc803..e8f0a61e5 100644 --- a/docs/dqx/docs/guide/quality_checks_apply.mdx +++ b/docs/dqx/docs/guide/quality_checks_apply.mdx @@ -569,7 +569,7 @@ To enable summary metrics programmatically, create and pass a `DQMetricsObserver print(metrics_observation.get) # Option 2: apply quality checks on the DataFrame, provide valid and invalid (quarantined) DataFrames, and return a metrics observation - valid_df, invalid_df = dq_engine.apply_checks_and_split(input_df, checks) + valid_df, invalid_df, metrics_observation = dq_engine.apply_checks_and_split(input_df, checks) print(metrics_observation.get) # Option 3 End-to-End approach: apply quality checks to a table, save results to valid and invalid (quarantined) tables, and save metrics to a metrics table diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index 2b44aa4eb..a1f1897a1 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -42,12 +42,12 @@ count(case when array_size(_errors) > 1) as count_multiple_errors ## Usage Examples -### Basic Usage with Built-in Metrics +### Accessing Metrics when Applying Checks -Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark Observation with 1 or more output DataFrames. Data quality metrics can be accessed after any action is performed on the output DataFrames. +Engine methods (e.g. `apply_checks`, `apply_checks_by_metadata`) return a Spark Observation with 1 or more output DataFrames. Data quality metrics can be accessed from the Spark Observation after any action is performed on the output DataFrames. -Metrics are not directly accessible from the returned Spark Observation when data is processed with streaming. Use DQX's built-in methods to persist streaming metrics to an output table or implement a custom [StreamingQueryListener](https://spark.apache.org/docs/latest/api/python/reference/pyspark.ss/api/pyspark.sql.streaming.StreamingQueryListener.html) to process the metrics. +Metrics are not directly accessible from the returned Spark Observation when data is processed with streaming. Use DQX's built-in methods to persist streaming metrics to an output table. See [Writing Metrics to a Table with Streaming](#writing-metrics-to-a-table-with-streaming) for more details. @@ -57,7 +57,7 @@ Metrics are not directly accessible from the returned Spark Observation when dat from databricks.labs.dqx.metrics_observer import DQMetricsObserver # Create observer - observer = DQMetricsObserver(name="basic_metrics") + observer = DQMetricsObserver(name="dq_metrics") # Create the engine with the optional observer engine = DQEngine(WorkspaceClient(), observer=observer) @@ -79,7 +79,11 @@ Metrics are not directly accessible from the returned Spark Observation when dat -### Writing Metrics to a Table in Batch +### Writing Metrics to a Table + +Engine methods (e.g. `apply_checks_and_save_in_table`, `apply_checks_by_metadata_and_save_in_table`, and `save_results_in_table`) can write summary metrics into a table. Metrics can be written to a table in batch or streaming. You can write metrics for different datasets or workloads into a common metrics table to track data quality over time. + +#### Writing Metrics to a Table in Batch Summary metrics can be written to a table when calling `DQEngine` methods to apply checks and write output data. When the input data is read as a batch source, metrics will be collected and written in batch. @@ -89,7 +93,7 @@ Summary metrics can be written to a table when calling `DQEngine` methods to app from databricks.labs.dqx import check_funcs from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.metrics_observer import DQMetricsObserver - from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule + from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule from databricks.labs.dqx.config import InputConfig, OutputConfig from databricks.sdk import WorkspaceClient @@ -115,7 +119,7 @@ Summary metrics can be written to a table when calling `DQEngine` methods to app ] # Create the observer - observer = DQMetricsObserver(name="basic_metrics") + observer = DQMetricsObserver(name="dq_metrics") # Create the engine with the metrics observer engine = DQEngine(WorkspaceClient(), observer=observer) @@ -140,7 +144,7 @@ Summary metrics can be written to a table when calling `DQEngine` methods to app -### Writing Metrics to a Table with Streaming +#### Writing Metrics to a Table with Streaming Summary metrics can also be written in streaming. When the input data is read as a streaming source, metrics will be written for each streaming micro-batch: @@ -150,7 +154,7 @@ Summary metrics can also be written in streaming. When the input data is read as from databricks.labs.dqx import check_funcs from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.metrics_observer import DQMetricsObserver - from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule, DQForEachColRule + from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule from databricks.labs.dqx.config import InputConfig, OutputConfig from databricks.sdk import WorkspaceClient @@ -176,7 +180,7 @@ Summary metrics can also be written in streaming. When the input data is read as ] # Create the observer - observer = DQMetricsObserver(name="basic_metrics") + observer = DQMetricsObserver(name="dq_metrics") # Create the engine with the metrics observer engine = DQEngine(WorkspaceClient(), observer=observer) @@ -202,7 +206,69 @@ Summary metrics can also be written in streaming. When the input data is read as -### Advanced Usage with Custom Metrics +#### Saving Metrics to a Table + +Summary metrics can be written to a table when calling `save_results_in_table`. After applying checks, pass the Spark Observation and output DataFrame(s) with the appropriate output configuration. + + + + ```python + from databricks.labs.dqx import check_funcs + from databricks.labs.dqx.engine import DQEngine + from databricks.labs.dqx.metrics_observer import DQMetricsObserver + from databricks.labs.dqx.rule import DQRowRule, DQDatasetRule + from databricks.labs.dqx.config import OutputConfig + from databricks.sdk import WorkspaceClient + + # Define the checks + checks = [ + DQRowRule( + criticality="warn", + check_func=check_funcs.is_not_null, + column="col3", + ), + DQDatasetRule( + criticality="error", + check_func=check_funcs.is_unique, + columns=["col1", "col2"], + ), + DQRowRule( + name="email_invalid_format", + criticality="error", + check_func=check_funcs.regex_match, + column="email", + check_func_kwargs={"regex": "^[a-zA-Z0-9._%+-]+@[a-zA-Z0-9.-]+\.[a-zA-Z]{2,}$"}, + ), + ] + + # Create the observer + observer = DQMetricsObserver(name="dq_metrics") + + # Create the engine with the metrics observer + engine = DQEngine(WorkspaceClient(), observer=observer) + + # Apply checks and get metrics + valid_df, quarantine_df, observation = engine.apply_checks_by_metadata_and_split(df, checks) + + # Create the output, quarantine, and metrics configs + output_config = OutputConfig("main.demo.valid_data") + quarantine_config = OutputConfig("main.demo.quarantine_data") + metrics_config = OutputConfig("main.demo.metrics_data") + + # Write the data to valid and quarantine tables, and write metrics to the metrics table + engine.apply_checks_and_save_in_table( + valid_df, + quarantine_df, + observation, + output_config, + quarantine_config, + metrics_config + ) + ``` + + + +### Configuring Custom Metrics Pass custom metrics as Spark SQL expressions when creating the `DQMetricsObserver`. Custom metrics should be defined as Spark SQL expressions with column aliases and will be accessible by their alias. diff --git a/docs/dqx/docs/reference/engine.mdx b/docs/dqx/docs/reference/engine.mdx index ae5b87ec0..c3440cefe 100644 --- a/docs/dqx/docs/reference/engine.mdx +++ b/docs/dqx/docs/reference/engine.mdx @@ -49,20 +49,20 @@ The following table outlines the available methods of the `DQEngine` and their f
**Available DQX engine methods** -| Method | Description | Arguments | Supports local execution | -| ---------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ----------------------------------------- | -| `apply_checks` | Applies quality checks to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_and_split` | Applies quality checks to the DataFrame; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_and_save_in_table` | Applies quality checks using DQRule objects, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | -| `apply_checks_by_metadata` | Applies quality checks defined as a dictionary to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_by_metadata_and_split` | Applies quality checks defined as a dictionary; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | -| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | -| `validate_checks` | Validates the provided quality checks to ensure they conform to the expected structure and types. | `checks`: List of checks to validate; `custom_check_functions`: (optional) dictionary of custom check functions that can be used; `validate_custom_check_functions`: (optional) if set to True, validates custom check functions (defaults to True). | Yes | -| `get_invalid` | Retrieves records from the DataFrame that violate data quality checks (records with warnings and errors). | `df`: Input DataFrame. | Yes | -| `get_valid` | Retrieves records from the DataFrame that pass all data quality checks. | `df`: Input DataFrame. | Yes | -| `load_checks` | Loads quality rules (checks) from storage backend. Multiple storage backends are supported including tables, files or workspace files, installation-managed sources where the location is inferred automatically from run config. | `config`: Configuration for loading checks from a storage backend, i.e. `FileChecksStorageConfig`: file in a local filesystem (YAML or JSON), or workspace files if invoked from Databricks notebook or job; `WorkspaceFileChecksStorageConfig`: file in a workspace (YAML or JSON) using absolute paths; `VolumeFileChecksStorageConfig`: file in a Unity Catalog Volume (YAML or JSON); `TableChecksStorageConfig`: a table; `InstallationChecksStorageConfig`: installation-managed storage backend, using the `checks_location` field from the run configuration. See more details below. | Yes (only with `FileChecksStorageConfig`) | -| `save_checks` | Saves quality rules (checks) to storage backend. Multiple storage backends are supported including tables, files or workspace files, installation-managed targets where the location is inferred automatically from run config. | `checks`: List of checks defined as dictionary; `config`: Configuration for saving checks in a storage backend, i.e. `FileChecksStorageConfig`: file in a local filesystem (YAML or JSON), or workspace files if invoked from Databricks notebook or job; `WorkspaceFileChecksStorageConfig`: file in a workspace (YAML or JSON); `VolumeFileChecksStorageConfig`: file in a Unity Catalog Volume (YAML or JSON); `TableChecksStorageConfig`: a table; `InstallationChecksStorageConfig`: storage defined in the installation context, using the `checks_location` field from the run configuration. See more details below. | Yes (only with `FileChecksStorageConfig`) | -| `save_results_in_table` | Save quality checking results in delta table(s). | `output_df`: (optional) Dataframe containing the output data; `quarantine_df`: (optional) Dataframe containing the output data; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `run_config_name`: Name of the run config to use; `install_folder`: (optional) installation folder where DQX is installed, only required when custom installation folder is used; `assume_user`: (optional) If True, assume user installation, otherwise global installation (skipped if `install_folder` is provided). | No | +| Method | Description | Arguments | Supports local execution | +| ---------------------------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | ----------------------------------------- | +| `apply_checks` | Applies quality checks to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_and_split` | Applies quality checks to the DataFrame; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_and_save_in_table` | Applies quality checks using DQRule objects, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined using DQX classes, each check is an instance of the DQRule class; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | +| `apply_checks_by_metadata` | Applies quality checks defined as a dictionary to the DataFrame; Returns a DataFrame with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_by_metadata_and_split` | Applies quality checks defined as a dictionary; Returns valid and invalid (quarantine) DataFrames with result columns and an optional Spark Observation with summary metrics. | `df`: DataFrame to check; `checks`: List of checks defined as dictionary; `custom_check_functions`: (optional) dictionary with custom check functions (e.g., globals() of the calling module); `ref_dfs`: Reference dataframes to use in the checks, if applicable. | Yes | +| `apply_checks_by_metadata_and_save_in_table` | Applies quality checks defined as a dictionary, writes results to valid and invalid Delta table(s) with result columns, and optionally writes summary metrics to a Delta table. | `input_config`: `InputConfig` object with the table name and options for reading the input data; `checks`: List of checks defined as dictionary; `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics; `custom_check_functions`: (optional) dictionary with custom check functions; `ref_dfs`: Reference dataframes to use in the checks, if applicable. | No | +| `validate_checks` | Validates the provided quality checks to ensure they conform to the expected structure and types. | `checks`: List of checks to validate; `custom_check_functions`: (optional) dictionary of custom check functions that can be used; `validate_custom_check_functions`: (optional) if set to True, validates custom check functions (defaults to True). | Yes | +| `get_invalid` | Retrieves records from the DataFrame that violate data quality checks (records with warnings and errors). | `df`: Input DataFrame. | Yes | +| `get_valid` | Retrieves records from the DataFrame that pass all data quality checks. | `df`: Input DataFrame. | Yes | +| `load_checks` | Loads quality rules (checks) from storage backend. Multiple storage backends are supported including tables, files or workspace files, installation-managed sources where the location is inferred automatically from run config. | `config`: Configuration for loading checks from a storage backend, i.e. `FileChecksStorageConfig`: file in a local filesystem (YAML or JSON), or workspace files if invoked from Databricks notebook or job; `WorkspaceFileChecksStorageConfig`: file in a workspace (YAML or JSON) using absolute paths; `VolumeFileChecksStorageConfig`: file in a Unity Catalog Volume (YAML or JSON); `TableChecksStorageConfig`: a table; `InstallationChecksStorageConfig`: installation-managed storage backend, using the `checks_location` field from the run configuration. See more details below. | Yes (only with `FileChecksStorageConfig`) | +| `save_checks` | Saves quality rules (checks) to storage backend. Multiple storage backends are supported including tables, files or workspace files, installation-managed targets where the location is inferred automatically from run config. | `checks`: List of checks defined as dictionary; `config`: Configuration for saving checks in a storage backend, i.e. `FileChecksStorageConfig`: file in a local filesystem (YAML or JSON), or workspace files if invoked from Databricks notebook or job; `WorkspaceFileChecksStorageConfig`: file in a workspace (YAML or JSON); `VolumeFileChecksStorageConfig`: file in a Unity Catalog Volume (YAML or JSON); `TableChecksStorageConfig`: a table; `InstallationChecksStorageConfig`: storage defined in the installation context, using the `checks_location` field from the run configuration. See more details below. | Yes (only with `FileChecksStorageConfig`) | +| `save_results_in_table` | Save quality checking results and summary metrics to Delta table(s). | `output_df`: (optional) Dataframe containing the output data; `quarantine_df`: (optional) Dataframe containing the output data; `observation`: (optional) Spark Observation tracking data quality summary metrics (output when applying checks using DQEngine); `output_config`: `OutputConfig` object with the table name, output mode, and options for the output data; `quarantine_config`: `OutputConfig` object with the table name, output mode, and options for the quarantine data - if provided, data will be split; `metrics_config`: `OutputConfig` object with the table name, output mode, and options for the summary metrics data; `run_config_name`: Name of the run config to use; `install_folder`: (optional) installation folder where DQX is installed, only required when custom installation folder is used; `assume_user`: (optional) If True, assume user installation, otherwise global installation (skipped if `install_folder` is provided). | No | The 'Supports local execution' in the above table indicates which methods can be used for local testing without a Databricks workspace (see the usage in [local testing section](/docs/reference/testing/#local-execution-and-testing-with-dqengine)). diff --git a/src/databricks/labs/dqx/config.py b/src/databricks/labs/dqx/config.py index 69d5fcd31..01525e3c9 100644 --- a/src/databricks/labs/dqx/config.py +++ b/src/databricks/labs/dqx/config.py @@ -138,7 +138,13 @@ def get_run_config(self, run_config_name: str | None = "default") -> RunConfig: @dataclass class BaseChecksStorageConfig(abc.ABC): - """Marker base class for storage configuration.""" + """Marker base class for storage configuration. + + Args: + location: The file path or table name where checks are stored. + """ + + location: str @dataclass diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index c8eaa969f..355b2de13 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -78,8 +78,8 @@ def __init__( self.spark = SparkSession.builder.getOrCreate() if spark is None else spark self.run_time = datetime.fromisoformat(extra_params.run_time) self.engine_user_metadata = extra_params.user_metadata - if observer: - self.observer = observer + self.observer = observer + if self.observer: self.observer.set_column_names( error_column_name=self._result_column_names[ColumnArguments.ERRORS], warning_column_name=self._result_column_names[ColumnArguments.WARNINGS], @@ -424,7 +424,7 @@ def __init__( self, workspace_client: WorkspaceClient, spark: SparkSession | None = None, - engine: DQEngineCoreBase | None = None, + engine: DQEngineCore | None = None, extra_params: ExtraParams | None = None, checks_handler_factory: BaseChecksStorageHandlerFactory | None = None, run_config_loader: RunConfigLoader | None = None, @@ -562,7 +562,7 @@ def apply_checks_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) - if metrics_config and isinstance(self._engine, DQEngineCore) and df.isStreaming: + if self._engine.observer and metrics_config and df.isStreaming: listener = self._get_streaming_metrics_listener( input_config=input_config, output_config=output_config, @@ -581,7 +581,7 @@ def apply_checks_and_save_in_table( checked_df, observation = self.apply_checks(df, checks, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if metrics_config and isinstance(self._engine, DQEngineCore) and not df.isStreaming: + if self._engine.observer and metrics_config and not df.isStreaming: # Create DataFrame with observation metrics - keys as column names, values as data metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, @@ -633,7 +633,7 @@ def apply_checks_by_metadata_and_save_in_table( # Read data from the specified table df = read_input_data(self.spark, input_config) - if metrics_config and isinstance(self._engine, DQEngineCore) and self._engine.observer and df.isStreaming: + if self._engine.observer and metrics_config and df.isStreaming: listener = self._get_streaming_metrics_listener( input_config=input_config, output_config=output_config, @@ -654,7 +654,7 @@ def apply_checks_by_metadata_and_save_in_table( checked_df, observation = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) save_dataframe_as_table(checked_df, output_config) - if metrics_config and not df.isStreaming and isinstance(self._engine, DQEngineCore): + if self._engine.observer and metrics_config and not df.isStreaming: # Create DataFrame with observation metrics - keys as column names, values as data metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, @@ -720,8 +720,10 @@ def save_results_in_table( self, output_df: DataFrame | None = None, quarantine_df: DataFrame | None = None, + observation: Observation | None = None, output_config: OutputConfig | None = None, quarantine_config: OutputConfig | None = None, + metrics_config: OutputConfig | None = None, run_config_name: str | None = "default", product_name: str = "dqx", assume_user: bool = True, @@ -737,8 +739,10 @@ def save_results_in_table( Args: output_df: DataFrame with valid rows to be saved (optional). quarantine_df: DataFrame with invalid rows to be saved (optional). + observation: Spark Observation with data quality summary metrics (optional). output_config: Configuration describing where/how to write the valid rows. If omitted, falls back to the run config. quarantine_config: Configuration describing where/how to write the invalid rows (optional). If omitted, falls back to the run config. + metrics_config: Configuration describing where/how to write the summary metrics (optional). If omitted, falls back to the run config. run_config_name: Name of the run configuration to load when a config parameter is omitted. product_name: Product/installation identifier used to resolve installation paths for config loading in install_folder is not provided ("dqx" as default). assume_user: Whether to assume a per-user installation when loading the run configuration (True as default, skipped if install_folder is provided). @@ -765,12 +769,46 @@ def save_results_in_table( ) quarantine_config = run_config.quarantine_config + if observation is not None and metrics_config is None: + run_config = self._run_config_loader.load_run_config( + run_config_name=run_config_name, + assume_user=assume_user, + product_name=product_name, + install_folder=install_folder, + ) + metrics_config = run_config.metrics_config + + is_streaming_output = output_df is not None and output_df.isStreaming + is_streaming_quarantine = quarantine_df is not None and quarantine_df.isStreaming + is_streaming = is_streaming_output or is_streaming_quarantine + + if observation is not None and metrics_config is not None and is_streaming: + listener = self._get_streaming_metrics_listener( + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) + self.spark.streams.addListener(listener) + if output_df is not None and output_config is not None: save_dataframe_as_table(output_df, output_config) if quarantine_df is not None and quarantine_config is not None: save_dataframe_as_table(quarantine_df, quarantine_config) + if self._engine.observer and observation is not None and metrics_config is not None: + metrics_observation = DQMetricsObservation( + observer_name=self._engine.observer.name, + observed_metrics=observation.get, + error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], + warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], + output_location=output_config.location if output_config else None, + quarantine_location=quarantine_config.location if quarantine_config else None, + user_metadata=self._engine.engine_user_metadata, + ) + metrics_df = self._engine.observer.build_metrics_df(self.spark, metrics_observation) + save_dataframe_as_table(metrics_df, metrics_config) + def load_checks(self, config: BaseChecksStorageConfig) -> list[dict]: """Load DQ rules (checks) from the storage backend described by *config*. @@ -1047,43 +1085,6 @@ def load_run_config( run_config_name=run_config_name, assume_user=assume_user, product_name=product_name ) - def _get_batch_metrics_df( - self, - observation: Observation, - input_config: InputConfig, - output_config: OutputConfig | None, - quarantine_config: OutputConfig | None, - checks_config: BaseChecksStorageConfig | None, - ) -> DataFrame: - """ - Gets a Spark `DataFrame` with data quality summary metrics. - - Args: - observation: Spark `Observation` - input_config: Optional configuration (e.g., table/view or file location and read options) for reading data. - output_config: Optional configuration (e.g., table name, mode, and write options) for writing records. - quarantine_config: Optional configuration for writing invalid records. - checks_config: Optional configuration for loading quality checks. - """ - if not isinstance(self._engine, DQEngineCore): - raise ValueError(f"Metrics cannot be collected for engine with type '{self._engine.__class__.__name__}'") - - if not self._engine.observer: - raise ValueError("Metrics cannot be collected for engine with no observer") - - metrics_observation = DQMetricsObservation( - observer_name=self._engine.observer.name, - observed_metrics=observation.get, - error_column_name=self._engine.result_column_names[ColumnArguments.ERRORS], - warning_column_name=self._engine.result_column_names[ColumnArguments.WARNINGS], - input_location=input_config.location if input_config else None, - output_location=output_config.location if output_config else None, - quarantine_location=quarantine_config.location if quarantine_config else None, - checks_location=checks_config.location if checks_config else None, - user_metadata=self._engine.engine_user_metadata, - ) - return self._engine.observer.build_metrics_df(self.spark, metrics_observation) - def _get_streaming_metrics_listener( self, metrics_config: OutputConfig, diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index 23fea515b..e9e8cf4b7 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -103,7 +103,7 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): assert actual_metrics == expected_metrics -def test_quality_checker_workflow_without_metrics_config(ws, setup_workflows_with_metrics): +def test_quality_checker_workflow_without_metrics_config(setup_workflows_with_metrics): """Test that workflow works normally when metrics config is not provided.""" _, run_config = setup_workflows_with_metrics(metrics=False) assert run_config.metrics_config is None diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 4253f50e2..36926618e 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -350,3 +350,97 @@ def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, mak } actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() assert actual_metrics == expected_metrics + + +def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_random): + """Test save_results_in_table method with metrics configured for batch processing.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + quarantine_table_name = f"{catalog_name}.{schema.name}.quarantine_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + + observer = DQMetricsObserver(name="test_save_batch_observer") + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], # This will trigger an error + [4, None, 28, 55000], # This will trigger a warning + ], + TEST_SCHEMA, + ) + output_df, quarantine_df, observation = dq_engine.apply_checks_by_metadata_and_split(test_df, TEST_CHECKS) + + output_config = OutputConfig(location=output_table_name) + quarantine_config = OutputConfig(location=quarantine_table_name) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.save_results_in_table( + output_df=output_df, + quarantine_df=quarantine_df, + observation=observation, + output_config=output_config, + quarantine_config=quarantine_config, + metrics_config=metrics_config, + ) + + expected_metrics = { + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, + } + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + assert actual_metrics == expected_metrics + + +def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, make_random, make_volume): + """Test save_results_in_table method with metrics configured for streaming processing.""" + catalog_name = "main" + schema = make_schema(catalog_name=catalog_name) + input_table_name = f"{catalog_name}.{schema.name}.input_{make_random(6).lower()}" + output_table_name = f"{catalog_name}.{schema.name}.output_{make_random(6).lower()}" + metrics_table_name = f"{catalog_name}.{schema.name}.metrics_{make_random(6).lower()}" + volume = make_volume(catalog_name=catalog_name, schema_name=schema.name) + + observer = DQMetricsObserver(name="test_save_streaming_observer") + dq_engine = DQEngine(workspace_client=ws, spark=spark, observer=observer, extra_params=EXTRA_PARAMS) + + test_df = spark.createDataFrame( + [ + [1, "Alice", 30, 50000], + [2, "Bob", 25, 45000], + [None, "Charlie", 35, 60000], # This will trigger an error + [4, None, 28, 55000], # This will trigger a warning + ], + TEST_SCHEMA, + ) + test_df.write.saveAsTable(input_table_name) + streaming_df = spark.readStream.table(input_table_name) + checked_df, observation = dq_engine.apply_checks_by_metadata(streaming_df, TEST_CHECKS) + + output_config = OutputConfig( + location=output_table_name, + options={"checkPointLocation": f"/Volumes/{volume.catalog_name}/{volume.schema_name}/{volume.name}/output"}, + trigger={"availableNow": True}, + ) + metrics_config = OutputConfig(location=metrics_table_name) + + dq_engine.save_results_in_table( + output_df=checked_df, + observation=observation, + output_config=output_config, + metrics_config=metrics_config, + ) + + expected_metrics = { + "input_row_count": 4, + "error_row_count": 1, + "warning_row_count": 1, + "valid_row_count": 2, + } + actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + assert actual_metrics == expected_metrics From 9da72dc875193b973508f2d0c825306ea584238c Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Wed, 1 Oct 2025 10:58:51 -0400 Subject: [PATCH 18/27] Update unit tests --- tests/unit/test_observer.py | 30 +++++++++++++++--------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/unit/test_observer.py b/tests/unit/test_observer.py index d01b9dc4c..5827bd76e 100644 --- a/tests/unit/test_observer.py +++ b/tests/unit/test_observer.py @@ -12,10 +12,10 @@ def test_dq_observer_default_initialization(): assert observer.custom_metrics is None expected_default_metrics = [ - "count(1) as input_count", - "count(case when _errors is not null then 1 end) as error_count", - "count(case when _warnings is not null then 1 end) as warning_count", - "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + "count(1) as input_row_count", + "count(case when _errors is not null then 1 end) as error_row_count", + "count(case when _warnings is not null then 1 end) as warning_row_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_row_count", ] assert observer.metrics == expected_default_metrics @@ -29,10 +29,10 @@ def test_dq_observer_with_custom_metrics(): assert observer.custom_metrics == custom_metrics expected_metrics = [ - "count(1) as input_count", - "count(case when _errors is not null then 1 end) as error_count", - "count(case when _warnings is not null then 1 end) as warning_count", - "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + "count(1) as input_row_count", + "count(case when _errors is not null then 1 end) as error_row_count", + "count(case when _warnings is not null then 1 end) as warning_row_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_row_count", "avg(age) as avg_age", "count(case when age > 65 then 1 end) as senior_count", ] @@ -44,10 +44,10 @@ def test_dq_observer_empty_custom_metrics(): observer = DQMetricsObserver(custom_metrics=[]) expected_default_metrics = [ - "count(1) as input_count", - "count(case when _errors is not null then 1 end) as error_count", - "count(case when _warnings is not null then 1 end) as warning_count", - "count(case when _errors is null and _warnings is null then 1 end) as valid_count", + "count(1) as input_row_count", + "count(case when _errors is not null then 1 end) as error_row_count", + "count(case when _warnings is not null then 1 end) as warning_row_count", + "count(case when _errors is null and _warnings is null then 1 end) as valid_row_count", ] assert observer.metrics == expected_default_metrics @@ -58,10 +58,10 @@ def test_dq_observer_default_column_names(): errors_column = DefaultColumnNames.ERRORS.value warnings_column = DefaultColumnNames.WARNINGS.value - assert f"count(case when {errors_column} is not null then 1 end) as error_count" in observer.metrics - assert f"count(case when {warnings_column} is not null then 1 end) as warning_count" in observer.metrics + assert f"count(case when {errors_column} is not null then 1 end) as error_row_count" in observer.metrics + assert f"count(case when {warnings_column} is not null then 1 end) as warning_row_count" in observer.metrics assert ( - f"count(case when {errors_column} is null and {warnings_column} is null then 1 end) as valid_count" + f"count(case when {errors_column} is null and {warnings_column} is null then 1 end) as valid_row_count" in observer.metrics ) From 9cfee10f8b17094b09333f966cdf612b02c9ec8a Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Wed, 1 Oct 2025 11:07:29 -0400 Subject: [PATCH 19/27] Fix unit test type --- tests/unit/test_observer.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/unit/test_observer.py b/tests/unit/test_observer.py index 5827bd76e..3dfb5b4ad 100644 --- a/tests/unit/test_observer.py +++ b/tests/unit/test_observer.py @@ -1,6 +1,7 @@ """Unit tests for DQObserver class.""" from pyspark.sql import Observation +from pyspark.sql.connect.observation import Observation as SparkConnectObservation from databricks.labs.dqx.metrics_observer import DQMetricsObserver from databricks.labs.dqx.rule import DefaultColumnNames @@ -70,5 +71,5 @@ def test_dq_observer_observation_property(): """Test that the observation property creates a Spark Observation.""" observer = DQMetricsObserver(name="test_obs") observation = observer.observation - assert isinstance(observation, Observation) assert observation is not None + assert isinstance(observation, Observation | SparkConnectObservation) From 52e3a2238e8dd7a7bb8a6560fe15c83e033bf245 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Thu, 2 Oct 2025 18:23:36 -0400 Subject: [PATCH 20/27] Refactor --- src/databricks/labs/dqx/base.py | 16 ++--- src/databricks/labs/dqx/engine.py | 104 ++++++++++++++++++------------ 2 files changed, 69 insertions(+), 51 deletions(-) diff --git a/src/databricks/labs/dqx/base.py b/src/databricks/labs/dqx/base.py index 741df435c..df63d32f5 100644 --- a/src/databricks/labs/dqx/base.py +++ b/src/databricks/labs/dqx/base.py @@ -43,7 +43,7 @@ class DQEngineCoreBase(DQEngineBase): @abc.abstractmethod def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """Apply data quality checks to the given DataFrame. Args: @@ -53,13 +53,13 @@ def apply_checks( Returns: A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality - summary metrics. + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ @abc.abstractmethod def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -71,7 +71,7 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows with errors or warnings and the corresponding result columns) and an optional Observation which tracks data - quality summary metrics. + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ @abc.abstractmethod @@ -81,7 +81,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """ Apply data quality checks defined as metadata to the given DataFrame. @@ -97,7 +97,7 @@ def apply_checks_by_metadata( Returns: A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality - summary metrics. + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ @abc.abstractmethod @@ -107,7 +107,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -124,7 +124,7 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows with errors or warnings and the corresponding result columns) and an optional Observation which tracks data - quality summary metrics. + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ @staticmethod diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 355b2de13..718f214e9 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -91,7 +91,7 @@ def result_column_names(self) -> dict[ColumnArguments, str]: def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """Apply data quality checks to the given DataFrame. Args: @@ -100,14 +100,14 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. Raises: InvalidCheckError: If any of the checks are invalid. """ if not checks: - return self._append_empty_checks(df), None + return self._append_empty_checks(df) if not DQEngineCore._all_are_dq_rules(checks): raise InvalidCheckError( @@ -123,13 +123,17 @@ def apply_checks( result_df = self._create_results_array( result_df, warning_checks, self._result_column_names[ColumnArguments.WARNINGS], ref_dfs ) - observed_df, observation = self._observe_metrics(result_df) + observed_result = self._observe_metrics(result_df) - return observed_df, observation + if isinstance(observed_result, tuple): + observed_df, observation = observed_result + return observed_df, observation + + return observed_result def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -140,26 +144,31 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. Raises: InvalidCheckError: If any of the checks are invalid. """ if not checks: - return df, self._append_empty_checks(df).limit(0), None + return df, self._append_empty_checks(df).limit(0) if not DQEngineCore._all_are_dq_rules(checks): raise InvalidCheckError( "All elements in the 'checks' list must be instances of DQRule. Use 'apply_checks_by_metadata_and_split' to pass checks as list of dicts instead." ) - checked_df, observation = self.apply_checks(df, checks, ref_dfs) + observed_result = self.apply_checks(df, checks, ref_dfs) - good_df = self.get_valid(checked_df) - bad_df = self.get_invalid(checked_df) + if isinstance(observed_result, tuple): + checked_df, observation = observed_result + good_df = self.get_valid(checked_df) + bad_df = self.get_invalid(checked_df) + return good_df, bad_df, observation - return good_df, bad_df, observation + good_df = self.get_valid(observed_result) + bad_df = self.get_invalid(observed_result) + return good_df, bad_df def apply_checks_by_metadata( self, @@ -167,7 +176,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """Apply data quality checks defined as metadata to the given DataFrame. Args: @@ -181,8 +190,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) @@ -194,7 +203,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -210,16 +219,20 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. Raises: InvalidCheckError: If any of the checks are invalid. """ dq_rule_checks = deserialize_checks(checks, custom_check_functions) - good_df, bad_df, observation = self.apply_checks_and_split(df, dq_rule_checks, ref_dfs) - return good_df, bad_df, observation + good_df, bad_df, *observations = self.apply_checks_and_split(df, dq_rule_checks, ref_dfs) + + if self.observer: + return good_df, bad_df, observations[0] + + return good_df, bad_df @staticmethod def validate_checks( @@ -389,7 +402,7 @@ def _create_results_array( # Ensure the result DataFrame has the same columns as the input DataFrame + the new result column return result_df.select(*df.columns, dest_col) - def _observe_metrics(self, df: DataFrame) -> tuple[DataFrame, Observation | None]: + def _observe_metrics(self, df: DataFrame) -> DataFrame | tuple[DataFrame, Observation]: """ Adds Spark observable metrics to the input DataFrame. @@ -400,13 +413,12 @@ def _observe_metrics(self, df: DataFrame) -> tuple[DataFrame, Observation | None The unmodified DataFrame with observed metrics and the corresponding Spark Observation """ if not self.observer: - return df, None + return df observation = self.observer.observation - observer_id = self.observer.observation_id return ( df.observe( - observer_id, + observation, *[F.expr(metric_statement) for metric_statement in self.observer.metrics], ), observation, @@ -442,7 +454,7 @@ def __init__( @telemetry_logger("engine", "apply_checks") def apply_checks( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """Apply data quality checks to the given DataFrame. Args: @@ -451,15 +463,15 @@ def apply_checks( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ return self._engine.apply_checks(df, checks, ref_dfs) @telemetry_logger("engine", "apply_checks_and_split") def apply_checks_and_split( self, df: DataFrame, checks: list[DQRule], ref_dfs: dict[str, DataFrame] | None = None - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -470,8 +482,8 @@ def apply_checks_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. Raises: InvalidCheckError: If any of the checks are invalid. @@ -485,7 +497,7 @@ def apply_checks_by_metadata( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, Observation | None]: + ) -> DataFrame | tuple[DataFrame, Observation]: """Apply data quality checks defined as metadata to the given DataFrame. Args: @@ -499,8 +511,8 @@ def apply_checks_by_metadata( ref_dfs: Optional reference DataFrames to use in the checks. Returns: - A DataFrame with errors and warnings result columns and an Observation which tracks data quality summary - metrics. + A DataFrame with errors and warnings result columns and an optional Observation which tracks data quality + summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ return self._engine.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) @@ -511,7 +523,7 @@ def apply_checks_by_metadata_and_split( checks: list[dict], custom_check_functions: dict[str, Callable] | None = None, ref_dfs: dict[str, DataFrame] | None = None, - ) -> tuple[DataFrame, DataFrame, Observation | None]: + ) -> tuple[DataFrame, DataFrame] | tuple[DataFrame, DataFrame, Observation]: """Apply data quality checks defined as metadata to the given DataFrame and split the results into two DataFrames ("good" and "bad"). @@ -527,8 +539,8 @@ def apply_checks_by_metadata_and_split( Returns: A tuple of two DataFrames: "good" (may include rows with warnings but no result columns) and "bad" (rows - with errors or warnings and the corresponding result columns) and an Observation which tracks data quality - summary metrics. + with errors or warnings and the corresponding result columns) and an optional Observation which tracks data + quality summary metrics. Summary metrics are returned by any `DQEngine` with an `observer` specified. """ return self._engine.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) @@ -549,7 +561,10 @@ def apply_checks_and_save_in_table( - valid records are written using *output_config*. - invalid records are written using *quarantine_config*. - If *quarantine_config* is not provided, write all rows (including result columns) using *output_config*. + If *quarantine_config* is not provided, write all rows (including result columns) using *quarantine_config*. + + If *metrics_config* is provided and the `DQEngine` has a valid `observer`, data quality summary metrics will be + tracked and written using *metrics_config*. Args: checks: List of *DQRule* checks to apply. @@ -573,16 +588,17 @@ def apply_checks_and_save_in_table( if quarantine_config: # Split data into good and bad records - good_df, bad_df, observation = self.apply_checks_and_split(df, checks, ref_dfs) + good_df, bad_df, *observations = self.apply_checks_and_split(df, checks, ref_dfs) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df, observation = self.apply_checks(df, checks, ref_dfs) + checked_df, *observations = self.apply_checks(df, checks, ref_dfs) save_dataframe_as_table(checked_df, output_config) if self._engine.observer and metrics_config and not df.isStreaming: # Create DataFrame with observation metrics - keys as column names, values as data + observation = observations[0] metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, @@ -644,18 +660,19 @@ def apply_checks_by_metadata_and_save_in_table( if quarantine_config: # Split data into good and bad records - good_df, bad_df, observation = self.apply_checks_by_metadata_and_split( + good_df, bad_df, *observations = self.apply_checks_by_metadata_and_split( df, checks, custom_check_functions, ref_dfs ) save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: # Apply checks and write all data to single table - checked_df, observation = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + checked_df, *observations = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) save_dataframe_as_table(checked_df, output_config) if self._engine.observer and metrics_config and not df.isStreaming: # Create DataFrame with observation metrics - keys as column names, values as data + observation = observations[0] metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, @@ -734,6 +751,7 @@ def save_results_in_table( Behavior: - If *output_df* is provided and *output_config* is None, load the run config and use its *output_config*. - If *quarantine_df* is provided and *quarantine_config* is None, load the run config and use its *quarantine_config*. + - If *observation* is provided and *metrics_config* is None, load the run config and use its *metrics_config* - A write occurs only when both a DataFrame and its corresponding config are available. Args: From 9c319e6547cff0b9a804f8af864cfc63e817f029 Mon Sep 17 00:00:00 2001 From: ghanse Date: Thu, 2 Oct 2025 23:11:08 +0000 Subject: [PATCH 21/27] Add pytest-benchmark performance baseline --- docs/dqx/docs/reference/benchmarks.mdx | 16 + tests/perf/.benchmarks/baseline.json | 560 +++++++++++++++++++++++++ 2 files changed, 576 insertions(+) diff --git a/docs/dqx/docs/reference/benchmarks.mdx b/docs/dqx/docs/reference/benchmarks.mdx index 019fea8fc..a2e99bd9e 100644 --- a/docs/dqx/docs/reference/benchmarks.mdx +++ b/docs/dqx/docs/reference/benchmarks.mdx @@ -51,7 +51,10 @@ sidebar_position: 13 | test_benchmark_foreach_sql_expression[n_rows_100000000_n_columns_5] | 0.895489 | 0.888982 | 0.853895 | 0.950998 | 0.041479 | 0.071722 | 0.858589 | 0.930311 | 5 | 0 | 2 | 1.12 | | test_benchmark_foreach_sql_query[n_rows_100000000_n_columns_5] | 4.578799 | 4.602143 | 4.442396 | 4.644892 | 0.083901 | 0.113694 | 4.530776 | 4.644470 | 5 | 0 | 1 | 0.22 | | test_benchmark_foreign_key | 31.784272 | 31.787610 | 31.414708 | 32.123221 | 0.269713 | 0.386951 | 31.597198 | 31.984149 | 5 | 0 | 2 | 0.03 | +| test_benchmark_has_dimension | 0.243909 | 0.239996 | 0.229273 | 0.266771 | 0.015751 | 0.025493 | 0.230572 | 0.256065 | 5 | 0 | 1 | 4.10 | | test_benchmark_has_valid_schema | 0.172078 | 0.172141 | 0.163793 | 0.181081 | 0.006715 | 0.009295 | 0.167010 | 0.176305 | 6 | 0 | 2 | 5.81 | +| test_benchmark_has_x_coordinate_between | 0.229219 | 0.228885 | 0.225284 | 0.234972 | 0.004111 | 0.006897 | 0.225441 | 0.232337 | 5 | 0 | 1 | 4.36 | +| test_benchmark_has_y_coordinate_between | 0.223497 | 0.224757 | 0.217859 | 0.225831 | 0.003231 | 0.002968 | 0.222367 | 0.225335 | 5 | 1 | 1 | 4.47 | | test_benchmark_is_aggr_equal | 0.304401 | 0.305693 | 0.266624 | 0.330403 | 0.026888 | 0.044641 | 0.284540 | 0.329181 | 5 | 0 | 1 | 3.29 | | test_benchmark_is_aggr_not_equal | 0.296462 | 0.296800 | 0.275119 | 0.312035 | 0.013498 | 0.013448 | 0.291054 | 0.304502 | 5 | 0 | 2 | 3.37 | | test_benchmark_is_aggr_not_greater_than | 0.307771 | 0.315185 | 0.277924 | 0.316280 | 0.016705 | 0.010701 | 0.304974 | 0.315675 | 5 | 1 | 1 | 3.25 | @@ -59,6 +62,9 @@ sidebar_position: 13 | test_benchmark_is_data_fresh | 0.279160 | 0.235545 | 0.231767 | 0.430390 | 0.085563 | 0.072198 | 0.233457 | 0.305655 | 5 | 1 | 1 | 3.58 | | test_benchmark_is_data_fresh_per_time_window | 0.259995 | 0.246444 | 0.242483 | 0.291510 | 0.022543 | 0.037112 | 0.243019 | 0.280132 | 5 | 0 | 1 | 3.85 | | test_benchmark_is_equal_to | 0.241270 | 0.241646 | 0.226919 | 0.248632 | 0.008799 | 0.010992 | 0.237380 | 0.248371 | 5 | 0 | 1 | 4.14 | +| test_benchmark_is_geography | 0.226571 | 0.230901 | 0.217006 | 0.234781 | 0.008722 | 0.016172 | 0.217200 | 0.233372 | 5 | 0 | 2 | 4.41 | +| test_benchmark_is_geometry | 0.218641 | 0.217695 | 0.212085 | 0.228994 | 0.006269 | 0.005765 | 0.215164 | 0.220929 | 5 | 0 | 2 | 4.57 | +| test_benchmark_is_geometrycollection | 0.226185 | 0.223787 | 0.220809 | 0.234175 | 0.005740 | 0.009460 | 0.221701 | 0.231161 | 5 | 0 | 1 | 4.42 | | test_benchmark_is_in_list[col1] | 0.236102 | 0.236503 | 0.227829 | 0.246569 | 0.007036 | 0.008989 | 0.230909 | 0.239897 | 5 | 0 | 2 | 4.24 | | test_benchmark_is_in_list[col2] | 0.289593 | 0.276054 | 0.232456 | 0.372550 | 0.053040 | 0.065903 | 0.255264 | 0.321166 | 5 | 0 | 2 | 3.45 | | test_benchmark_is_in_list[col3] | 0.278913 | 0.276144 | 0.261990 | 0.308485 | 0.018107 | 0.021908 | 0.265834 | 0.287742 | 5 | 0 | 1 | 3.59 | @@ -75,6 +81,13 @@ sidebar_position: 13 | test_benchmark_is_ipv6_address_in_cidr[col6_ipv6_c_mid1] | 0.197962 | 0.202916 | 0.185055 | 0.209342 | 0.010224 | 0.016226 | 0.188373 | 0.204599 | 5 | 0 | 2 | 5.05 | | test_benchmark_is_ipv6_address_in_cidr[col7_ipv6_c_mid4] | 0.215360 | 0.201467 | 0.188151 | 0.292795 | 0.038750 | 0.016073 | 0.196102 | 0.212175 | 6 | 1 | 1 | 4.64 | | test_benchmark_is_ipv6_address_in_cidr[col8_ipv6_u_prefix] | 0.196697 | 0.191199 | 0.185784 | 0.217491 | 0.012402 | 0.013114 | 0.189732 | 0.202846 | 5 | 0 | 1 | 5.08 | +| test_benchmark_is_latitude | 0.232348 | 0.235253 | 0.220568 | 0.242974 | 0.010793 | 0.020761 | 0.221168 | 0.241928 | 5 | 0 | 2 | 4.30 | +| test_benchmark_is_linestring | 0.224962 | 0.225696 | 0.217528 | 0.231035 | 0.005424 | 0.008661 | 0.220697 | 0.229357 | 5 | 0 | 2 | 4.45 | +| test_benchmark_is_longitude | 0.224432 | 0.223481 | 0.218468 | 0.234974 | 0.006265 | 0.005907 | 0.220691 | 0.226598 | 5 | 0 | 1 | 4.46 | +| test_benchmark_is_multilinestring | 0.234181 | 0.238290 | 0.221883 | 0.243374 | 0.009128 | 0.014868 | 0.225982 | 0.240850 | 5 | 0 | 2 | 4.27 | +| test_benchmark_is_multipoint | 0.234101 | 0.235909 | 0.220248 | 0.246061 | 0.009554 | 0.011954 | 0.227918 | 0.239873 | 5 | 0 | 2 | 4.27 | +| test_benchmark_is_multipolygon | 0.236655 | 0.241523 | 0.213344 | 0.250682 | 0.015312 | 0.022927 | 0.225686 | 0.248614 | 5 | 0 | 1 | 4.23 | +| test_benchmark_is_non_empty_geometry | 0.235376 | 0.227167 | 0.218128 | 0.266182 | 0.019337 | 0.025741 | 0.222195 | 0.247936 | 5 | 0 | 1 | 4.25 | | test_benchmark_is_not_empty[col1] | 0.247378 | 0.244527 | 0.234753 | 0.262444 | 0.010315 | 0.012830 | 0.241421 | 0.254251 | 5 | 0 | 2 | 4.04 | | test_benchmark_is_not_empty[col2] | 0.227237 | 0.223543 | 0.216799 | 0.239127 | 0.010264 | 0.018784 | 0.218867 | 0.237651 | 5 | 0 | 2 | 4.40 | | test_benchmark_is_not_empty[col3] | 0.224964 | 0.221986 | 0.212208 | 0.245085 | 0.012278 | 0.012591 | 0.217945 | 0.230536 | 5 | 0 | 2 | 4.45 | @@ -109,8 +122,11 @@ sidebar_position: 13 | test_benchmark_is_null_or_empty[col6] | 0.271883 | 0.288303 | 0.233084 | 0.291877 | 0.025875 | 0.038427 | 0.251213 | 0.289639 | 5 | 0 | 1 | 3.68 | | test_benchmark_is_null_or_empty[col7] | 0.255479 | 0.255281 | 0.230887 | 0.289014 | 0.022139 | 0.029264 | 0.238686 | 0.267950 | 5 | 0 | 2 | 3.91 | | test_benchmark_is_null_or_empty[col8] | 0.219256 | 0.217609 | 0.214083 | 0.226223 | 0.005137 | 0.008602 | 0.215124 | 0.223726 | 5 | 0 | 2 | 4.56 | +| test_benchmark_is_ogc_valid | 0.220437 | 0.220308 | 0.215645 | 0.225761 | 0.004216 | 0.007236 | 0.216734 | 0.223970 | 5 | 0 | 2 | 4.54 | | test_benchmark_is_older_than_col2_for_n_days | 0.235241 | 0.230978 | 0.224354 | 0.254865 | 0.011884 | 0.013734 | 0.227788 | 0.241522 | 5 | 0 | 1 | 4.25 | | test_benchmark_is_older_than_n_days | 0.246935 | 0.248889 | 0.234393 | 0.253353 | 0.007733 | 0.010372 | 0.242547 | 0.252920 | 5 | 0 | 1 | 4.05 | +| test_benchmark_is_point | 0.222127 | 0.221192 | 0.216315 | 0.229766 | 0.005090 | 0.006753 | 0.218645 | 0.225398 | 5 | 0 | 2 | 4.50 | +| test_benchmark_is_polygon | 0.224824 | 0.222314 | 0.219383 | 0.238668 | 0.007833 | 0.005581 | 0.220875 | 0.226456 | 5 | 1 | 1 | 4.45 | | test_benchmark_is_unique | 0.248614 | 0.245772 | 0.243054 | 0.259295 | 0.006731 | 0.009422 | 0.243688 | 0.253109 | 5 | 0 | 1 | 4.02 | | test_benchmark_is_valid_date | 0.242393 | 0.229097 | 0.222715 | 0.270948 | 0.022056 | 0.037082 | 0.226619 | 0.263702 | 5 | 0 | 1 | 4.13 | | test_benchmark_is_valid_ipv4_address[col1_ipv4_standard] | 0.165535 | 0.161430 | 0.158404 | 0.188479 | 0.011344 | 0.002309 | 0.160578 | 0.162887 | 6 | 1 | 1 | 6.04 | diff --git a/tests/perf/.benchmarks/baseline.json b/tests/perf/.benchmarks/baseline.json index d65319fec..522f8b57f 100644 --- a/tests/perf/.benchmarks/baseline.json +++ b/tests/perf/.benchmarks/baseline.json @@ -1541,6 +1541,41 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_has_dimension", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_dimension", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22927346699998452, + "max": 0.26677058599989323, + "mean": 0.24390853020004216, + "stddev": 0.01575137524903365, + "rounds": 5, + "median": 0.2399963640000351, + "iqr": 0.0254931987500413, + "q1": 0.23057224500007578, + "q3": 0.2560654437501171, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22927346699998452, + "hd15iqr": 0.26677058599989323, + "ops": 4.099897609894364, + "total": 1.2195426510002108, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_has_valid_schema", @@ -1576,6 +1611,76 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_has_x_coordinate_between", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_x_coordinate_between", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22528417800003808, + "max": 0.23497163499996532, + "mean": 0.22921857799992723, + "stddev": 0.004110638636705224, + "rounds": 5, + "median": 0.22888549199979025, + "iqr": 0.0068965210004989785, + "q1": 0.22544056049969186, + "q3": 0.23233708150019083, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22528417800003808, + "hd15iqr": 0.23497163499996532, + "ops": 4.362648127065501, + "total": 1.146092889999636, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_has_y_coordinate_between", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_y_coordinate_between", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21785882099993614, + "max": 0.2258314989999235, + "mean": 0.22349742219985275, + "stddev": 0.003231259303985809, + "rounds": 5, + "median": 0.22475687199994354, + "iqr": 0.002968186749740198, + "q1": 0.22236716624991004, + "q3": 0.22533535299965024, + "iqr_outliers": 1, + "stddev_outliers": 1, + "outliers": "1;1", + "ld15iqr": 0.22386994799990134, + "hd15iqr": 0.2258314989999235, + "ops": 4.4743245365299735, + "total": 1.1174871109992637, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_aggr_equal", @@ -1821,6 +1926,111 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_geography", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geography", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21700612699987687, + "max": 0.2347806469997522, + "mean": 0.22657076599998618, + "stddev": 0.008722428619097212, + "rounds": 5, + "median": 0.23090056099999856, + "iqr": 0.0161717707499065, + "q1": 0.2171998970001141, + "q3": 0.2333716677500206, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21700612699987687, + "hd15iqr": 0.2347806469997522, + "ops": 4.413632074669603, + "total": 1.132853829999931, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_geometry", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geometry", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21208457500006261, + "max": 0.22899367299987716, + "mean": 0.21864081899993834, + "stddev": 0.006269493398778712, + "rounds": 5, + "median": 0.21769451999989542, + "iqr": 0.005765286750033738, + "q1": 0.2151638852499218, + "q3": 0.22092917199995554, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21208457500006261, + "hd15iqr": 0.22899367299987716, + "ops": 4.573711370886706, + "total": 1.0932040949996917, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_geometrycollection", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geometrycollection", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22080926100034048, + "max": 0.2341752649999762, + "mean": 0.22618524760000583, + "stddev": 0.005740098679045008, + "rounds": 5, + "median": 0.22378674699984913, + "iqr": 0.009460075249876354, + "q1": 0.22170114000005015, + "q3": 0.2311612152499265, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22080926100034048, + "hd15iqr": 0.2341752649999762, + "ops": 4.421154830435432, + "total": 1.1309262380000291, + "iterations": 1 + } + }, { "group": "test_benchmark_is_in_list col1", "name": "test_benchmark_is_in_list[col1]", @@ -2411,6 +2621,251 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_latitude", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_latitude", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22056750799993097, + "max": 0.24297376000004078, + "mean": 0.23234819859999334, + "stddev": 0.01079252112627353, + "rounds": 5, + "median": 0.2352526399999988, + "iqr": 0.02076050825019138, + "q1": 0.22116756124989934, + "q3": 0.24192806950009071, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.22056750799993097, + "hd15iqr": 0.24297376000004078, + "ops": 4.303885315339082, + "total": 1.1617409929999667, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_linestring", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_linestring", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21752809200006595, + "max": 0.23103526600016266, + "mean": 0.22496202360007372, + "stddev": 0.0054244459300489154, + "rounds": 5, + "median": 0.22569596500034095, + "iqr": 0.00866051725017769, + "q1": 0.2206967092498644, + "q3": 0.2293572265000421, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21752809200006595, + "hd15iqr": 0.23103526600016266, + "ops": 4.445194722188978, + "total": 1.1248101180003687, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_longitude", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_longitude", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21846815100025196, + "max": 0.23497388700025112, + "mean": 0.22443222580013752, + "stddev": 0.006264883515721421, + "rounds": 5, + "median": 0.22348119900016172, + "iqr": 0.005906900999775644, + "q1": 0.22069101375018363, + "q3": 0.22659791474995927, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21846815100025196, + "hd15iqr": 0.23497388700025112, + "ops": 4.455688110006648, + "total": 1.1221611290006877, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multilinestring", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multilinestring", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22188278100020398, + "max": 0.2433739359998981, + "mean": 0.23418097320000014, + "stddev": 0.009128031976731316, + "rounds": 5, + "median": 0.23829027100009625, + "iqr": 0.014867763250094868, + "q1": 0.22598241224989124, + "q3": 0.2408501754999861, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.22188278100020398, + "hd15iqr": 0.2433739359998981, + "ops": 4.270201743272964, + "total": 1.1709048660000008, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multipoint", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multipoint", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.2202482810002948, + "max": 0.2460606760000701, + "mean": 0.23410069620003923, + "stddev": 0.009553865226408068, + "rounds": 5, + "median": 0.235909207000077, + "iqr": 0.011954297000329461, + "q1": 0.22791846499978874, + "q3": 0.2398727620001182, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.2202482810002948, + "hd15iqr": 0.2460606760000701, + "ops": 4.27166606606543, + "total": 1.1705034810001962, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multipolygon", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multipolygon", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21334442300030787, + "max": 0.2506819240002187, + "mean": 0.2366548676000093, + "stddev": 0.015311892912790935, + "rounds": 5, + "median": 0.24152346799974111, + "iqr": 0.022927085000105762, + "q1": 0.22568644699993, + "q3": 0.24861353200003578, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21334442300030787, + "hd15iqr": 0.2506819240002187, + "ops": 4.2255627790009616, + "total": 1.1832743380000466, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_non_empty_geometry", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_non_empty_geometry", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21812769000007393, + "max": 0.26618207299998176, + "mean": 0.23537642080000296, + "stddev": 0.01933716738528616, + "rounds": 5, + "median": 0.2271670790000826, + "iqr": 0.025741081250089337, + "q1": 0.222195152999916, + "q3": 0.24793623425000533, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21812769000007393, + "hd15iqr": 0.26618207299998176, + "ops": 4.248513919113802, + "total": 1.1768821040000148, + "iterations": 1 + } + }, { "group": "test_benchmark_is_not_empty col1", "name": "test_benchmark_is_not_empty[col1]", @@ -3655,6 +4110,41 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_ogc_valid", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_ogc_valid", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21564534099979937, + "max": 0.2257612359999257, + "mean": 0.22043705859996407, + "stddev": 0.004216087117037423, + "rounds": 5, + "median": 0.22030801499977315, + "iqr": 0.007236216000251261, + "q1": 0.2167342269999608, + "q3": 0.22397044300021207, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21564534099979937, + "hd15iqr": 0.2257612359999257, + "ops": 4.536442313062886, + "total": 1.1021852929998204, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_older_than_col2_for_n_days", @@ -3725,6 +4215,76 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_point", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_point", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21631457500006945, + "max": 0.2297655649999797, + "mean": 0.2221274610000364, + "stddev": 0.005089639674878682, + "rounds": 5, + "median": 0.22119198900008996, + "iqr": 0.006752966500016555, + "q1": 0.218645475250014, + "q3": 0.22539844175003054, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21631457500006945, + "hd15iqr": 0.2297655649999797, + "ops": 4.501919733372525, + "total": 1.110637305000182, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_polygon", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_polygon", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21938274400008595, + "max": 0.23866784099982397, + "mean": 0.22482422739994945, + "stddev": 0.007833050782835247, + "rounds": 5, + "median": 0.22231378099968424, + "iqr": 0.005580997999913961, + "q1": 0.22087461325008917, + "q3": 0.22645561125000313, + "iqr_outliers": 1, + "stddev_outliers": 1, + "outliers": "1;1", + "ld15iqr": 0.21938274400008595, + "hd15iqr": 0.23866784099982397, + "ops": 4.447919210330731, + "total": 1.1241211369997473, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_unique", From 835e1a7c8d4133372505c6d0a77d1b79532120ec Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Thu, 2 Oct 2025 21:38:33 -0400 Subject: [PATCH 22/27] Update implementation, tests, and docs --- docs/dqx/docs/guide/summary_metrics.mdx | 57 ++- src/databricks/labs/dqx/engine.py | 58 +-- tests/integration/conftest.py | 10 + tests/integration/test_metrics_workflow.py | 372 +++++++++++++-- tests/integration/test_summary_metrics.py | 507 ++++++++++++++++++--- 5 files changed, 864 insertions(+), 140 deletions(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index a1f1897a1..b84a3d669 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -366,13 +366,19 @@ custom_metrics: Metrics can be written automatically and centralized by specifying a metrics table. The metrics table will contain the following fields: -| Column Name | Column Type | Description | -|-----------------|-----------------------|-----------------------------------------------------------| -| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | -| `input_table` | `STRING` | Location of the input dataset | -| `metric_key` | `STRING` | Name of the metric | -| `metric_value` | `STRING` | Value of the metric (as a string) | -| `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | +| Column Name | Column Type | Description | +|-----------------------|-----------------------|--------------------------------------------------------------| +| `run_name` | `STRING` | Name of the metrics observer | +| `input_location` | `STRING` | Location of the input dataset (table name or file path) | +| `output_location` | `STRING` | Location of the output dataset (table name or file path) | +| `quarantine_location` | `STRING` | Location of the quarantine dataset (table name or file path) | +| `checks_location` | `STRING` | Location where checks are stored (table name or file path) | +| `metric_name` | `STRING` | Name of the metric (e.g., 'input_row_count') | +| `metric_value` | `STRING` | Value of the metric (stored as string) | +| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | +| `error_column_name` | `STRING` | Name of the error column (default: '_errors') | +| `warning_column_name` | `STRING` | Name of the warning column (default: '_warnings') | +| `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | ## Best Practices @@ -390,16 +396,39 @@ Metrics can be written automatically and centralized by specifying a metrics tab The example query below shows how you can analyze metrics persisted to a table. ```sql /* EXAMPLE: Identify quality degradation */ +WITH daily_metrics AS ( + SELECT + date_trunc('day', run_ts) as run_date, + input_location, + metric_name, + CAST(metric_value AS DOUBLE) as metric_value + FROM + main.analytics.dq_metrics + WHERE + run_ts >= current_date - INTERVAL 30 DAYS + AND metric_name IN ('input_row_count', 'error_row_count', 'warning_row_count') +), +pivoted_metrics AS ( + SELECT + run_date, + input_location, + MAX(CASE WHEN metric_name = 'input_row_count' THEN metric_value END) as input_count, + MAX(CASE WHEN metric_name = 'error_row_count' THEN metric_value END) as error_count, + MAX(CASE WHEN metric_name = 'warning_row_count' THEN metric_value END) as warning_count + FROM daily_metrics + GROUP BY run_date, input_location +) SELECT - date_trunc('day', run_ts) as run_date, - avg(error_count * 100.0 / input_count) as avg_error_rate, - avg(warning_count * 100.0 / input_count) as avg_warning_rate + run_date, + input_location, + avg(error_count * 100.0 / NULLIF(input_count, 0)) as avg_error_rate, + avg(warning_count * 100.0 / NULLIF(input_count, 0)) as avg_warning_rate FROM - main.analytics.dq_metrics + pivoted_metrics WHERE - run_date >= current_date - INTERVAL 30 DAYS + input_count > 0 GROUP BY - date_trunc('day', run_date) + run_date, input_location ORDER BY - run_date DESC + run_date DESC, input_location ``` diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 718f214e9..3c81672ae 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -415,14 +415,14 @@ def _observe_metrics(self, df: DataFrame) -> DataFrame | tuple[DataFrame, Observ if not self.observer: return df - observation = self.observer.observation - return ( - df.observe( - observation, - *[F.expr(metric_statement) for metric_statement in self.observer.metrics], - ), - observation, - ) + metric_exprs = [F.expr(metric_statement) for metric_statement in self.observer.metrics] + if not metric_exprs: + return df + + if df.isStreaming: + return df.observe(self.observer.observation_id, *metric_exprs), self.observer.observation + + return df.observe(self.observer.observation, *metric_exprs), self.observer.observation class DQEngine(DQEngineBase): @@ -574,7 +574,6 @@ def apply_checks_and_save_in_table( metrics_config: Optional configuration for writing summary metrics. ref_dfs: Optional reference DataFrames used by checks. """ - # Read data from the specified table df = read_input_data(self.spark, input_config) if self._engine.observer and metrics_config and df.isStreaming: @@ -586,19 +585,24 @@ def apply_checks_and_save_in_table( ) self.spark.streams.addListener(listener) + observation = None if quarantine_config: - # Split data into good and bad records - good_df, bad_df, *observations = self.apply_checks_and_split(df, checks, ref_dfs) + split_result = self.apply_checks_and_split(df, checks, ref_dfs) + if self._engine.observer: + good_df, bad_df, observation = split_result + else: + good_df, bad_df = split_result save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: - # Apply checks and write all data to single table - checked_df, *observations = self.apply_checks(df, checks, ref_dfs) + check_result = self.apply_checks(df, checks, ref_dfs) + if self._engine.observer: + checked_df, observation = check_result + else: + checked_df = check_result save_dataframe_as_table(checked_df, output_config) - if self._engine.observer and metrics_config and not df.isStreaming: - # Create DataFrame with observation metrics - keys as column names, values as data - observation = observations[0] + if self._engine.observer and metrics_config and not df.isStreaming and observation is not None: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, @@ -646,7 +650,6 @@ def apply_checks_by_metadata_and_save_in_table( to callables/modules (e.g., globals()). ref_dfs: Optional reference DataFrames used by checks. """ - # Read data from the specified table df = read_input_data(self.spark, input_config) if self._engine.observer and metrics_config and df.isStreaming: @@ -658,21 +661,24 @@ def apply_checks_by_metadata_and_save_in_table( ) self.spark.streams.addListener(listener) + observation = None if quarantine_config: - # Split data into good and bad records - good_df, bad_df, *observations = self.apply_checks_by_metadata_and_split( - df, checks, custom_check_functions, ref_dfs - ) + split_result = self.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) + if self._engine.observer: + good_df, bad_df, observation = split_result + else: + good_df, bad_df = split_result save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: - # Apply checks and write all data to single table - checked_df, *observations = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + check_result = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + if self._engine.observer: + checked_df, observation = check_result + else: + checked_df = check_result save_dataframe_as_table(checked_df, output_config) - if self._engine.observer and metrics_config and not df.isStreaming: - # Create DataFrame with observation metrics - keys as column names, values as data - observation = observations[0] + if self._engine.observer and metrics_config and not df.isStreaming and observation is not None: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5f4fd2d81..e820c9769 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -316,3 +316,13 @@ def contains_expected_workflows(workflows, state): if all(item in workflow.items() for item in state.items()): return True return False + + +def validate_metrics(actual_metrics, expected_metrics): + for metric_name in expected_metrics: + actual = actual_metrics[metric_name] + expected = expected_metrics[metric_name] + actual_without_ts = {k: v for k, v in actual.items() if k != "run_ts"} + expected_without_ts = {k: v for k, v in expected.items() if k != "run_ts"} + + assert actual_without_ts == expected_without_ts diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index e9e8cf4b7..d83c8c49a 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -1,3 +1,6 @@ +from tests.integration.conftest import validate_metrics + + def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metrics): """Test that quality checker workflow saves metrics when configured.""" ctx, run_config = setup_workflows_with_metrics(metrics=True) @@ -6,24 +9,73 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri output_df = spark.table(run_config.output_config.location) output_count = output_df.count() - expected_metrics = { - "input_row_count": output_count, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": output_count, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 4 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workflows_with_metrics): """Test workflow with both quarantine and metrics configurations.""" ctx, run_config = setup_workflows_with_metrics( - quarantine=True, metrics=True, custom_metrics=["count(distinct name) as unique_names"] + quarantine=True, metrics=True, custom_metrics=["count(1) as total_names"] ) ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) @@ -32,19 +84,80 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf quarantine_df = spark.table(run_config.quarantine_config.location) quarantine_count = quarantine_df.count() - expected_metrics = { - "input_row_count": output_count + quarantine_count, - "error_row_count": quarantine_count, - "warning_row_count": 0, - "valid_row_count": output_count, - "unique_names": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count + quarantine_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": str(quarantine_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "total_names", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 5 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): @@ -57,20 +170,93 @@ def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): output_df = spark.table(run_config.output_config.location) output_count = output_df.count() - expected_metrics = { - "input_row_count": output_count, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": output_count, - "max_id": 1, - "min_id": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "max_id", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "min_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 6 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): @@ -84,23 +270,121 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=custom_metrics) - expected_metrics = { - "input_row_count": 0, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": 0, - "average_id": 0, - "total_id": 0, - "unique_names": 0, - "id_range": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "average_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "unique_names", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "id_range", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) metrics_df = spark.table(run_config.metrics_config.location) metrics_rows = metrics_df.collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 8 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_quality_checker_workflow_without_metrics_config(setup_workflows_with_metrics): diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 36926618e..9e41f2d9e 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -4,6 +4,7 @@ from databricks.labs.dqx.config import InputConfig, OutputConfig, ExtraParams from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.metrics_observer import DQMetricsObserver +from tests.integration.conftest import validate_metrics RUN_TIME = datetime(2025, 1, 1, 0, 0, 0, 0, tzinfo=timezone.utc) @@ -144,17 +145,90 @@ def test_observer_metrics_output(ws, spark, make_schema, make_random): checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + expected_metrics = [ + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random): @@ -197,17 +271,90 @@ def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_ra metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + expected_metrics = [ + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_random): @@ -279,16 +426,90 @@ def test_streaming_observer_metrics_output(ws, spark, make_schema, make_random, checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random, make_volume): @@ -340,16 +561,90 @@ def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, mak metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_random): @@ -387,14 +682,64 @@ def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_r metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, make_random, make_volume): @@ -436,11 +781,61 @@ def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, ma metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) From f74d877508578f8e132e7d9808a40ca58961bec4 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Thu, 2 Oct 2025 22:04:31 -0400 Subject: [PATCH 23/27] Fix observation return --- src/databricks/labs/dqx/engine.py | 5 +++-- tests/integration/test_metrics_workflow.py | 10 +++++----- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 3c81672ae..91d5d497f 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -419,10 +419,11 @@ def _observe_metrics(self, df: DataFrame) -> DataFrame | tuple[DataFrame, Observ if not metric_exprs: return df + observation = self.observer.observation if df.isStreaming: - return df.observe(self.observer.observation_id, *metric_exprs), self.observer.observation + return df.observe(self.observer.name, *metric_exprs), observation - return df.observe(self.observer.observation, *metric_exprs), self.observer.observation + return df.observe(observation, *metric_exprs), observation class DQEngine(DQEngineBase): diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index d83c8c49a..5b13060de 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -75,7 +75,7 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workflows_with_metrics): """Test workflow with both quarantine and metrics configurations.""" ctx, run_config = setup_workflows_with_metrics( - quarantine=True, metrics=True, custom_metrics=["count(1) as total_names"] + quarantine=True, metrics=True, custom_metrics=["count(1) as total_ids"] ) ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) @@ -143,7 +143,7 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf "output_location": run_config.output_config.location, "quarantine_location": run_config.quarantine_config.location, "checks_location": None, - "metric_name": "total_names", + "metric_name": "total_ids", "metric_value": "4", "run_ts": None, # Will be set at runtime "error_column_name": "_errors", @@ -264,7 +264,7 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): custom_metrics = [ "avg(id) as average_id", "sum(id) as total_id", - "count(distinct name) as unique_names", + "count(1) as total_ids", "max(id) - min(id) as id_range", ] @@ -355,8 +355,8 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): "output_location": run_config.output_config.location, "quarantine_location": None, "checks_location": None, - "metric_name": "unique_names", - "metric_value": "0", + "metric_name": "total_ids", + "metric_value": "1", "run_ts": None, # Will be set at runtime "error_column_name": "_errors", "warning_column_name": "_warnings", From 6fcf0840e58914048782f30d18e3a74a698e21e8 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Fri, 3 Oct 2025 09:04:01 -0400 Subject: [PATCH 24/27] Fix user metadata in summary metrics table --- src/databricks/labs/dqx/engine.py | 4 ++-- src/databricks/labs/dqx/metrics_observer.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 91d5d497f..447bd2891 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -807,7 +807,7 @@ def save_results_in_table( is_streaming_quarantine = quarantine_df is not None and quarantine_df.isStreaming is_streaming = is_streaming_output or is_streaming_quarantine - if observation is not None and metrics_config is not None and is_streaming: + if self._engine.observer and observation is not None and metrics_config is not None and is_streaming: listener = self._get_streaming_metrics_listener( output_config=output_config, quarantine_config=quarantine_config, @@ -821,7 +821,7 @@ def save_results_in_table( if quarantine_df is not None and quarantine_config is not None: save_dataframe_as_table(quarantine_df, quarantine_config) - if self._engine.observer and observation is not None and metrics_config is not None: + if self._engine.observer and observation is not None and metrics_config is not None and not is_streaming: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, diff --git a/src/databricks/labs/dqx/metrics_observer.py b/src/databricks/labs/dqx/metrics_observer.py index 4674ea55c..a0092228e 100644 --- a/src/databricks/labs/dqx/metrics_observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -94,7 +94,7 @@ def metrics(self) -> list[str]: default_metrics.extend(self.custom_metrics) return default_metrics - @property + @cached_property def observation(self) -> Observation: """ Spark `Observation` which can be attached to a `DataFrame` to track summary metrics. Metrics will be collected @@ -147,7 +147,7 @@ def build_metrics_df(spark: SparkSession, observation: DQMetricsObservation) -> observation.run_time, observation.error_column_name, observation.warning_column_name, - observation.user_metadata, + observation.user_metadata if observation.user_metadata else None, ] for metric_key, metric_value in observation.observed_metrics.items() ], From 88525ca0e0ce03cb147c6e9af2da1f69494536ff Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Fri, 3 Oct 2025 09:04:01 -0400 Subject: [PATCH 25/27] Fix user metadata in summary metrics table --- docs/dqx/docs/guide/summary_metrics.mdx | 57 +- docs/dqx/docs/reference/benchmarks.mdx | 16 + src/databricks/labs/dqx/engine.py | 61 ++- src/databricks/labs/dqx/metrics_observer.py | 4 +- tests/integration/conftest.py | 10 + tests/integration/test_metrics_workflow.py | 374 +++++++++++-- tests/integration/test_summary_metrics.py | 507 ++++++++++++++++-- tests/perf/.benchmarks/baseline.json | 560 ++++++++++++++++++++ 8 files changed, 1445 insertions(+), 144 deletions(-) diff --git a/docs/dqx/docs/guide/summary_metrics.mdx b/docs/dqx/docs/guide/summary_metrics.mdx index a1f1897a1..b84a3d669 100644 --- a/docs/dqx/docs/guide/summary_metrics.mdx +++ b/docs/dqx/docs/guide/summary_metrics.mdx @@ -366,13 +366,19 @@ custom_metrics: Metrics can be written automatically and centralized by specifying a metrics table. The metrics table will contain the following fields: -| Column Name | Column Type | Description | -|-----------------|-----------------------|-----------------------------------------------------------| -| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | -| `input_table` | `STRING` | Location of the input dataset | -| `metric_key` | `STRING` | Name of the metric | -| `metric_value` | `STRING` | Value of the metric (as a string) | -| `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | +| Column Name | Column Type | Description | +|-----------------------|-----------------------|--------------------------------------------------------------| +| `run_name` | `STRING` | Name of the metrics observer | +| `input_location` | `STRING` | Location of the input dataset (table name or file path) | +| `output_location` | `STRING` | Location of the output dataset (table name or file path) | +| `quarantine_location` | `STRING` | Location of the quarantine dataset (table name or file path) | +| `checks_location` | `STRING` | Location where checks are stored (table name or file path) | +| `metric_name` | `STRING` | Name of the metric (e.g., 'input_row_count') | +| `metric_value` | `STRING` | Value of the metric (stored as string) | +| `run_ts` | `TIMESTAMP` | Run timestamp when the summary metrics were calculated | +| `error_column_name` | `STRING` | Name of the error column (default: '_errors') | +| `warning_column_name` | `STRING` | Name of the warning column (default: '_warnings') | +| `user_metadata` | `MAP[STRING, STRING]` | User-defined, run-level metadata | ## Best Practices @@ -390,16 +396,39 @@ Metrics can be written automatically and centralized by specifying a metrics tab The example query below shows how you can analyze metrics persisted to a table. ```sql /* EXAMPLE: Identify quality degradation */ +WITH daily_metrics AS ( + SELECT + date_trunc('day', run_ts) as run_date, + input_location, + metric_name, + CAST(metric_value AS DOUBLE) as metric_value + FROM + main.analytics.dq_metrics + WHERE + run_ts >= current_date - INTERVAL 30 DAYS + AND metric_name IN ('input_row_count', 'error_row_count', 'warning_row_count') +), +pivoted_metrics AS ( + SELECT + run_date, + input_location, + MAX(CASE WHEN metric_name = 'input_row_count' THEN metric_value END) as input_count, + MAX(CASE WHEN metric_name = 'error_row_count' THEN metric_value END) as error_count, + MAX(CASE WHEN metric_name = 'warning_row_count' THEN metric_value END) as warning_count + FROM daily_metrics + GROUP BY run_date, input_location +) SELECT - date_trunc('day', run_ts) as run_date, - avg(error_count * 100.0 / input_count) as avg_error_rate, - avg(warning_count * 100.0 / input_count) as avg_warning_rate + run_date, + input_location, + avg(error_count * 100.0 / NULLIF(input_count, 0)) as avg_error_rate, + avg(warning_count * 100.0 / NULLIF(input_count, 0)) as avg_warning_rate FROM - main.analytics.dq_metrics + pivoted_metrics WHERE - run_date >= current_date - INTERVAL 30 DAYS + input_count > 0 GROUP BY - date_trunc('day', run_date) + run_date, input_location ORDER BY - run_date DESC + run_date DESC, input_location ``` diff --git a/docs/dqx/docs/reference/benchmarks.mdx b/docs/dqx/docs/reference/benchmarks.mdx index 019fea8fc..a2e99bd9e 100644 --- a/docs/dqx/docs/reference/benchmarks.mdx +++ b/docs/dqx/docs/reference/benchmarks.mdx @@ -51,7 +51,10 @@ sidebar_position: 13 | test_benchmark_foreach_sql_expression[n_rows_100000000_n_columns_5] | 0.895489 | 0.888982 | 0.853895 | 0.950998 | 0.041479 | 0.071722 | 0.858589 | 0.930311 | 5 | 0 | 2 | 1.12 | | test_benchmark_foreach_sql_query[n_rows_100000000_n_columns_5] | 4.578799 | 4.602143 | 4.442396 | 4.644892 | 0.083901 | 0.113694 | 4.530776 | 4.644470 | 5 | 0 | 1 | 0.22 | | test_benchmark_foreign_key | 31.784272 | 31.787610 | 31.414708 | 32.123221 | 0.269713 | 0.386951 | 31.597198 | 31.984149 | 5 | 0 | 2 | 0.03 | +| test_benchmark_has_dimension | 0.243909 | 0.239996 | 0.229273 | 0.266771 | 0.015751 | 0.025493 | 0.230572 | 0.256065 | 5 | 0 | 1 | 4.10 | | test_benchmark_has_valid_schema | 0.172078 | 0.172141 | 0.163793 | 0.181081 | 0.006715 | 0.009295 | 0.167010 | 0.176305 | 6 | 0 | 2 | 5.81 | +| test_benchmark_has_x_coordinate_between | 0.229219 | 0.228885 | 0.225284 | 0.234972 | 0.004111 | 0.006897 | 0.225441 | 0.232337 | 5 | 0 | 1 | 4.36 | +| test_benchmark_has_y_coordinate_between | 0.223497 | 0.224757 | 0.217859 | 0.225831 | 0.003231 | 0.002968 | 0.222367 | 0.225335 | 5 | 1 | 1 | 4.47 | | test_benchmark_is_aggr_equal | 0.304401 | 0.305693 | 0.266624 | 0.330403 | 0.026888 | 0.044641 | 0.284540 | 0.329181 | 5 | 0 | 1 | 3.29 | | test_benchmark_is_aggr_not_equal | 0.296462 | 0.296800 | 0.275119 | 0.312035 | 0.013498 | 0.013448 | 0.291054 | 0.304502 | 5 | 0 | 2 | 3.37 | | test_benchmark_is_aggr_not_greater_than | 0.307771 | 0.315185 | 0.277924 | 0.316280 | 0.016705 | 0.010701 | 0.304974 | 0.315675 | 5 | 1 | 1 | 3.25 | @@ -59,6 +62,9 @@ sidebar_position: 13 | test_benchmark_is_data_fresh | 0.279160 | 0.235545 | 0.231767 | 0.430390 | 0.085563 | 0.072198 | 0.233457 | 0.305655 | 5 | 1 | 1 | 3.58 | | test_benchmark_is_data_fresh_per_time_window | 0.259995 | 0.246444 | 0.242483 | 0.291510 | 0.022543 | 0.037112 | 0.243019 | 0.280132 | 5 | 0 | 1 | 3.85 | | test_benchmark_is_equal_to | 0.241270 | 0.241646 | 0.226919 | 0.248632 | 0.008799 | 0.010992 | 0.237380 | 0.248371 | 5 | 0 | 1 | 4.14 | +| test_benchmark_is_geography | 0.226571 | 0.230901 | 0.217006 | 0.234781 | 0.008722 | 0.016172 | 0.217200 | 0.233372 | 5 | 0 | 2 | 4.41 | +| test_benchmark_is_geometry | 0.218641 | 0.217695 | 0.212085 | 0.228994 | 0.006269 | 0.005765 | 0.215164 | 0.220929 | 5 | 0 | 2 | 4.57 | +| test_benchmark_is_geometrycollection | 0.226185 | 0.223787 | 0.220809 | 0.234175 | 0.005740 | 0.009460 | 0.221701 | 0.231161 | 5 | 0 | 1 | 4.42 | | test_benchmark_is_in_list[col1] | 0.236102 | 0.236503 | 0.227829 | 0.246569 | 0.007036 | 0.008989 | 0.230909 | 0.239897 | 5 | 0 | 2 | 4.24 | | test_benchmark_is_in_list[col2] | 0.289593 | 0.276054 | 0.232456 | 0.372550 | 0.053040 | 0.065903 | 0.255264 | 0.321166 | 5 | 0 | 2 | 3.45 | | test_benchmark_is_in_list[col3] | 0.278913 | 0.276144 | 0.261990 | 0.308485 | 0.018107 | 0.021908 | 0.265834 | 0.287742 | 5 | 0 | 1 | 3.59 | @@ -75,6 +81,13 @@ sidebar_position: 13 | test_benchmark_is_ipv6_address_in_cidr[col6_ipv6_c_mid1] | 0.197962 | 0.202916 | 0.185055 | 0.209342 | 0.010224 | 0.016226 | 0.188373 | 0.204599 | 5 | 0 | 2 | 5.05 | | test_benchmark_is_ipv6_address_in_cidr[col7_ipv6_c_mid4] | 0.215360 | 0.201467 | 0.188151 | 0.292795 | 0.038750 | 0.016073 | 0.196102 | 0.212175 | 6 | 1 | 1 | 4.64 | | test_benchmark_is_ipv6_address_in_cidr[col8_ipv6_u_prefix] | 0.196697 | 0.191199 | 0.185784 | 0.217491 | 0.012402 | 0.013114 | 0.189732 | 0.202846 | 5 | 0 | 1 | 5.08 | +| test_benchmark_is_latitude | 0.232348 | 0.235253 | 0.220568 | 0.242974 | 0.010793 | 0.020761 | 0.221168 | 0.241928 | 5 | 0 | 2 | 4.30 | +| test_benchmark_is_linestring | 0.224962 | 0.225696 | 0.217528 | 0.231035 | 0.005424 | 0.008661 | 0.220697 | 0.229357 | 5 | 0 | 2 | 4.45 | +| test_benchmark_is_longitude | 0.224432 | 0.223481 | 0.218468 | 0.234974 | 0.006265 | 0.005907 | 0.220691 | 0.226598 | 5 | 0 | 1 | 4.46 | +| test_benchmark_is_multilinestring | 0.234181 | 0.238290 | 0.221883 | 0.243374 | 0.009128 | 0.014868 | 0.225982 | 0.240850 | 5 | 0 | 2 | 4.27 | +| test_benchmark_is_multipoint | 0.234101 | 0.235909 | 0.220248 | 0.246061 | 0.009554 | 0.011954 | 0.227918 | 0.239873 | 5 | 0 | 2 | 4.27 | +| test_benchmark_is_multipolygon | 0.236655 | 0.241523 | 0.213344 | 0.250682 | 0.015312 | 0.022927 | 0.225686 | 0.248614 | 5 | 0 | 1 | 4.23 | +| test_benchmark_is_non_empty_geometry | 0.235376 | 0.227167 | 0.218128 | 0.266182 | 0.019337 | 0.025741 | 0.222195 | 0.247936 | 5 | 0 | 1 | 4.25 | | test_benchmark_is_not_empty[col1] | 0.247378 | 0.244527 | 0.234753 | 0.262444 | 0.010315 | 0.012830 | 0.241421 | 0.254251 | 5 | 0 | 2 | 4.04 | | test_benchmark_is_not_empty[col2] | 0.227237 | 0.223543 | 0.216799 | 0.239127 | 0.010264 | 0.018784 | 0.218867 | 0.237651 | 5 | 0 | 2 | 4.40 | | test_benchmark_is_not_empty[col3] | 0.224964 | 0.221986 | 0.212208 | 0.245085 | 0.012278 | 0.012591 | 0.217945 | 0.230536 | 5 | 0 | 2 | 4.45 | @@ -109,8 +122,11 @@ sidebar_position: 13 | test_benchmark_is_null_or_empty[col6] | 0.271883 | 0.288303 | 0.233084 | 0.291877 | 0.025875 | 0.038427 | 0.251213 | 0.289639 | 5 | 0 | 1 | 3.68 | | test_benchmark_is_null_or_empty[col7] | 0.255479 | 0.255281 | 0.230887 | 0.289014 | 0.022139 | 0.029264 | 0.238686 | 0.267950 | 5 | 0 | 2 | 3.91 | | test_benchmark_is_null_or_empty[col8] | 0.219256 | 0.217609 | 0.214083 | 0.226223 | 0.005137 | 0.008602 | 0.215124 | 0.223726 | 5 | 0 | 2 | 4.56 | +| test_benchmark_is_ogc_valid | 0.220437 | 0.220308 | 0.215645 | 0.225761 | 0.004216 | 0.007236 | 0.216734 | 0.223970 | 5 | 0 | 2 | 4.54 | | test_benchmark_is_older_than_col2_for_n_days | 0.235241 | 0.230978 | 0.224354 | 0.254865 | 0.011884 | 0.013734 | 0.227788 | 0.241522 | 5 | 0 | 1 | 4.25 | | test_benchmark_is_older_than_n_days | 0.246935 | 0.248889 | 0.234393 | 0.253353 | 0.007733 | 0.010372 | 0.242547 | 0.252920 | 5 | 0 | 1 | 4.05 | +| test_benchmark_is_point | 0.222127 | 0.221192 | 0.216315 | 0.229766 | 0.005090 | 0.006753 | 0.218645 | 0.225398 | 5 | 0 | 2 | 4.50 | +| test_benchmark_is_polygon | 0.224824 | 0.222314 | 0.219383 | 0.238668 | 0.007833 | 0.005581 | 0.220875 | 0.226456 | 5 | 1 | 1 | 4.45 | | test_benchmark_is_unique | 0.248614 | 0.245772 | 0.243054 | 0.259295 | 0.006731 | 0.009422 | 0.243688 | 0.253109 | 5 | 0 | 1 | 4.02 | | test_benchmark_is_valid_date | 0.242393 | 0.229097 | 0.222715 | 0.270948 | 0.022056 | 0.037082 | 0.226619 | 0.263702 | 5 | 0 | 1 | 4.13 | | test_benchmark_is_valid_ipv4_address[col1_ipv4_standard] | 0.165535 | 0.161430 | 0.158404 | 0.188479 | 0.011344 | 0.002309 | 0.160578 | 0.162887 | 6 | 1 | 1 | 6.04 | diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 718f214e9..447bd2891 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -415,14 +415,15 @@ def _observe_metrics(self, df: DataFrame) -> DataFrame | tuple[DataFrame, Observ if not self.observer: return df + metric_exprs = [F.expr(metric_statement) for metric_statement in self.observer.metrics] + if not metric_exprs: + return df + observation = self.observer.observation - return ( - df.observe( - observation, - *[F.expr(metric_statement) for metric_statement in self.observer.metrics], - ), - observation, - ) + if df.isStreaming: + return df.observe(self.observer.name, *metric_exprs), observation + + return df.observe(observation, *metric_exprs), observation class DQEngine(DQEngineBase): @@ -574,7 +575,6 @@ def apply_checks_and_save_in_table( metrics_config: Optional configuration for writing summary metrics. ref_dfs: Optional reference DataFrames used by checks. """ - # Read data from the specified table df = read_input_data(self.spark, input_config) if self._engine.observer and metrics_config and df.isStreaming: @@ -586,19 +586,24 @@ def apply_checks_and_save_in_table( ) self.spark.streams.addListener(listener) + observation = None if quarantine_config: - # Split data into good and bad records - good_df, bad_df, *observations = self.apply_checks_and_split(df, checks, ref_dfs) + split_result = self.apply_checks_and_split(df, checks, ref_dfs) + if self._engine.observer: + good_df, bad_df, observation = split_result + else: + good_df, bad_df = split_result save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: - # Apply checks and write all data to single table - checked_df, *observations = self.apply_checks(df, checks, ref_dfs) + check_result = self.apply_checks(df, checks, ref_dfs) + if self._engine.observer: + checked_df, observation = check_result + else: + checked_df = check_result save_dataframe_as_table(checked_df, output_config) - if self._engine.observer and metrics_config and not df.isStreaming: - # Create DataFrame with observation metrics - keys as column names, values as data - observation = observations[0] + if self._engine.observer and metrics_config and not df.isStreaming and observation is not None: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, @@ -646,7 +651,6 @@ def apply_checks_by_metadata_and_save_in_table( to callables/modules (e.g., globals()). ref_dfs: Optional reference DataFrames used by checks. """ - # Read data from the specified table df = read_input_data(self.spark, input_config) if self._engine.observer and metrics_config and df.isStreaming: @@ -658,21 +662,24 @@ def apply_checks_by_metadata_and_save_in_table( ) self.spark.streams.addListener(listener) + observation = None if quarantine_config: - # Split data into good and bad records - good_df, bad_df, *observations = self.apply_checks_by_metadata_and_split( - df, checks, custom_check_functions, ref_dfs - ) + split_result = self.apply_checks_by_metadata_and_split(df, checks, custom_check_functions, ref_dfs) + if self._engine.observer: + good_df, bad_df, observation = split_result + else: + good_df, bad_df = split_result save_dataframe_as_table(good_df, output_config) save_dataframe_as_table(bad_df, quarantine_config) else: - # Apply checks and write all data to single table - checked_df, *observations = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + check_result = self.apply_checks_by_metadata(df, checks, custom_check_functions, ref_dfs) + if self._engine.observer: + checked_df, observation = check_result + else: + checked_df = check_result save_dataframe_as_table(checked_df, output_config) - if self._engine.observer and metrics_config and not df.isStreaming: - # Create DataFrame with observation metrics - keys as column names, values as data - observation = observations[0] + if self._engine.observer and metrics_config and not df.isStreaming and observation is not None: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, @@ -800,7 +807,7 @@ def save_results_in_table( is_streaming_quarantine = quarantine_df is not None and quarantine_df.isStreaming is_streaming = is_streaming_output or is_streaming_quarantine - if observation is not None and metrics_config is not None and is_streaming: + if self._engine.observer and observation is not None and metrics_config is not None and is_streaming: listener = self._get_streaming_metrics_listener( output_config=output_config, quarantine_config=quarantine_config, @@ -814,7 +821,7 @@ def save_results_in_table( if quarantine_df is not None and quarantine_config is not None: save_dataframe_as_table(quarantine_df, quarantine_config) - if self._engine.observer and observation is not None and metrics_config is not None: + if self._engine.observer and observation is not None and metrics_config is not None and not is_streaming: metrics_observation = DQMetricsObservation( observer_name=self._engine.observer.name, observed_metrics=observation.get, diff --git a/src/databricks/labs/dqx/metrics_observer.py b/src/databricks/labs/dqx/metrics_observer.py index 4674ea55c..a0092228e 100644 --- a/src/databricks/labs/dqx/metrics_observer.py +++ b/src/databricks/labs/dqx/metrics_observer.py @@ -94,7 +94,7 @@ def metrics(self) -> list[str]: default_metrics.extend(self.custom_metrics) return default_metrics - @property + @cached_property def observation(self) -> Observation: """ Spark `Observation` which can be attached to a `DataFrame` to track summary metrics. Metrics will be collected @@ -147,7 +147,7 @@ def build_metrics_df(spark: SparkSession, observation: DQMetricsObservation) -> observation.run_time, observation.error_column_name, observation.warning_column_name, - observation.user_metadata, + observation.user_metadata if observation.user_metadata else None, ] for metric_key, metric_value in observation.observed_metrics.items() ], diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 5f4fd2d81..e820c9769 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -316,3 +316,13 @@ def contains_expected_workflows(workflows, state): if all(item in workflow.items() for item in state.items()): return True return False + + +def validate_metrics(actual_metrics, expected_metrics): + for metric_name in expected_metrics: + actual = actual_metrics[metric_name] + expected = expected_metrics[metric_name] + actual_without_ts = {k: v for k, v in actual.items() if k != "run_ts"} + expected_without_ts = {k: v for k, v in expected.items() if k != "run_ts"} + + assert actual_without_ts == expected_without_ts diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index e9e8cf4b7..5b13060de 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -1,3 +1,6 @@ +from tests.integration.conftest import validate_metrics + + def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metrics): """Test that quality checker workflow saves metrics when configured.""" ctx, run_config = setup_workflows_with_metrics(metrics=True) @@ -6,24 +9,73 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri output_df = spark.table(run_config.output_config.location) output_count = output_df.count() - expected_metrics = { - "input_row_count": output_count, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": output_count, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 4 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workflows_with_metrics): """Test workflow with both quarantine and metrics configurations.""" ctx, run_config = setup_workflows_with_metrics( - quarantine=True, metrics=True, custom_metrics=["count(distinct name) as unique_names"] + quarantine=True, metrics=True, custom_metrics=["count(1) as total_ids"] ) ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) @@ -32,19 +84,80 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf quarantine_df = spark.table(run_config.quarantine_config.location) quarantine_count = quarantine_df.count() - expected_metrics = { - "input_row_count": output_count + quarantine_count, - "error_row_count": quarantine_count, - "warning_row_count": 0, - "valid_row_count": output_count, - "unique_names": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count + quarantine_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": str(quarantine_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": run_config.quarantine_config.location, + "checks_location": None, + "metric_name": "total_ids", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 5 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): @@ -57,20 +170,93 @@ def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): output_df = spark.table(run_config.output_config.location) output_count = output_df.count() - expected_metrics = { - "input_row_count": output_count, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": output_count, - "max_id": 1, - "min_id": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": str(output_count), + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "max_id", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "min_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 6 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): @@ -78,29 +264,127 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): custom_metrics = [ "avg(id) as average_id", "sum(id) as total_id", - "count(distinct name) as unique_names", + "count(1) as total_ids", "max(id) - min(id) as id_range", ] ctx, run_config = setup_workflows_with_metrics(metrics=True, custom_metrics=custom_metrics) - expected_metrics = { - "input_row_count": 0, - "error_row_count": 0, - "warning_row_count": 0, - "valid_row_count": 0, - "average_id": 0, - "total_id": 0, - "unique_names": 0, - "id_range": 0, - } + expected_metrics = [ + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "average_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_id", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_ids", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "dqx", + "input_location": run_config.input_config.location, + "output_location": run_config.output_config.location, + "quarantine_location": None, + "checks_location": None, + "metric_name": "id_range", + "metric_value": "0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) metrics_df = spark.table(run_config.metrics_config.location) metrics_rows = metrics_df.collect() - assert len(metrics_rows) == 1 + assert len(metrics_rows) == 8 - actual_metrics = metrics_rows[0].asDict() - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_quality_checker_workflow_without_metrics_config(setup_workflows_with_metrics): diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 36926618e..9e41f2d9e 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -4,6 +4,7 @@ from databricks.labs.dqx.config import InputConfig, OutputConfig, ExtraParams from databricks.labs.dqx.engine import DQEngine from databricks.labs.dqx.metrics_observer import DQMetricsObserver +from tests.integration.conftest import validate_metrics RUN_TIME = datetime(2025, 1, 1, 0, 0, 0, 0, tzinfo=timezone.utc) @@ -144,17 +145,90 @@ def test_observer_metrics_output(ws, spark, make_schema, make_random): checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + expected_metrics = [ + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random): @@ -197,17 +271,90 @@ def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_ra metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() + expected_metrics = [ + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] - assert actual_metrics == expected_metrics + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_random): @@ -279,16 +426,90 @@ def test_streaming_observer_metrics_output(ws, spark, make_schema, make_random, checks=TEST_CHECKS, input_config=input_config, output_config=output_config, metrics_config=metrics_config ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random, make_volume): @@ -340,16 +561,90 @@ def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, mak metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - "avg_error_age": 35.0, - "total_warning_salary": 55000, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "avg_error_age", + "metric_value": "35.0", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_streaming_quarantine_observer", + "input_location": input_table_name, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "total_warning_salary", + "metric_value": "55000", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_random): @@ -387,14 +682,64 @@ def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_r metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_batch_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": quarantine_table_name, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, make_random, make_volume): @@ -436,11 +781,61 @@ def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, ma metrics_config=metrics_config, ) - expected_metrics = { - "input_row_count": 4, - "error_row_count": 1, - "warning_row_count": 1, - "valid_row_count": 2, - } - actual_metrics = spark.table(metrics_table_name).collect()[0].asDict() - assert actual_metrics == expected_metrics + expected_metrics = [ + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "input_row_count", + "metric_value": "4", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "error_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "warning_row_count", + "metric_value": "1", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + { + "run_name": "test_save_streaming_observer", + "input_location": None, + "output_location": output_table_name, + "quarantine_location": None, + "checks_location": None, + "metric_name": "valid_row_count", + "metric_value": "2", + "run_ts": None, # Will be set at runtime + "error_column_name": "_errors", + "warning_column_name": "_warnings", + "user_metadata": None, + }, + ] + + actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} + expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} + validate_metrics(actual_metrics_dict, expected_metrics_dict) diff --git a/tests/perf/.benchmarks/baseline.json b/tests/perf/.benchmarks/baseline.json index d65319fec..522f8b57f 100644 --- a/tests/perf/.benchmarks/baseline.json +++ b/tests/perf/.benchmarks/baseline.json @@ -1541,6 +1541,41 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_has_dimension", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_dimension", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22927346699998452, + "max": 0.26677058599989323, + "mean": 0.24390853020004216, + "stddev": 0.01575137524903365, + "rounds": 5, + "median": 0.2399963640000351, + "iqr": 0.0254931987500413, + "q1": 0.23057224500007578, + "q3": 0.2560654437501171, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22927346699998452, + "hd15iqr": 0.26677058599989323, + "ops": 4.099897609894364, + "total": 1.2195426510002108, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_has_valid_schema", @@ -1576,6 +1611,76 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_has_x_coordinate_between", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_x_coordinate_between", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22528417800003808, + "max": 0.23497163499996532, + "mean": 0.22921857799992723, + "stddev": 0.004110638636705224, + "rounds": 5, + "median": 0.22888549199979025, + "iqr": 0.0068965210004989785, + "q1": 0.22544056049969186, + "q3": 0.23233708150019083, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22528417800003808, + "hd15iqr": 0.23497163499996532, + "ops": 4.362648127065501, + "total": 1.146092889999636, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_has_y_coordinate_between", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_has_y_coordinate_between", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21785882099993614, + "max": 0.2258314989999235, + "mean": 0.22349742219985275, + "stddev": 0.003231259303985809, + "rounds": 5, + "median": 0.22475687199994354, + "iqr": 0.002968186749740198, + "q1": 0.22236716624991004, + "q3": 0.22533535299965024, + "iqr_outliers": 1, + "stddev_outliers": 1, + "outliers": "1;1", + "ld15iqr": 0.22386994799990134, + "hd15iqr": 0.2258314989999235, + "ops": 4.4743245365299735, + "total": 1.1174871109992637, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_aggr_equal", @@ -1821,6 +1926,111 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_geography", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geography", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21700612699987687, + "max": 0.2347806469997522, + "mean": 0.22657076599998618, + "stddev": 0.008722428619097212, + "rounds": 5, + "median": 0.23090056099999856, + "iqr": 0.0161717707499065, + "q1": 0.2171998970001141, + "q3": 0.2333716677500206, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21700612699987687, + "hd15iqr": 0.2347806469997522, + "ops": 4.413632074669603, + "total": 1.132853829999931, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_geometry", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geometry", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21208457500006261, + "max": 0.22899367299987716, + "mean": 0.21864081899993834, + "stddev": 0.006269493398778712, + "rounds": 5, + "median": 0.21769451999989542, + "iqr": 0.005765286750033738, + "q1": 0.2151638852499218, + "q3": 0.22092917199995554, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21208457500006261, + "hd15iqr": 0.22899367299987716, + "ops": 4.573711370886706, + "total": 1.0932040949996917, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_geometrycollection", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_geometrycollection", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22080926100034048, + "max": 0.2341752649999762, + "mean": 0.22618524760000583, + "stddev": 0.005740098679045008, + "rounds": 5, + "median": 0.22378674699984913, + "iqr": 0.009460075249876354, + "q1": 0.22170114000005015, + "q3": 0.2311612152499265, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.22080926100034048, + "hd15iqr": 0.2341752649999762, + "ops": 4.421154830435432, + "total": 1.1309262380000291, + "iterations": 1 + } + }, { "group": "test_benchmark_is_in_list col1", "name": "test_benchmark_is_in_list[col1]", @@ -2411,6 +2621,251 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_latitude", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_latitude", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22056750799993097, + "max": 0.24297376000004078, + "mean": 0.23234819859999334, + "stddev": 0.01079252112627353, + "rounds": 5, + "median": 0.2352526399999988, + "iqr": 0.02076050825019138, + "q1": 0.22116756124989934, + "q3": 0.24192806950009071, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.22056750799993097, + "hd15iqr": 0.24297376000004078, + "ops": 4.303885315339082, + "total": 1.1617409929999667, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_linestring", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_linestring", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21752809200006595, + "max": 0.23103526600016266, + "mean": 0.22496202360007372, + "stddev": 0.0054244459300489154, + "rounds": 5, + "median": 0.22569596500034095, + "iqr": 0.00866051725017769, + "q1": 0.2206967092498644, + "q3": 0.2293572265000421, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21752809200006595, + "hd15iqr": 0.23103526600016266, + "ops": 4.445194722188978, + "total": 1.1248101180003687, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_longitude", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_longitude", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21846815100025196, + "max": 0.23497388700025112, + "mean": 0.22443222580013752, + "stddev": 0.006264883515721421, + "rounds": 5, + "median": 0.22348119900016172, + "iqr": 0.005906900999775644, + "q1": 0.22069101375018363, + "q3": 0.22659791474995927, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21846815100025196, + "hd15iqr": 0.23497388700025112, + "ops": 4.455688110006648, + "total": 1.1221611290006877, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multilinestring", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multilinestring", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.22188278100020398, + "max": 0.2433739359998981, + "mean": 0.23418097320000014, + "stddev": 0.009128031976731316, + "rounds": 5, + "median": 0.23829027100009625, + "iqr": 0.014867763250094868, + "q1": 0.22598241224989124, + "q3": 0.2408501754999861, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.22188278100020398, + "hd15iqr": 0.2433739359998981, + "ops": 4.270201743272964, + "total": 1.1709048660000008, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multipoint", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multipoint", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.2202482810002948, + "max": 0.2460606760000701, + "mean": 0.23410069620003923, + "stddev": 0.009553865226408068, + "rounds": 5, + "median": 0.235909207000077, + "iqr": 0.011954297000329461, + "q1": 0.22791846499978874, + "q3": 0.2398727620001182, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.2202482810002948, + "hd15iqr": 0.2460606760000701, + "ops": 4.27166606606543, + "total": 1.1705034810001962, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_multipolygon", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_multipolygon", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21334442300030787, + "max": 0.2506819240002187, + "mean": 0.2366548676000093, + "stddev": 0.015311892912790935, + "rounds": 5, + "median": 0.24152346799974111, + "iqr": 0.022927085000105762, + "q1": 0.22568644699993, + "q3": 0.24861353200003578, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21334442300030787, + "hd15iqr": 0.2506819240002187, + "ops": 4.2255627790009616, + "total": 1.1832743380000466, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_non_empty_geometry", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_non_empty_geometry", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21812769000007393, + "max": 0.26618207299998176, + "mean": 0.23537642080000296, + "stddev": 0.01933716738528616, + "rounds": 5, + "median": 0.2271670790000826, + "iqr": 0.025741081250089337, + "q1": 0.222195152999916, + "q3": 0.24793623425000533, + "iqr_outliers": 0, + "stddev_outliers": 1, + "outliers": "1;0", + "ld15iqr": 0.21812769000007393, + "hd15iqr": 0.26618207299998176, + "ops": 4.248513919113802, + "total": 1.1768821040000148, + "iterations": 1 + } + }, { "group": "test_benchmark_is_not_empty col1", "name": "test_benchmark_is_not_empty[col1]", @@ -3655,6 +4110,41 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_ogc_valid", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_ogc_valid", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21564534099979937, + "max": 0.2257612359999257, + "mean": 0.22043705859996407, + "stddev": 0.004216087117037423, + "rounds": 5, + "median": 0.22030801499977315, + "iqr": 0.007236216000251261, + "q1": 0.2167342269999608, + "q3": 0.22397044300021207, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21564534099979937, + "hd15iqr": 0.2257612359999257, + "ops": 4.536442313062886, + "total": 1.1021852929998204, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_older_than_col2_for_n_days", @@ -3725,6 +4215,76 @@ "iterations": 1 } }, + { + "group": null, + "name": "test_benchmark_is_point", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_point", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21631457500006945, + "max": 0.2297655649999797, + "mean": 0.2221274610000364, + "stddev": 0.005089639674878682, + "rounds": 5, + "median": 0.22119198900008996, + "iqr": 0.006752966500016555, + "q1": 0.218645475250014, + "q3": 0.22539844175003054, + "iqr_outliers": 0, + "stddev_outliers": 2, + "outliers": "2;0", + "ld15iqr": 0.21631457500006945, + "hd15iqr": 0.2297655649999797, + "ops": 4.501919733372525, + "total": 1.110637305000182, + "iterations": 1 + } + }, + { + "group": null, + "name": "test_benchmark_is_polygon", + "fullname": "tests/perf/test_apply_checks.py::test_benchmark_is_polygon", + "params": null, + "param": null, + "extra_info": {}, + "options": { + "disable_gc": false, + "timer": "perf_counter", + "min_rounds": 5, + "max_time": 1.0, + "min_time": 0.000005, + "warmup": false + }, + "stats": { + "min": 0.21938274400008595, + "max": 0.23866784099982397, + "mean": 0.22482422739994945, + "stddev": 0.007833050782835247, + "rounds": 5, + "median": 0.22231378099968424, + "iqr": 0.005580997999913961, + "q1": 0.22087461325008917, + "q3": 0.22645561125000313, + "iqr_outliers": 1, + "stddev_outliers": 1, + "outliers": "1;1", + "ld15iqr": 0.21938274400008595, + "hd15iqr": 0.23866784099982397, + "ops": 4.447919210330731, + "total": 1.1241211369997473, + "iterations": 1 + } + }, { "group": null, "name": "test_benchmark_is_unique", From a5c2944e1ad3c7938bbe06b0c0e97eaf15730e93 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Mon, 6 Oct 2025 12:21:14 -0400 Subject: [PATCH 26/27] Update comments and tests --- src/databricks/labs/dqx/config.py | 3 +-- src/databricks/labs/dqx/engine.py | 1 - src/databricks/labs/dqx/metrics_listener.py | 2 +- .../labs/dqx/quality_checker/quality_checker_runner.py | 1 - tests/unit/test_storage_factory.py | 2 +- 5 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/databricks/labs/dqx/config.py b/src/databricks/labs/dqx/config.py index 73e35468d..d59db36a9 100644 --- a/src/databricks/labs/dqx/config.py +++ b/src/databricks/labs/dqx/config.py @@ -111,8 +111,7 @@ class WorkspaceConfig: profiler_max_parallelism: int = 4 # max parallelism for profiling multiple tables quality_checker_max_parallelism: int = 4 # max parallelism for quality checking multiple tables - - custom_metrics: list[str] | None = None + custom_metrics: list[str] | None = None # custom summary metrics tracked by the observer when applying checks def get_run_config(self, run_config_name: str | None = "default") -> RunConfig: """Get the run configuration for a given run name, or the default configuration if no run name is provided. diff --git a/src/databricks/labs/dqx/engine.py b/src/databricks/labs/dqx/engine.py index 98a4c43b3..f6a724758 100644 --- a/src/databricks/labs/dqx/engine.py +++ b/src/databricks/labs/dqx/engine.py @@ -36,7 +36,6 @@ ) from databricks.labs.dqx.checks_validator import ChecksValidator, ChecksValidationStatus from databricks.labs.dqx.schema import dq_result_schema -from databricks.labs.dqx.utils import read_input_data, save_dataframe_as_table from databricks.labs.dqx.metrics_observer import DQMetricsObservation, DQMetricsObserver from databricks.labs.dqx.metrics_listener import StreamingMetricsListener from databricks.labs.dqx.io import read_input_data, save_dataframe_as_table, get_reference_dataframes diff --git a/src/databricks/labs/dqx/metrics_listener.py b/src/databricks/labs/dqx/metrics_listener.py index 8794587ce..fbddc1e9b 100644 --- a/src/databricks/labs/dqx/metrics_listener.py +++ b/src/databricks/labs/dqx/metrics_listener.py @@ -4,7 +4,7 @@ from pyspark.sql.streaming import listener from databricks.labs.dqx.config import OutputConfig from databricks.labs.dqx.metrics_observer import DQMetricsObservation, DQMetricsObserver -from databricks.labs.dqx.utils import save_dataframe_as_table +from databricks.labs.dqx.io import save_dataframe_as_table logger = logging.getLogger(__name__) diff --git a/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py b/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py index aa7d2a085..fc85e89c1 100644 --- a/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py +++ b/src/databricks/labs/dqx/quality_checker/quality_checker_runner.py @@ -16,7 +16,6 @@ def __init__(self, spark: SparkSession, dq_engine: DQEngine): self.spark = spark self.dq_engine = dq_engine - def run(self, run_configs: list[RunConfig], max_parallelism: int | None = os.cpu_count()) -> None: """ Run the DQX data quality job for the provided run configs. diff --git a/tests/unit/test_storage_factory.py b/tests/unit/test_storage_factory.py index 2f161700a..2652f5d56 100644 --- a/tests/unit/test_storage_factory.py +++ b/tests/unit/test_storage_factory.py @@ -56,7 +56,7 @@ def test_create_unsupported_config(): class UnsupportedConfig(BaseChecksStorageConfig): pass - config = UnsupportedConfig() + config = UnsupportedConfig(location="INVALID") with pytest.raises(InvalidConfigError, match="Unsupported storage config type"): STORAGE_FACTORY.create(config) From d402446d8954473b230d0afc21612a17b6e86854 Mon Sep 17 00:00:00 2001 From: Greg Hansen Date: Mon, 6 Oct 2025 18:23:26 -0400 Subject: [PATCH 27/27] Update tests --- tests/integration/conftest.py | 10 ------ tests/integration/test_metrics_workflow.py | 39 ++++++++------------- tests/integration/test_summary_metrics.py | 40 +++++++++++----------- 3 files changed, 34 insertions(+), 55 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 4082801b8..8fce9081c 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -320,13 +320,3 @@ def contains_expected_workflows(workflows, state): if all(item in workflow.items() for item in state.items()): return True return False - - -def validate_metrics(actual_metrics, expected_metrics): - for metric_name in expected_metrics: - actual = actual_metrics[metric_name] - expected = expected_metrics[metric_name] - actual_without_ts = {k: v for k, v in actual.items() if k != "run_ts"} - expected_without_ts = {k: v for k, v in expected.items() if k != "run_ts"} - - assert actual_without_ts == expected_without_ts diff --git a/tests/integration/test_metrics_workflow.py b/tests/integration/test_metrics_workflow.py index 5b13060de..941c9e2dc 100644 --- a/tests/integration/test_metrics_workflow.py +++ b/tests/integration/test_metrics_workflow.py @@ -1,4 +1,5 @@ -from tests.integration.conftest import validate_metrics +from databricks.labs.dqx.metrics_observer import OBSERVATION_TABLE_SCHEMA +from chispa import assert_df_equality def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metrics): @@ -64,12 +65,9 @@ def test_quality_checker_workflow_with_metrics(spark, setup_workflows_with_metri }, ] - metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 4 - - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(run_config.metrics_config.location).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workflows_with_metrics): @@ -152,12 +150,9 @@ def test_quality_checker_workflow_with_quarantine_and_metrics(spark, setup_workf }, ] - metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 5 - - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(run_config.metrics_config.location).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): @@ -251,12 +246,9 @@ def test_e2e_workflow_with_metrics(spark, setup_workflows_with_metrics): }, ] - metrics_rows = spark.table(run_config.metrics_config.location).collect() - assert len(metrics_rows) == 6 - - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(run_config.metrics_config.location).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): @@ -378,13 +370,10 @@ def test_custom_metrics_in_workflow(spark, setup_workflows_with_metrics): ] ctx.deployed_workflows.run_workflow("quality-checker", run_config.name) - metrics_df = spark.table(run_config.metrics_config.location) - metrics_rows = metrics_df.collect() - assert len(metrics_rows) == 8 - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in metrics_rows} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(run_config.metrics_config.location).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_quality_checker_workflow_without_metrics_config(setup_workflows_with_metrics): diff --git a/tests/integration/test_summary_metrics.py b/tests/integration/test_summary_metrics.py index 9e41f2d9e..7f70693da 100644 --- a/tests/integration/test_summary_metrics.py +++ b/tests/integration/test_summary_metrics.py @@ -3,8 +3,8 @@ from databricks.labs.dqx.config import InputConfig, OutputConfig, ExtraParams from databricks.labs.dqx.engine import DQEngine -from databricks.labs.dqx.metrics_observer import DQMetricsObserver -from tests.integration.conftest import validate_metrics +from databricks.labs.dqx.metrics_observer import DQMetricsObserver, OBSERVATION_TABLE_SCHEMA +from chispa import assert_df_equality RUN_TIME = datetime(2025, 1, 1, 0, 0, 0, 0, tzinfo=timezone.utc) @@ -226,9 +226,9 @@ def test_observer_metrics_output(ws, spark, make_schema, make_random): }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random): @@ -352,9 +352,9 @@ def test_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_ra }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_engine_without_observer_no_metrics_saved(ws, spark, make_schema, make_random): @@ -507,9 +507,9 @@ def test_streaming_observer_metrics_output(ws, spark, make_schema, make_random, }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, make_schema, make_random, make_volume): @@ -642,9 +642,9 @@ def test_engine_streaming_observer_metrics_output_with_quarantine(ws, spark, mak }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_random): @@ -737,9 +737,9 @@ def test_save_results_in_table_batch_with_metrics(ws, spark, make_schema, make_r }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df) def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, make_random, make_volume): @@ -836,6 +836,6 @@ def test_save_results_in_table_streaming_with_metrics(ws, spark, make_schema, ma }, ] - actual_metrics_dict = {row["metric_name"]: row.asDict() for row in spark.table(metrics_table_name).collect()} - expected_metrics_dict = {metric["metric_name"]: metric for metric in expected_metrics} - validate_metrics(actual_metrics_dict, expected_metrics_dict) + expected_metrics_df = spark.createDataFrame(expected_metrics, schema=OBSERVATION_TABLE_SCHEMA).drop("run_ts") + actual_metrics_df = spark.table(metrics_table_name).drop("run_ts") + assert_df_equality(expected_metrics_df, actual_metrics_df)