Skip to main content

Overview

Materialized views in Materialize are views whose results are persisted in durable storage and incrementally updated as new data arrives. Unlike traditional databases that recompute materialized views on a schedule, Materialize maintains them continuously using differential dataflow.
CREATE MATERIALIZED VIEW sales_summary AS
SELECT 
    product_id,
    COUNT(*) as order_count,
    SUM(quantity) as total_quantity,
    SUM(amount) as total_revenue
FROM orders
GROUP BY product_id;
Materialized views can be queried from any cluster, making them ideal for cross-cluster data sharing and multi-tier architectures.

Materialized Views vs Regular Views

Regular Views

Computation: Recomputed from scratch on every queryStorage: No persistent storageUpdates: N/A (computed on demand)Clusters: Not associated with a clusterBest for: Simple queries, rarely accessed data

Materialized Views

Computation: Incrementally maintainedStorage: Results in durable storageUpdates: Continuous, as data arrivesClusters: Associated with a compute clusterBest for: Complex queries, frequent access, cross-cluster sharing

How Incremental Maintenance Works

Materialize uses differential dataflow to maintain views incrementally. Instead of recomputing entire results, only changes are processed:

Example: Order Aggregation

CREATE MATERIALIZED VIEW customer_orders AS
SELECT 
    customer_id,
    COUNT(*) as order_count,
    SUM(total) as lifetime_value
FROM orders
GROUP BY customer_id;
When a new order arrives:
INSERT INTO orders VALUES (1001, 'CUST-123', 49.99);
Materialize processes:
(data={customer_id: 'CUST-123', order_count: +1, lifetime_value: +49.99}, time=T, diff=+1)
Only the affected customer’s aggregates are updated — not the entire view.

Delta Joins

For joins, Materialize computes delta joins that avoid intermediate state blowup:
CREATE MATERIALIZED VIEW enriched_orders AS
SELECT 
    o.order_id,
    c.customer_name,
    p.product_name,
    o.quantity,
    o.amount
FROM orders o
JOIN customers c ON o.customer_id = c.id
JOIN products p ON o.product_id = p.id;
When a new order arrives, Materialize:
  1. Looks up the customer in an arrangement (indexed state)
  2. Looks up the product in another arrangement
  3. Emits only the new joined record
Delta joins can handle joins of up to 64 relations efficiently, something impossible with traditional nested loop approaches.

Arrangements Under the Hood

Materialize builds arrangements — indexed, multi-versioned representations of data:
// Conceptual structure
Arrangement<K, V, T, D> = Index<K, Vec<(V, T, D)>>
  • K: Key (columns used for lookup)
  • V: Value (remaining columns)
  • T: Timestamp
  • D: Diff (+1 or -1)
Arrangements enable:
  • Fast point lookups by key
  • Historical queries at any timestamp
  • Shared state across operators

Example: Join Operator

In1 --> Arrangement(key1)  ───┐
                               ├──> Join
In2 --> Arrangement(key2)  ───┘
Each input must be arranged by the join key. Arrangements can be shared if multiple operators need the same data organized the same way.
Arrangements consume memory proportional to the number of distinct (key, time) pairs. Monitor arrangement sizes using mz_arrangement_sizes.

Dataflow Execution

Materialized views compile to dataflow graphs executed by the compute layer:
CREATE MATERIALIZED VIEW daily_revenue AS
SELECT 
    DATE_TRUNC('day', order_time) as day,
    SUM(amount) as revenue
FROM orders
WHERE status = 'completed'
GROUP BY DATE_TRUNC('day', order_time);
Compiles to:
Source(orders)
  |
  ├─> Filter(status = 'completed')
  |     |
  |     ├─> Map(day = DATE_TRUNC('day', order_time))
  |           |
  |           ├─> Reduce(GROUP BY day, SUM(amount))
  |                 |
  |                 └─> Materialize (persist to storage)
Each operator:
  • Processes updates incrementally
  • Maintains arrangements as needed
  • Propagates only changes downstream

Hydration

When a materialized view is created or its cluster restarts, it undergoes hydration — reconstructing in-memory state from storage:
1

