Merge 'Support UNION queries in DBSP-based Materialized Views' from Glauber Costa

UNION queries, while useful on their own, are a cornerstone of recursive
CTEs.
This PR implements:
* the merge operator, required to merge both sides of a union query.
* the circuitry necessary to issue the Merge operator.
* extraction of tables mentioned in union and CTE expressions, so we can
correctly populate tables that contain them.

Closes #3234
This commit is contained in:
Pekka Enberg
2025-09-22 11:33:19 +03:00
committed by GitHub
6 changed files with 2222 additions and 525 deletions

View File

@@ -298,6 +298,8 @@ pub enum DbspOperator {
},
/// Input operator - source of data
Input { name: String, schema: SchemaRef },
/// Merge operator for combining streams (used in recursive CTEs and UNION)
Merge { schema: SchemaRef },
}
/// Represents an expression in DBSP
@@ -807,6 +809,13 @@ impl DbspCircuit {
DbspOperator::Input { name, .. } => {
writeln!(f, "{indent}Input[{node_id}]: {name}")?;
}
DbspOperator::Merge { schema } => {
writeln!(
f,
"{indent}Merge[{node_id}]: UNION/Recursive (schema: {} columns)",
schema.columns.len()
)?;
}
}
for input_id in &node.inputs {
@@ -1300,8 +1309,12 @@ impl DbspCompiler {
);
Ok(node_id)
}
LogicalPlan::Union(union) => {
// Handle UNION and UNION ALL
self.compile_union(union)
}
_ => Err(LimboError::ParseError(
format!("Unsupported operator in DBSP compiler: only Filter, Projection, Join and Aggregate are supported, got: {:?}",
format!("Unsupported operator in DBSP compiler: only Filter, Projection, Join, Aggregate, and Union are supported, got: {:?}",
match plan {
LogicalPlan::Sort(_) => "Sort",
LogicalPlan::Limit(_) => "Limit",
@@ -1318,6 +1331,116 @@ impl DbspCompiler {
}
}
/// Extract a representative table name from a logical plan (for UNION ALL identification)
/// Returns a string that uniquely identifies the source of the data
fn extract_source_identifier(plan: &LogicalPlan) -> String {
match plan {
LogicalPlan::TableScan(scan) => {
// Direct table scan - use the table name
scan.table_name.clone()
}
LogicalPlan::Projection(proj) => {
// Pass through to input
Self::extract_source_identifier(&proj.input)
}
LogicalPlan::Filter(filter) => {
// Pass through to input
Self::extract_source_identifier(&filter.input)
}
LogicalPlan::Aggregate(agg) => {
// Aggregate of a table
format!("agg_{}", Self::extract_source_identifier(&agg.input))
}
LogicalPlan::Sort(sort) => {
// Pass through to input
Self::extract_source_identifier(&sort.input)
}
LogicalPlan::Limit(limit) => {
// Pass through to input
Self::extract_source_identifier(&limit.input)
}
LogicalPlan::Join(join) => {
// Join of two sources - combine their identifiers
let left_id = Self::extract_source_identifier(&join.left);
let right_id = Self::extract_source_identifier(&join.right);
format!("join_{left_id}_{right_id}")
}
LogicalPlan::Union(union) => {
// Union of multiple sources
if union.inputs.is_empty() {
"union_empty".to_string()
} else {
let identifiers: Vec<String> = union
.inputs
.iter()
.map(|input| Self::extract_source_identifier(input))
.collect();
format!("union_{}", identifiers.join("_"))
}
}
LogicalPlan::Distinct(distinct) => {
// Distinct of a source
format!(
"distinct_{}",
Self::extract_source_identifier(&distinct.input)
)
}
LogicalPlan::WithCTE(with_cte) => {
// CTE body
Self::extract_source_identifier(&with_cte.body)
}
LogicalPlan::CTERef(cte_ref) => {
// CTE reference - use the CTE name
format!("cte_{}", cte_ref.name)
}
LogicalPlan::EmptyRelation(_) => "empty".to_string(),
LogicalPlan::Values(_) => "values".to_string(),
}
}
/// Compile a UNION operator
fn compile_union(&mut self, union: &crate::translate::logical::Union) -> Result<usize> {
if union.inputs.len() != 2 {
return Err(LimboError::ParseError(format!(
"UNION requires exactly 2 inputs, got {}",
union.inputs.len()
)));
}
// Extract source identifiers from each input (for UNION ALL)
let left_source = Self::extract_source_identifier(&union.inputs[0]);
let right_source = Self::extract_source_identifier(&union.inputs[1]);
// Compile left and right inputs
let left_id = self.compile_plan(&union.inputs[0])?;
let right_id = self.compile_plan(&union.inputs[1])?;
use crate::incremental::merge_operator::{MergeOperator, UnionMode};
// Create a merge operator that handles the rowid transformation
let operator_id = self.circuit.next_id;
let mode = if union.all {
// For UNION ALL, pass the source identifiers
UnionMode::All {
left_table: left_source,
right_table: right_source,
}
} else {
UnionMode::Distinct
};
let merge_operator = Box::new(MergeOperator::new(operator_id, mode));
let merge_id = self.circuit.add_node(
DbspOperator::Merge {
schema: union.schema.clone(),
},
vec![left_id, right_id],
merge_operator,
);
Ok(merge_id)
}
/// Convert a logical expression to a DBSP expression
fn compile_expr(expr: &LogicalExpr) -> Result<DbspExpr> {
match expr {

View File

@@ -0,0 +1,187 @@
// Merge operator for DBSP - combines two delta streams
// Used in recursive CTEs and UNION operations
use crate::incremental::dbsp::{Delta, DeltaPair, HashableRow};
use crate::incremental::operator::{
ComputationTracker, DbspStateCursors, EvalState, IncrementalOperator,
};
use crate::types::IOResult;
use crate::Result;
use std::collections::{hash_map::DefaultHasher, HashMap};
use std::fmt::{self, Display};
use std::hash::{Hash, Hasher};
use std::sync::{Arc, Mutex};
/// How the merge operator should handle rowids when combining deltas
#[derive(Debug, Clone)]
pub enum UnionMode {
/// For UNION (distinct) - hash values only to merge duplicates
Distinct,
/// For UNION ALL - include source table name in hash to keep duplicates separate
All {
left_table: String,
right_table: String,
},
}
/// Merge operator that combines two input deltas into one output delta
/// Handles both recursive CTEs and UNION/UNION ALL operations
#[derive(Debug)]
pub struct MergeOperator {
operator_id: usize,
union_mode: UnionMode,
/// For UNION: tracks seen value hashes with their assigned rowids
/// For UNION ALL: tracks (source_id, original_rowid) -> assigned_rowid mappings
seen_rows: HashMap<u64, i64>, // hash -> assigned_rowid
/// Next rowid to assign for new rows
next_rowid: i64,
}
impl MergeOperator {
/// Create a new merge operator with specified union mode
pub fn new(operator_id: usize, mode: UnionMode) -> Self {
Self {
operator_id,
union_mode: mode,
seen_rows: HashMap::new(),
next_rowid: 1,
}
}
/// Transform a delta's rowids based on the union mode with state tracking
fn transform_delta(&mut self, delta: Delta, is_left: bool) -> Delta {
match &self.union_mode {
UnionMode::Distinct => {
// For UNION distinct, track seen values and deduplicate
let mut output = Delta::new();
for (row, weight) in delta.changes {
// Hash only the values (not rowid) for deduplication
let temp_row = HashableRow::new(0, row.values.clone());
let value_hash = temp_row.cached_hash();
// Check if we've seen this value before
let assigned_rowid =
if let Some(&existing_rowid) = self.seen_rows.get(&value_hash) {
// Value already seen - use existing rowid
existing_rowid
} else {
// New value - assign new rowid and remember it
let new_rowid = self.next_rowid;
self.next_rowid += 1;
self.seen_rows.insert(value_hash, new_rowid);
new_rowid
};
// Output the row with the assigned rowid
let final_row = HashableRow::new(assigned_rowid, temp_row.values);
output.changes.push((final_row, weight));
}
output
}
UnionMode::All {
left_table,
right_table,
} => {
// For UNION ALL, maintain consistent rowid mapping per source
let table = if is_left { left_table } else { right_table };
let mut source_hasher = DefaultHasher::new();
table.hash(&mut source_hasher);
let source_id = source_hasher.finish();
let mut output = Delta::new();
for (row, weight) in delta.changes {
// Create a unique key for this (source, rowid) pair
let mut key_hasher = DefaultHasher::new();
source_id.hash(&mut key_hasher);
row.rowid.hash(&mut key_hasher);
let key_hash = key_hasher.finish();
// Check if we've seen this (source, rowid) before
let assigned_rowid =
if let Some(&existing_rowid) = self.seen_rows.get(&key_hash) {
// Use existing rowid for this (source, rowid) pair
existing_rowid
} else {
// New row - assign new rowid
let new_rowid = self.next_rowid;
self.next_rowid += 1;
self.seen_rows.insert(key_hash, new_rowid);
new_rowid
};
// Create output row with consistent rowid
let final_row = HashableRow::new(assigned_rowid, row.values.clone());
output.changes.push((final_row, weight));
}
output
}
}
}
}
impl Display for MergeOperator {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match &self.union_mode {
UnionMode::Distinct => write!(f, "MergeOperator({}, UNION)", self.operator_id),
UnionMode::All { .. } => write!(f, "MergeOperator({}, UNION ALL)", self.operator_id),
}
}
}
impl IncrementalOperator for MergeOperator {
fn eval(
&mut self,
input: &mut EvalState,
_cursors: &mut DbspStateCursors,
) -> Result<IOResult<Delta>> {
match input {
EvalState::Init { deltas } => {
// Extract deltas from the evaluation state
let delta_pair = std::mem::take(deltas);
// Transform deltas based on union mode (with state tracking)
let left_transformed = self.transform_delta(delta_pair.left, true);
let right_transformed = self.transform_delta(delta_pair.right, false);
// Merge the transformed deltas
let mut output = Delta::new();
output.merge(&left_transformed);
output.merge(&right_transformed);
// Move to Done state
*input = EvalState::Done;
Ok(IOResult::Done(output))
}
EvalState::Aggregate(_) | EvalState::Join(_) | EvalState::Uninitialized => {
// Merge operator only handles Init state
unreachable!("MergeOperator only handles Init state")
}
EvalState::Done => {
// Already evaluated
Ok(IOResult::Done(Delta::new()))
}
}
}
fn commit(
&mut self,
deltas: DeltaPair,
_cursors: &mut DbspStateCursors,
) -> Result<IOResult<Delta>> {
// Transform deltas based on union mode
let left_transformed = self.transform_delta(deltas.left, true);
let right_transformed = self.transform_delta(deltas.right, false);
// Merge the transformed deltas
let mut output = Delta::new();
output.merge(&left_transformed);
output.merge(&right_transformed);
Ok(IOResult::Done(output))
}
fn set_tracker(&mut self, _tracker: Arc<Mutex<ComputationTracker>>) {
// Merge operator doesn't need tracking for now
}
}

View File

@@ -6,6 +6,7 @@ pub mod expr_compiler;
pub mod filter_operator;
pub mod input_operator;
pub mod join_operator;
pub mod merge_operator;
pub mod operator;
pub mod persistence;
pub mod project_operator;

View File

@@ -3674,4 +3674,340 @@ mod tests {
assert!(was_new, "Duplicate rowid found: {}. This would cause rows to overwrite each other in btree storage!", row.rowid);
}
}
// Merge operator tests
use crate::incremental::merge_operator::{MergeOperator, UnionMode};
#[test]
fn test_merge_operator_basic() {
let (_pager, table_root_page_id, index_root_page_id) = create_test_pager();
let table_cursor = BTreeCursor::new_table(None, _pager.clone(), table_root_page_id, 5);
let index_def = create_dbsp_state_index(index_root_page_id);
let index_cursor =
BTreeCursor::new_index(None, _pager.clone(), index_root_page_id, &index_def, 4);
let mut cursors = DbspStateCursors::new(table_cursor, index_cursor);
let mut merge_op = MergeOperator::new(
1,
UnionMode::All {
left_table: "table1".to_string(),
right_table: "table2".to_string(),
},
);
// Create two deltas
let mut left_delta = Delta::new();
left_delta.insert(1, vec![Value::Integer(1)]);
left_delta.insert(2, vec![Value::Integer(2)]);
let mut right_delta = Delta::new();
right_delta.insert(3, vec![Value::Integer(3)]);
right_delta.insert(4, vec![Value::Integer(4)]);
let delta_pair = DeltaPair::new(left_delta, right_delta);
// Evaluate merge
let result = merge_op.commit(delta_pair, &mut cursors).unwrap();
if let IOResult::Done(merged) = result {
// Should have all 4 entries
assert_eq!(merged.len(), 4);
// Check that all values are present
let values: Vec<i64> = merged
.changes
.iter()
.filter_map(|(row, weight)| {
if *weight > 0 && !row.values.is_empty() {
if let Value::Integer(n) = &row.values[0] {
Some(*n)
} else {
None
}
} else {
None
}
})
.collect();
assert!(values.contains(&1));
assert!(values.contains(&2));
assert!(values.contains(&3));
assert!(values.contains(&4));
} else {
panic!("Expected Done result");
}
}
#[test]
fn test_merge_operator_stateful_distinct() {
let (_pager, table_root_page_id, index_root_page_id) = create_test_pager();
let table_cursor = BTreeCursor::new_table(None, _pager.clone(), table_root_page_id, 5);
let index_def = create_dbsp_state_index(index_root_page_id);
let index_cursor =
BTreeCursor::new_index(None, _pager.clone(), index_root_page_id, &index_def, 4);
let mut cursors = DbspStateCursors::new(table_cursor, index_cursor);
// Test that UNION (distinct) properly deduplicates across multiple operations
let mut merge_op = MergeOperator::new(7, UnionMode::Distinct);
// First operation: insert values 1, 2, 3 from left and 2, 3, 4 from right
let mut left_delta1 = Delta::new();
left_delta1.insert(1, vec![Value::Integer(1)]);
left_delta1.insert(2, vec![Value::Integer(2)]);
left_delta1.insert(3, vec![Value::Integer(3)]);
let mut right_delta1 = Delta::new();
right_delta1.insert(4, vec![Value::Integer(2)]); // Duplicate value 2
right_delta1.insert(5, vec![Value::Integer(3)]); // Duplicate value 3
right_delta1.insert(6, vec![Value::Integer(4)]);
let result1 = merge_op
.commit(DeltaPair::new(left_delta1, right_delta1), &mut cursors)
.unwrap();
if let IOResult::Done(merged1) = result1 {
// Should have 4 unique values (1, 2, 3, 4)
// But 6 total entries (3 from left + 3 from right)
assert_eq!(merged1.len(), 6);
// Collect unique rowids - should be 4
let unique_rowids: std::collections::HashSet<i64> =
merged1.changes.iter().map(|(row, _)| row.rowid).collect();
assert_eq!(
unique_rowids.len(),
4,
"Should have 4 unique rowids for 4 unique values"
);
} else {
panic!("Expected Done result");
}
// Second operation: insert value 2 again from left, and value 5 from right
let mut left_delta2 = Delta::new();
left_delta2.insert(7, vec![Value::Integer(2)]); // Duplicate of existing value
let mut right_delta2 = Delta::new();
right_delta2.insert(8, vec![Value::Integer(5)]); // New value
let result2 = merge_op
.commit(DeltaPair::new(left_delta2, right_delta2), &mut cursors)
.unwrap();
if let IOResult::Done(merged2) = result2 {
assert_eq!(merged2.len(), 2, "Should have 2 entries in delta");
// Check that value 2 got the same rowid as before
let has_existing_rowid = merged2
.changes
.iter()
.any(|(row, _)| row.values == vec![Value::Integer(2)] && row.rowid <= 4);
assert!(has_existing_rowid, "Value 2 should reuse existing rowid");
// Check that value 5 got a new rowid
let has_new_rowid = merged2
.changes
.iter()
.any(|(row, _)| row.values == vec![Value::Integer(5)] && row.rowid > 4);
assert!(has_new_rowid, "Value 5 should get a new rowid");
} else {
panic!("Expected Done result");
}
}
#[test]
fn test_merge_operator_single_sided_inputs_union_all() {
let (_pager, table_root_page_id, index_root_page_id) = create_test_pager();
let table_cursor = BTreeCursor::new_table(None, _pager.clone(), table_root_page_id, 5);
let index_def = create_dbsp_state_index(index_root_page_id);
let index_cursor =
BTreeCursor::new_index(None, _pager.clone(), index_root_page_id, &index_def, 4);
let mut cursors = DbspStateCursors::new(table_cursor, index_cursor);
// Test UNION ALL with inputs coming from only one side at a time
let mut merge_op = MergeOperator::new(
10,
UnionMode::All {
left_table: "orders".to_string(),
right_table: "archived_orders".to_string(),
},
);
// First: only left side (orders) has data
let mut left_delta1 = Delta::new();
left_delta1.insert(100, vec![Value::Integer(1001)]);
left_delta1.insert(101, vec![Value::Integer(1002)]);
let right_delta1 = Delta::new(); // Empty right side
let result1 = merge_op
.commit(DeltaPair::new(left_delta1, right_delta1), &mut cursors)
.unwrap();
let first_rowids = if let IOResult::Done(ref merged1) = result1 {
assert_eq!(merged1.len(), 2, "Should have 2 entries from left only");
merged1
.changes
.iter()
.map(|(row, _)| row.rowid)
.collect::<Vec<_>>()
} else {
panic!("Expected Done result");
};
// Second: only right side (archived_orders) has data
let left_delta2 = Delta::new(); // Empty left side
let mut right_delta2 = Delta::new();
right_delta2.insert(100, vec![Value::Integer(2001)]); // Same rowid as left, different table
right_delta2.insert(102, vec![Value::Integer(2002)]);
let result2 = merge_op
.commit(DeltaPair::new(left_delta2, right_delta2), &mut cursors)
.unwrap();
let second_result_rowid_100 = if let IOResult::Done(ref merged2) = result2 {
assert_eq!(merged2.len(), 2, "Should have 2 entries from right only");
// Rowids should be different from the left side even though original rowid 100 is the same
let second_rowids: Vec<i64> =
merged2.changes.iter().map(|(row, _)| row.rowid).collect();
for rowid in &second_rowids {
assert!(
!first_rowids.contains(rowid),
"Right side rowids should be different from left side rowids"
);
}
// Save rowid for archived_orders.100
merged2
.changes
.iter()
.find(|(row, _)| row.values == vec![Value::Integer(2001)])
.map(|(row, _)| row.rowid)
.unwrap()
} else {
panic!("Expected Done result");
};
// Third: left side again with same rowids as before
let mut left_delta3 = Delta::new();
left_delta3.insert(100, vec![Value::Integer(1003)]); // Same rowid 100 from orders
left_delta3.insert(101, vec![Value::Integer(1004)]); // Same rowid 101 from orders
let right_delta3 = Delta::new(); // Empty right side
let result3 = merge_op
.commit(DeltaPair::new(left_delta3, right_delta3), &mut cursors)
.unwrap();
if let IOResult::Done(merged3) = result3 {
assert_eq!(merged3.len(), 2, "Should have 2 entries from left");
// Should get the same assigned rowids as the first operation
let third_rowids: Vec<i64> = merged3.changes.iter().map(|(row, _)| row.rowid).collect();
assert_eq!(
first_rowids, third_rowids,
"Same (table, rowid) pairs should get same assigned rowids"
);
} else {
panic!("Expected Done result");
}
// Fourth: right side again with rowid 100
let left_delta4 = Delta::new(); // Empty left side
let mut right_delta4 = Delta::new();
right_delta4.insert(100, vec![Value::Integer(2003)]); // Same rowid 100 from archived_orders
let result4 = merge_op
.commit(DeltaPair::new(left_delta4, right_delta4), &mut cursors)
.unwrap();
if let IOResult::Done(merged4) = result4 {
assert_eq!(merged4.len(), 1, "Should have 1 entry from right");
// Should get same assigned rowid as second operation for archived_orders.100
let fourth_rowid = merged4.changes[0].0.rowid;
assert_eq!(
fourth_rowid, second_result_rowid_100,
"archived_orders rowid 100 should consistently map to same assigned rowid"
);
} else {
panic!("Expected Done result");
}
}
#[test]
fn test_merge_operator_both_sides_empty() {
let (_pager, table_root_page_id, index_root_page_id) = create_test_pager();
let table_cursor = BTreeCursor::new_table(None, _pager.clone(), table_root_page_id, 5);
let index_def = create_dbsp_state_index(index_root_page_id);
let index_cursor =
BTreeCursor::new_index(None, _pager.clone(), index_root_page_id, &index_def, 4);
let mut cursors = DbspStateCursors::new(table_cursor, index_cursor);
// Test that both sides being empty works correctly
let mut merge_op = MergeOperator::new(
12,
UnionMode::All {
left_table: "t1".to_string(),
right_table: "t2".to_string(),
},
);
// First: insert some data to establish state
let mut left_delta1 = Delta::new();
left_delta1.insert(1, vec![Value::Integer(100)]);
let mut right_delta1 = Delta::new();
right_delta1.insert(1, vec![Value::Integer(200)]);
let result1 = merge_op
.commit(DeltaPair::new(left_delta1, right_delta1), &mut cursors)
.unwrap();
let original_t1_rowid = if let IOResult::Done(ref merged1) = result1 {
assert_eq!(merged1.len(), 2, "Should have 2 entries initially");
// Save the rowid for t1.rowid=1
merged1
.changes
.iter()
.find(|(row, _)| row.values == vec![Value::Integer(100)])
.map(|(row, _)| row.rowid)
.unwrap()
} else {
panic!("Expected Done result");
};
// Second: both sides empty - should produce empty output
let empty_left = Delta::new();
let empty_right = Delta::new();
let result2 = merge_op
.commit(DeltaPair::new(empty_left, empty_right), &mut cursors)
.unwrap();
if let IOResult::Done(merged2) = result2 {
assert_eq!(
merged2.len(),
0,
"Both empty sides should produce empty output"
);
} else {
panic!("Expected Done result");
}
// Third: add more data to verify state is still intact
let mut left_delta3 = Delta::new();
left_delta3.insert(1, vec![Value::Integer(101)]); // Same rowid as before
let right_delta3 = Delta::new();
let result3 = merge_op
.commit(DeltaPair::new(left_delta3, right_delta3), &mut cursors)
.unwrap();
if let IOResult::Done(merged3) = result3 {
assert_eq!(merged3.len(), 1, "Should have 1 entry");
// Should reuse the same assigned rowid for t1.rowid=1
let rowid = merged3.changes[0].0.rowid;
assert_eq!(
rowid, original_t1_rowid,
"Should maintain consistent rowid mapping after empty operation"
);
} else {
panic!("Expected Done result");
}
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -1091,3 +1091,340 @@ do_execsql_test_on_specific_db {:memory:} matview-join-complex-where {
} {Charlie|10|100|1000
Alice|5|100|500
Charlie|6|75|450}
# Test UNION queries in materialized views
do_execsql_test_on_specific_db {:memory:} matview-union-simple {
CREATE TABLE sales_online(id INTEGER, product TEXT, amount INTEGER);
CREATE TABLE sales_store(id INTEGER, product TEXT, amount INTEGER);
INSERT INTO sales_online VALUES
(1, 'Laptop', 1200),
(2, 'Mouse', 25),
(3, 'Monitor', 400);
INSERT INTO sales_store VALUES
(1, 'Keyboard', 75),
(2, 'Chair', 150),
(3, 'Desk', 350);
-- Create a view that combines both sources
CREATE MATERIALIZED VIEW all_sales AS
SELECT product, amount FROM sales_online
UNION ALL
SELECT product, amount FROM sales_store;
SELECT * FROM all_sales ORDER BY product;
} {Chair|150
Desk|350
Keyboard|75
Laptop|1200
Monitor|400
Mouse|25}
do_execsql_test_on_specific_db {:memory:} matview-union-with-where {
CREATE TABLE employees(id INTEGER, name TEXT, dept TEXT, salary INTEGER);
CREATE TABLE contractors(id INTEGER, name TEXT, dept TEXT, rate INTEGER);
INSERT INTO employees VALUES
(1, 'Alice', 'Engineering', 90000),
(2, 'Bob', 'Sales', 60000),
(3, 'Charlie', 'Engineering', 85000);
INSERT INTO contractors VALUES
(1, 'David', 'Engineering', 150),
(2, 'Eve', 'Marketing', 120),
(3, 'Frank', 'Engineering', 180);
-- High-earning staff from both categories
CREATE MATERIALIZED VIEW high_earners AS
SELECT name, dept, salary as compensation FROM employees WHERE salary > 80000
UNION ALL
SELECT name, dept, rate * 2000 as compensation FROM contractors WHERE rate > 140;
SELECT * FROM high_earners ORDER BY name;
} {Alice|Engineering|90000
Charlie|Engineering|85000
David|Engineering|300000
Frank|Engineering|360000}
do_execsql_test_on_specific_db {:memory:} matview-union-same-table-different-filters {
CREATE TABLE orders(id INTEGER, customer_id INTEGER, product TEXT, amount INTEGER, status TEXT);
INSERT INTO orders VALUES
(1, 1, 'Laptop', 1200, 'completed'),
(2, 2, 'Mouse', 25, 'pending'),
(3, 1, 'Monitor', 400, 'completed'),
(4, 3, 'Keyboard', 75, 'cancelled'),
(5, 2, 'Desk', 350, 'completed'),
(6, 3, 'Chair', 150, 'pending');
-- View showing priority orders: high-value OR pending status
CREATE MATERIALIZED VIEW priority_orders AS
SELECT id, customer_id, product, amount FROM orders WHERE amount > 300
UNION ALL
SELECT id, customer_id, product, amount FROM orders WHERE status = 'pending';
SELECT * FROM priority_orders ORDER BY id;
} {1|1|Laptop|1200
2|2|Mouse|25
3|1|Monitor|400
5|2|Desk|350
6|3|Chair|150}
do_execsql_test_on_specific_db {:memory:} matview-union-with-aggregation {
CREATE TABLE q1_sales(product TEXT, quantity INTEGER, revenue INTEGER);
CREATE TABLE q2_sales(product TEXT, quantity INTEGER, revenue INTEGER);
INSERT INTO q1_sales VALUES
('Laptop', 10, 12000),
('Mouse', 50, 1250),
('Monitor', 8, 3200);
INSERT INTO q2_sales VALUES
('Laptop', 15, 18000),
('Mouse', 60, 1500),
('Keyboard', 30, 2250);
-- Combined quarterly summary
CREATE MATERIALIZED VIEW half_year_summary AS
SELECT 'Q1' as quarter, SUM(quantity) as total_units, SUM(revenue) as total_revenue
FROM q1_sales
UNION ALL
SELECT 'Q2' as quarter, SUM(quantity) as total_units, SUM(revenue) as total_revenue
FROM q2_sales;
SELECT * FROM half_year_summary ORDER BY quarter;
} {Q1|68|16450
Q2|105|21750}
do_execsql_test_on_specific_db {:memory:} matview-union-with-join {
CREATE TABLE customers(id INTEGER PRIMARY KEY, name TEXT, type TEXT);
CREATE TABLE orders(id INTEGER PRIMARY KEY, customer_id INTEGER, amount INTEGER);
CREATE TABLE quotes(id INTEGER PRIMARY KEY, customer_id INTEGER, amount INTEGER);
INSERT INTO customers VALUES
(1, 'Alice', 'premium'),
(2, 'Bob', 'regular'),
(3, 'Charlie', 'premium');
INSERT INTO orders VALUES
(1, 1, 1000),
(2, 2, 500),
(3, 3, 1500);
INSERT INTO quotes VALUES
(1, 1, 800),
(2, 2, 300),
(3, 3, 2000);
-- All premium customer transactions (orders and quotes)
CREATE MATERIALIZED VIEW premium_transactions AS
SELECT c.name, 'order' as type, o.amount
FROM customers c
JOIN orders o ON c.id = o.customer_id
WHERE c.type = 'premium'
UNION ALL
SELECT c.name, 'quote' as type, q.amount
FROM customers c
JOIN quotes q ON c.id = q.customer_id
WHERE c.type = 'premium';
SELECT * FROM premium_transactions ORDER BY name, type, amount;
} {Alice|order|1000
Alice|quote|800
Charlie|order|1500
Charlie|quote|2000}
do_execsql_test_on_specific_db {:memory:} matview-union-distinct {
CREATE TABLE active_users(id INTEGER, name TEXT, email TEXT);
CREATE TABLE inactive_users(id INTEGER, name TEXT, email TEXT);
INSERT INTO active_users VALUES
(1, 'Alice', 'alice@example.com'),
(2, 'Bob', 'bob@example.com'),
(3, 'Charlie', 'charlie@example.com');
INSERT INTO inactive_users VALUES
(4, 'David', 'david@example.com'),
(2, 'Bob', 'bob@example.com'), -- Bob appears in both
(5, 'Eve', 'eve@example.com');
-- All unique users (using UNION to deduplicate)
CREATE MATERIALIZED VIEW all_users AS
SELECT id, name, email FROM active_users
UNION
SELECT id, name, email FROM inactive_users;
SELECT * FROM all_users ORDER BY id;
} {1|Alice|alice@example.com
2|Bob|bob@example.com
3|Charlie|charlie@example.com
4|David|david@example.com
5|Eve|eve@example.com}
do_execsql_test_on_specific_db {:memory:} matview-union-complex-multiple-branches {
CREATE TABLE products(id INTEGER, name TEXT, category TEXT, price INTEGER);
INSERT INTO products VALUES
(1, 'Laptop', 'Electronics', 1200),
(2, 'Mouse', 'Electronics', 25),
(3, 'Desk', 'Furniture', 350),
(4, 'Chair', 'Furniture', 150),
(5, 'Monitor', 'Electronics', 400),
(6, 'Keyboard', 'Electronics', 75),
(7, 'Bookshelf', 'Furniture', 200),
(8, 'Tablet', 'Electronics', 600);
-- Products of interest: expensive electronics, all furniture, or very cheap items
CREATE MATERIALIZED VIEW featured_products AS
SELECT name, category, price, 'PremiumElectronic' as tag
FROM products
WHERE category = 'Electronics' AND price > 500
UNION ALL
SELECT name, category, price, 'Furniture' as tag
FROM products
WHERE category = 'Furniture'
UNION ALL
SELECT name, category, price, 'Budget' as tag
FROM products
WHERE price < 50;
SELECT * FROM featured_products ORDER BY tag, name;
} {Mouse|Electronics|25|Budget
Bookshelf|Furniture|200|Furniture
Chair|Furniture|150|Furniture
Desk|Furniture|350|Furniture
Laptop|Electronics|1200|PremiumElectronic
Tablet|Electronics|600|PremiumElectronic}
do_execsql_test_on_specific_db {:memory:} matview-union-maintenance-insert {
CREATE TABLE t1(id INTEGER, value INTEGER);
CREATE TABLE t2(id INTEGER, value INTEGER);
INSERT INTO t1 VALUES (1, 100), (2, 200);
INSERT INTO t2 VALUES (3, 300), (4, 400);
CREATE MATERIALIZED VIEW combined AS
SELECT id, value FROM t1 WHERE value > 150
UNION ALL
SELECT id, value FROM t2 WHERE value > 350;
SELECT * FROM combined ORDER BY id;
-- Insert into t1
INSERT INTO t1 VALUES (5, 500);
SELECT * FROM combined ORDER BY id;
-- Insert into t2
INSERT INTO t2 VALUES (6, 600);
SELECT * FROM combined ORDER BY id;
} {2|200
4|400
2|200
4|400
5|500
2|200
4|400
5|500
6|600}
do_execsql_test_on_specific_db {:memory:} matview-union-maintenance-delete {
CREATE TABLE source1(id INTEGER PRIMARY KEY, data TEXT);
CREATE TABLE source2(id INTEGER PRIMARY KEY, data TEXT);
INSERT INTO source1 VALUES (1, 'A'), (2, 'B'), (3, 'C');
INSERT INTO source2 VALUES (4, 'D'), (5, 'E'), (6, 'F');
CREATE MATERIALIZED VIEW merged AS
SELECT id, data FROM source1
UNION ALL
SELECT id, data FROM source2;
SELECT COUNT(*) FROM merged;
DELETE FROM source1 WHERE id = 2;
SELECT COUNT(*) FROM merged;
DELETE FROM source2 WHERE id > 4;
SELECT COUNT(*) FROM merged;
} {6
5
3}
do_execsql_test_on_specific_db {:memory:} matview-union-maintenance-update {
CREATE TABLE high_priority(id INTEGER PRIMARY KEY, task TEXT, priority INTEGER);
CREATE TABLE normal_priority(id INTEGER PRIMARY KEY, task TEXT, priority INTEGER);
INSERT INTO high_priority VALUES (1, 'Task A', 10), (2, 'Task B', 9);
INSERT INTO normal_priority VALUES (3, 'Task C', 5), (4, 'Task D', 6);
CREATE MATERIALIZED VIEW active_tasks AS
SELECT id, task, priority FROM high_priority WHERE priority >= 9
UNION ALL
SELECT id, task, priority FROM normal_priority WHERE priority >= 5;
SELECT COUNT(*) FROM active_tasks;
-- Update drops a high priority task below threshold
UPDATE high_priority SET priority = 8 WHERE id = 2;
SELECT COUNT(*) FROM active_tasks;
-- Update brings a normal task above threshold
UPDATE normal_priority SET priority = 3 WHERE id = 3;
SELECT COUNT(*) FROM active_tasks;
} {4
3
2}
# Test UNION ALL with same table and different WHERE conditions
do_execsql_test_on_specific_db {:memory:} matview-union-all-same-table {
CREATE TABLE test(id INTEGER PRIMARY KEY, value INTEGER);
INSERT INTO test VALUES (1, 10), (2, 20);
-- This UNION ALL should return both rows
CREATE MATERIALIZED VIEW union_view AS
SELECT id, value FROM test WHERE value < 15
UNION ALL
SELECT id, value FROM test WHERE value > 15;
-- Should return 2 rows: (1,10) and (2,20)
SELECT * FROM union_view ORDER BY id;
} {1|10
2|20}
# Test UNION ALL preserves all rows in count
do_execsql_test_on_specific_db {:memory:} matview-union-all-row-count {
CREATE TABLE data(id INTEGER PRIMARY KEY, num INTEGER);
INSERT INTO data VALUES (1, 5), (2, 15), (3, 25);
CREATE MATERIALIZED VIEW split_view AS
SELECT id, num FROM data WHERE num <= 10
UNION ALL
SELECT id, num FROM data WHERE num > 10;
-- Should return count of 3
SELECT COUNT(*) FROM split_view;
} {3}
# Test UNION ALL with text columns and filtering
do_execsql_test_on_specific_db {:memory:} matview-union-all-text-filter {
CREATE TABLE items(id INTEGER PRIMARY KEY, category TEXT, price INTEGER);
INSERT INTO items VALUES
(1, 'cheap', 10),
(2, 'expensive', 100),
(3, 'cheap', 20),
(4, 'expensive', 200);
CREATE MATERIALIZED VIEW price_categories AS
SELECT id, category, price FROM items WHERE category = 'cheap'
UNION ALL
SELECT id, category, price FROM items WHERE category = 'expensive';
-- Should return all 4 items
SELECT COUNT(*) FROM price_categories;
SELECT id FROM price_categories ORDER BY id;
} {4
1
2
3
4}