This post gives you a framework for enforcing data quality at every stage so you catch issues early, maintain trust, and build platforms that actually work in production.
The Quality Problem
You've built the pipeline. The data flows. The dashboards update. Everything works. Until it doesn't.
A partner changes their API schema without notice. A database migration introduces null values where there shouldn't be any. A transformation bug duplicates records. Suddenly, your "revenue increased 500%" dashboard will disappoint everyone when they find out it's broken data.
If you don't have proactive data quality the bad data will propagate through your entire data platform and you have a real mess on your hands both technically and organizationally as stakeholders are flying blind.
Most teams treat data quality as an afterthought, adding checks reactively after issues surface. This leads to downstream corruption, loss of trust, and expensive cleanup efforts.
This post gives you a framework for enforcing data quality at every stage so you catch issues early, maintain trust, and build platforms that actually work in production.
Why Data Quality Matters
The Cost of Poor Quality
Data quality refers to the ability of data to serve its intended purpose in a given context. It's not a single step or technology, but a framework that permeates the entire data lifecycle.
When data quality fails, the costs are real:
- Operational disruption: Bad data breaks downstream pipelines and dashboards
- Lost trust: Stakeholders stop relying on your data
- Regulatory risk: Incomplete or inaccurate data can violate compliance requirements
- Wasted time: Teams spend hours debugging instead of building
Consider a retail company relying on inventory data to forecast demand. If products are missing from the system (completeness), quantities are incorrect (accuracy), or timestamps are out of date (timeliness), business decisions become flawed, leading to overstock or missed sales.
The Six Dimensions of Data Quality
Data quality is typically defined according to six core dimensions:
Timeliness: How current and updated the data is. A financial institution expecting daily transaction reports may face regulatory risk if yesterday's data does not arrive on schedule.
Completeness: Ensuring that required data fields are present. In a customer table, if the 'email' column is expected to always be populated but half the entries are blank, marketing campaigns may fail to reach their targets.
Accuracy: Whether data represents reality. Consider a shipping company recording package weights; if the data indicates a negative weight or an impossible route, downstream cost calculations will be incorrect.
Validity: Ensuring that data follows defined rules or formats. For a bank, an 'account_type' field should only contain values like 'checking' or 'savings', not arbitrary entries.
Uniqueness: Guaranteeing that data expected to be singular is not duplicated. Duplicate invoice numbers in an accounting system can cause billing errors and customer complaints.
Consistency: Ensuring data does not conflict across sources or over time. If customer addresses differ between the CRM and the Support Ticketing System without cause, fulfillment and service quality may suffer.
These dimensions provide the foundation for data quality standards and inform how enforcement mechanisms are designed across the platform.
Why Enforcement Matters
Enforcing data quality is not restricted to a single "gate" in the data pipeline. Rather, it is an ongoing process that must be embedded at every stage, from the initial creation of data in source systems to its ultimate presentation to end-users and applications.
Without enforcement, quality issues compound:
- Bad data at ingestion propagates through transformations
- Missing validation at transformation creates downstream errors
- Lack of checks at consumption leads to incorrect business decisions
The earlier you catch quality issues, the cheaper they are to fix. A validation error at ingestion is a simple fix. A corrupted dashboard that's been running for weeks requires data cleanup, pipeline fixes, and stakeholder communication.
The Framework/Solution: Enforcing Quality at Every Stage
Stage 1: At the Application Layer
Quality enforcement begins where data originates. For example, a SaaS provider collecting email addresses during user sign-up can enforce basic client-side and server-side validation to ensure only correctly formed email addresses are accepted at the outset.
When this step is skipped, invalid data propagates downstream, where data warehouses may have to either reject, cleanse, or work around bad records. Fixing issues earlier is less expensive and more robust than correcting them late in the process.
What to enforce:
- Format validation (email addresses, phone numbers, dates)
- Required field checks
- Value range validation
- Business rule validation
How to implement:
- Client-side validation for immediate feedback
- Server-side validation as the source of truth
- API validation for external data sources
- Schema validation for structured data
Stage 2: During Data Ingestion and Replication
As data moves from operational systems into analytical platforms, quality checks should validate that incoming records still meet standards for completeness and validity.
Consider a scenario where a scheduled replication from a production database skips records due to a failed transform. If not detected, this leads to incomplete analytic datasets.
A well-designed ingestion pipeline would verify that the number of records imported matches expectations and that unique identifiers remain unique. Where discrepancies are found, alerts can be raised, and the process can be halted before errors compound downstream.
What to enforce:
- Record count validation
- Schema validation
- Uniqueness checks
- Completeness checks
- Data type validation
How to implement:
- Assets Checks in Dagster pipelines
- Great Expectations or Soda checks
- Custom validation logic
- Automated alerting on failures
Stage 3: In the Transformation and Modeling Layer
Raw data is commonly transformed, aggregated, or joined with additional sources before being made available for analysis. This stage is susceptible to issues such as unintended data loss, misapplied logic, or illegal joins.
Consider a data pipeline that combines a 'users' table with transactional event logs. If either source is missing records (completeness) or key fields are inconsistent (consistency), the resulting model may double-count users or assign transactions to the wrong party.
Transformation logic should therefore include checks for null values, reference integrity, and expected value ranges.
What to enforce:
- Referential integrity
- Aggregation accuracy
- Data loss detection
- Business logic validation
- Statistical checks (outliers, distributions)
How to implement:
- dbt tests for transformation logic
- Custom asset checks in transformation assets
- Comparison with known benchmarks
- Row count and value range checks
Stage 4: At the Consumption and Reporting Stage
Before data is surfaced to business stakeholders or reported externally, additional validations should be carried out. For example, if a sales dashboard receives data from multiple sources, final row counts, derived metrics, and aggregated figures should be compared against known benchmarks.
If a projected sales figure suddenly drops to zero without explanation, prompt detection and investigation can prevent significant business misunderstanding or misreporting.
What to enforce:
- Metric reasonableness checks
- Comparison with historical values
- Cross-source consistency
- Final data completeness
- Business rule validation
How to implement:
- Dashboard-level validation
- Automated data quality reports
- Alerting on anomalies
- Manual review processes for critical metrics
Practical Implementation: Building Quality Checks with Dagster
I created this demo project that covers the various ways to implement Data quality in Dagster.
Asset checks
Dagster's Software-Defined Assets make it easy to add asset checks as first-class citizens in your pipeline. Validation assets can check data quality and halt downstream processing if issues are detected. They are super flexible and you can define quality checks that don't halt execution but surface issues for investigation.
@dg.asset_check(asset=raw_customers, name="check_accuracy_names")
def check_accuracy_names(
context: dg.AssetCheckExecutionContext,
raw_customers: pd.DataFrame,
) -> dg.AssetCheckResult:
"""Check that customer names appear to be real names (accuracy).
Accuracy issues detected:
- Placeholder names like "TEST USER", "N/A", "Unknown"
- Generic names that may be fake data
- Invalid patterns
"""
suspicious_patterns = ["TEST", "XXXX", "N/A", "Unknown", "Placeholder"]
suspicious_names = raw_customers["name"].apply(
lambda x: any(pattern in str(x).upper() for pattern in suspicious_patterns)
if pd.notna(x)
else False
)
suspicious_count = suspicious_names.sum()
total = len(raw_customers)
if suspicious_count > 0:
rate = suspicious_count / total
return dg.AssetCheckResult(
passed=False,
severity=dg.AssetCheckSeverity.WARN,
description=f"Found {suspicious_count} records with suspicious names ({rate:.1%})",
metadata={
"suspicious_count": int(suspicious_count),
"total_records": total,
"examples": raw_customers[suspicious_names]["name"].head(5).tolist(),
},
)
return dg.AssetCheckResult(
passed=True,
description="All customer names appear valid",
metadata={"total_records": total},
)Integration with Quality Frameworks
You can integrate existing data quality frameworks like Great Expectations into your Dagster pipelines.
import dagster as dg
import pandas as pd
from great_expectations.dataset import PandasDataset
from data_quality_patterns.defs.assets.raw_data import raw_products
@dg.asset_check(asset=raw_products, name="ge_check_sku_unique")
def ge_check_sku_unique(
context: dg.AssetCheckExecutionContext,
raw_products: pd.DataFrame,
) -> dg.AssetCheckResult:
"""Great Expectations check: SKU values are unique."""
ge_df = PandasDataset(raw_products)
result = ge_df.expect_column_values_to_be_unique("sku")
if result["success"]:
return dg.AssetCheckResult(
passed=True,
description="All SKU values are unique",
)
return dg.AssetCheckResult(
passed=False,
severity=dg.AssetCheckSeverity.ERROR,
description="Duplicate SKU values found",
metadata={
"unexpected_count": result["result"].get("unexpected_count", 0),
"unexpected_percent": result["result"].get("unexpected_percent", 0),
},
)Freshness Checks
Freshness checks are a way to manage the timeliness aspect of data quality. Making it easy to see which assets are in need of an update to either kick of a materialization or send you an alert.
from datetime import timedelta
import dagster as dg
daily_freshness_policy = dg.FreshnessPolicy.time_window(
fail_window=timedelta(hours=24),
warn_window=timedelta(hours=12),
)
hourly_freshness_policy = dg.FreshnessPolicy.time_window(
fail_window=timedelta(hours=2),
warn_window=timedelta(hours=1),
)dbt Tests
dbt is one of our most popular integrations and they have a great way to add simple tests to your dbt models as part of the Dagster integration with dbt, tests will show as asset checks within Dagster.
version: 2
models:
- name: cleaned_customers
description: Cleaned customer data with quality issues filtered out
config:
tags: ["marts"]
columns:
- name: customer_id
description: Unique customer identifier
data_tests:
- unique:
config:
severity: warn
- not_null:
config:
severity: warn
- name: email
data_tests:
- not_null:
config:
severity: warn
- name: region
data_tests:
- accepted_values:
arguments:
values: ['US', 'EU', 'APAC', 'LATAM']
config:
severity: warn
- name: age
data_tests:
- not_null:
config:
severity: warnBest Practices and Common Pitfalls
Start Early, Enforce Consistently
Don't wait for quality issues to surface. Implement validation from day one, and enforce it consistently across all pipelines.
How to implement:
- Add validation to every new pipeline
- Use standardized validation patterns
- Document quality requirements
- Review quality metrics regularly
Common pitfall: Adding quality checks reactively after issues occur. It's much harder to retrofit quality than to build it in from the start.
Use the Right Tool for the Job
Different quality frameworks serve different purposes. Choose based on your needs.
Great Expectations: Best for comprehensive validation suites and data profiling
Soda: Good for simple, declarative quality checks
dbt tests: Ideal for transformation logic validation
Custom validation: Use for business-specific rules
Common pitfall: Using one tool for everything. Different stages need different approaches.
Balance Strictness with Practicality
Too strict, and you create operational friction. Too lenient, and bad data gets through.
How to balance:
- Start with critical fields and expand
- Use severity levels (ERROR vs WARNING)
- Allow overrides for known exceptions
- Review and adjust thresholds regularly
Common pitfall: Setting thresholds too strict initially. Start conservative and tighten as you learn.
Make Quality Visible
You can't improve what you can't see. Surface quality metrics, trends, and issues prominently.
How to implement:
- Dashboard for quality metrics
- Alerts on quality degradation
- Regular quality reports
- Quality scores in data catalog
Common pitfall: Hiding quality issues. Make them visible so teams can act.
Decision Framework: When to Enforce Quality
At Ingestion: Always
Always validate at ingestion. This is your first line of defense.
What to check:
- Schema validation
- Required fields
- Data types
- Basic format validation
Why: Cheapest place to catch issues. Prevents bad data from entering the system.
At Transformation: Usually
Validate transformations to catch logic errors and data loss.
What to check:
- Row count changes (with expected ranges)
- Referential integrity
- Aggregation accuracy
- Business rule validation
Why: Catches transformation bugs before they propagate.
At Consumption: For Critical Metrics
Validate final outputs before they reach stakeholders.
What to check:
- Metric reasonableness
- Comparison with historical values
- Cross-source consistency
- Final completeness
Why: Last chance to catch issues before business impact.
Quality Gates: For High-Risk Data
Use quality gates for data that affects critical business decisions or compliance.
When to use:
- Financial reporting data
- Regulatory compliance data
- Customer-facing metrics
- High-stakes business decisions
Why: Prevents bad data from reaching critical systems.
Closing: Building Trust Through Quality
Data quality is the foundation of trust. When stakeholders can't rely on your data, everything else becomes irrelevant. Enforcing quality at every stage is a key aspect of data engineering.
Start with validation at ingestion. Add checks at transformation. Validate critical outputs. Most importantly, make quality visible and actionable.
Three things to do this week:
- Add validation to one pipeline: Pick a critical pipeline and add comprehensive quality checks. Use it as a template for others.
- Set up quality monitoring: Create a dashboard or report showing quality metrics. Make it visible to your team.
- Document quality requirements: Write down what "good" data means for your use cases. Use it to guide validation logic.
Quality enforcement is an ongoing process, not a one-time effort. Start simple, iterate, and improve. Your stakeholders and your future self will thank you.