Read from Storage

Load persisted data from Materialize’s storage layer (no upstream re-ingestion)
2

Rebuild Arrangements

Reconstruct indexed state for operators that need it
3

Resume Updates

Begin processing new changes incrementally
During hydration, memory usage is proportional to both input and output sizes. Consider cluster sizing when creating large materialized views.

Refresh Strategies

By default, materialized views use REFRESH ON COMMIT (incremental updates). For specific use cases, alternative strategies are available:

Refresh on Commit (Default)

CREATE MATERIALIZED VIEW mv_name AS
SELECT * FROM source;
-- Implicit: REFRESH ON COMMIT
Updates continuously as data arrives. Recommended for 99% of use cases.

Refresh at Creation

CREATE MATERIALIZED VIEW snapshot_view
WITH (REFRESH AT CREATION) AS
SELECT * FROM historical_data;
Computes once at creation, then never updates. Useful for static snapshots.

Refresh at Specific Time

CREATE MATERIALIZED VIEW monthly_report
WITH (REFRESH AT '2024-01-01 00:00:00') AS
SELECT 
    DATE_TRUNC('month', order_time) as month,
    SUM(amount) as revenue
FROM orders
GROUP BY month;
Recomputes at the specified timestamp.

Refresh Every Interval

CREATE MATERIALIZED VIEW hourly_summary
WITH (REFRESH EVERY '1 hour') AS
SELECT 
    DATE_TRUNC('hour', event_time) as hour,
    COUNT(*) as event_count
FROM events
GROUP BY hour;
Non-incremental refresh strategies recompute the entire view on each refresh. Use sparingly for cold/archival data where staleness is acceptable.

Indexing Materialized Views

Query performance improves by indexing materialized views:
CREATE MATERIALIZED VIEW customer_summary AS
SELECT 
    customer_id,
    COUNT(*) as order_count,
    SUM(total) as lifetime_value
FROM orders
GROUP BY customer_id;

-- Create an index in the serving cluster
CREATE INDEX idx_customer IN CLUSTER serving_cluster
ON customer_summary (customer_id);
Without index: Query reads from storage (still fast, sub-second) With index: Query reads from memory (microseconds, especially for point lookups)
Indexes are local to a cluster. Create indexes in each cluster that queries the materialized view frequently.

Stacking Materialized Views

Build complex pipelines by stacking materialized views:
-- Stage 1: Filter and clean
CREATE MATERIALIZED VIEW clean_orders AS
SELECT 
    order_id,
    customer_id,
    order_time,
    amount
FROM raw_orders
WHERE amount > 0 AND customer_id IS NOT NULL;

-- Stage 2: Aggregate by customer
CREATE MATERIALIZED VIEW customer_totals AS
SELECT 
    customer_id,
    COUNT(*) as order_count,
    SUM(amount) as total_spent
FROM clean_orders
GROUP BY customer_id;

-- Stage 3: Classify customers
CREATE MATERIALIZED VIEW customer_segments AS
SELECT 
    customer_id,
    CASE 
        WHEN total_spent > 10000 THEN 'VIP'
        WHEN total_spent > 1000 THEN 'Premium'
        ELSE 'Standard'
    END as segment
FROM customer_totals;
Materialize optimizes stacked views by:
  • Sharing arrangements between views when possible
  • Only storing differences at each stage
  • Propagating changes incrementally through the pipeline

Transactional Consistency

Materialize maintains transactional consistency even across multiple sources:
-- PostgreSQL source
CREATE SOURCE pg_orders FROM POSTGRES
CONNECTION pg_conn (PUBLICATION 'mz_source');

-- Kafka source
CREATE SOURCE kafka_events FROM KAFKA
CONNECTION kafka_conn (TOPIC 'events')
FORMAT JSON;

-- Join across sources
CREATE MATERIALIZED VIEW event_enrichment AS
SELECT 
    e.event_id,
    e.timestamp,
    o.order_id,
    o.amount
FROM kafka_events e
JOIN pg_orders_orders o 
  ON e.order_id = o.order_id;
