mirror of
https://github.com/aljazceru/turso.git
synced 2026-01-31 13:54:27 +01:00
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:
@@ -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 {
|
||||
|
||||
187
core/incremental/merge_operator.rs
Normal file
187
core/incremental/merge_operator.rs
Normal 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
|
||||
}
|
||||
}
|
||||
@@ -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;
|
||||
|
||||
@@ -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
@@ -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}
|
||||
|
||||
Reference in New Issue
Block a user