Guarantee: Every query result reflects a consistent snapshot where:
  • All changes from completed PostgreSQL transactions are visible
  • All Kafka messages up to a specific offset are included
  • No partial or inconsistent data is ever returned
Materialize assigns timestamps to ensure linearizability — the strongest consistency guarantee possible.

Performance Optimization

Query Optimization

Materialize’s optimizer performs:
  1. Predicate pushdown: Filters applied as early as possible
  2. Join reordering: Optimal join order selected automatically
  3. Decorrelation: Subqueries converted to efficient joins
  4. Aggregate fusion: Multiple aggregations combined when possible
-- Before optimization
CREATE MATERIALIZED VIEW example AS
SELECT *
FROM (
    SELECT customer_id, SUM(amount) as total
    FROM orders
    GROUP BY customer_id
) subq
WHERE total > 1000;

-- After optimization (conceptual)
CREATE MATERIALIZED VIEW example AS
SELECT customer_id, SUM(amount) as total
FROM orders
GROUP BY customer_id
HAVING SUM(amount) > 1000;  -- Filter pushed into aggregation

Monitoring

-- View materialized view size
SELECT 
    mz_views.name,
    pg_size_pretty(SUM(mz_arrangement_sizes.size)) as total_size
FROM mz_materialized_views
JOIN mz_views ON mz_materialized_views.id = mz_views.id
JOIN mz_arrangement_sizes ON mz_views.id = mz_arrangement_sizes.object_id
GROUP BY mz_views.name;

-- Check hydration status
SELECT name, status FROM mz_internal.mz_materialized_view_refreshes;

-- Monitor freshness
SELECT object_id, lag
FROM mz_internal.mz_wallclock_global_lag
WHERE object_id IN (SELECT id FROM mz_materialized_views);

Best Practices

Create separate clusters for:
  • Sources (data ingestion)
  • Transforms (materialized views)
  • Serving (indexed views for queries)
This provides workload isolation and independent scaling.
When multiple clusters need the same data:
  • Create a materialized view in a transform cluster
  • Index the view in each serving cluster
  • Avoid duplicating expensive computations
During creation or restart, materialized views need memory for both input and output. Size clusters accordingly.
Large arrangements indicate potential optimization opportunities:
  • Add filters earlier in the pipeline
  • Reduce cardinality before joins
  • Consider breaking into smaller views

Example: Real-Time Analytics

Build a complete real-time analytics pipeline:
-- Ingest orders from PostgreSQL
CREATE SOURCE pg_source
FROM POSTGRES CONNECTION pg_conn
(PUBLICATION 'mz_source');

-- Ingest events from Kafka
CREATE SOURCE kafka_events
FROM KAFKA CONNECTION kafka_conn (TOPIC 'events')
FORMAT JSON;

-- Transform: Enrich orders with customer data
CREATE MATERIALIZED VIEW enriched_orders AS
SELECT 
    o.order_id,
    o.order_time,
    c.customer_name,
    c.segment,
    o.amount
FROM pg_source_orders o
JOIN pg_source_customers c ON o.customer_id = c.id;

-- Transform: Real-time aggregations
CREATE MATERIALIZED VIEW realtime_metrics AS
SELECT 
    DATE_TRUNC('minute', order_time) as minute,
    segment,
    COUNT(*) as order_count,
    SUM(amount) as revenue
FROM enriched_orders
GROUP BY minute, segment;

-- Index for fast dashboard queries
CREATE INDEX idx_metrics IN CLUSTER serving
ON realtime_metrics (minute, segment);

-- Sink to Kafka for downstream consumers
CREATE SINK metrics_sink
FROM realtime_metrics
INTO KAFKA CONNECTION kafka_conn (TOPIC 'metrics')
FORMAT JSON;

Next Steps

Create Indexes

Accelerate queries with in-memory indexes

Configure Clusters

Optimize compute resources and isolation

SQL Reference

Complete CREATE MATERIALIZED VIEW syntax

Optimization Guide

Advanced performance tuning techniques

Build docs developers (and LLMs) love