From e255fc9a81d7c2958dda294db4b4a33264190bbc Mon Sep 17 00:00:00 2001 From: Glauber Costa Date: Sun, 10 Aug 2025 16:10:45 -0500 Subject: [PATCH 01/10] Add table name to the delete bytecode When building views (soon), it will be important to know which table is being deleted. Getting from the cursor id is very cumbersome. What we are doing here is symmetrical to op_insert, and sqlite also passes table information in one of the registers (p4) --- core/translate/emitter.rs | 6 +++++- core/translate/index.rs | 1 + core/translate/schema.rs | 2 ++ core/vdbe/execute.rs | 8 +++++++- core/vdbe/explain.rs | 4 ++-- core/vdbe/insn.rs | 1 + 6 files changed, 18 insertions(+), 4 deletions(-) diff --git a/core/translate/emitter.rs b/core/translate/emitter.rs index 42d161317..e9b72735b 100644 --- a/core/translate/emitter.rs +++ b/core/translate/emitter.rs @@ -634,6 +634,7 @@ fn emit_delete_insns( program.emit_insn(Insn::Delete { cursor_id: main_table_cursor_id, + table_name: table_reference.table.get_name().to_string(), }); } if let Some(limit_ctx) = t_ctx.limit_ctx { @@ -1175,7 +1176,10 @@ fn emit_update_insns( // Insert instruction to update the cell. We need to first delete the current cell // and later insert the updated record if has_user_provided_rowid { - program.emit_insn(Insn::Delete { cursor_id }); + program.emit_insn(Insn::Delete { + cursor_id, + table_name: table_ref.table.get_name().to_string(), + }); } program.emit_insn(Insn::Insert { diff --git a/core/translate/index.rs b/core/translate/index.rs index b79d9d834..0a6175f46 100644 --- a/core/translate/index.rs +++ b/core/translate/index.rs @@ -462,6 +462,7 @@ pub fn translate_drop_index( program.emit_insn(Insn::Delete { cursor_id: sqlite_schema_cursor_id, + table_name: "sqlite_schema".to_string(), }); program.resolve_label(next_label, program.offset()); diff --git a/core/translate/schema.rs b/core/translate/schema.rs index e2d05b339..786e2684f 100644 --- a/core/translate/schema.rs +++ b/core/translate/schema.rs @@ -779,6 +779,7 @@ pub fn translate_drop_table( } program.emit_insn(Insn::Delete { cursor_id: sqlite_schema_cursor_id_0, + table_name: SQLITE_TABLEID.to_string(), }); program.resolve_label(next_label, program.offset()); @@ -978,6 +979,7 @@ pub fn translate_drop_table( }); program.emit_insn(Insn::Delete { cursor_id: sqlite_schema_cursor_id_1, + table_name: SQLITE_TABLEID.to_string(), }); program.emit_insn(Insn::Insert { cursor: sqlite_schema_cursor_id_1, diff --git a/core/vdbe/execute.rs b/core/vdbe/execute.rs index 10415eb64..856e9fcbb 100644 --- a/core/vdbe/execute.rs +++ b/core/vdbe/execute.rs @@ -5217,7 +5217,13 @@ pub fn op_delete( pager: &Rc, mv_store: Option<&Arc>, ) -> Result { - load_insn!(Delete { cursor_id }, insn); + load_insn!( + Delete { + cursor_id, + table_name: _ + }, + insn + ); { let mut cursor = state.get_cursor(*cursor_id); let cursor = cursor.as_btree_mut(); diff --git a/core/vdbe/explain.rs b/core/vdbe/explain.rs index e4245aba6..6f2f4352e 100644 --- a/core/vdbe/explain.rs +++ b/core/vdbe/explain.rs @@ -1109,12 +1109,12 @@ pub fn insn_to_str( flag.0 as u16, format!("intkey=r[{key_reg}] data=r[{record_reg}]"), ), - Insn::Delete { cursor_id } => ( + Insn::Delete { cursor_id, table_name } => ( "Delete", *cursor_id as i32, 0, 0, - Value::build_text(""), + Value::build_text(table_name), 0, "".to_string(), ), diff --git a/core/vdbe/insn.rs b/core/vdbe/insn.rs index b7a7e680d..dce8ea938 100644 --- a/core/vdbe/insn.rs +++ b/core/vdbe/insn.rs @@ -745,6 +745,7 @@ pub enum Insn { Delete { cursor_id: CursorID, + table_name: String, }, /// If P5 is not zero, then raise an SQLITE_CORRUPT_INDEX error if no matching index entry From d5b7533ff8e0e1b9e22ca01d760d3132dffe886b Mon Sep 17 00:00:00 2001 From: Glauber Costa Date: Fri, 8 Aug 2025 13:02:52 -0500 Subject: [PATCH 02/10] Implement a DBSP module We are not using the DBSP crate because it is very heavy on Tokio and other dependencies that won't make sense for us to consume. --- core/incremental/dbsp.rs | 119 ++ core/incremental/hashable_row.rs | 80 ++ core/incremental/mod.rs | 3 + core/incremental/operator.rs | 1889 ++++++++++++++++++++++++++++++ core/lib.rs | 1 + 5 files changed, 2092 insertions(+) create mode 100644 core/incremental/dbsp.rs create mode 100644 core/incremental/hashable_row.rs create mode 100644 core/incremental/mod.rs create mode 100644 core/incremental/operator.rs diff --git a/core/incremental/dbsp.rs b/core/incremental/dbsp.rs new file mode 100644 index 000000000..962ceb9c8 --- /dev/null +++ b/core/incremental/dbsp.rs @@ -0,0 +1,119 @@ +// Simplified DBSP integration for incremental view maintenance +// For now, we'll use a basic approach and can expand to full DBSP later + +use std::collections::HashMap; + +/// A simplified ZSet for incremental computation +/// Each element has a weight: positive for additions, negative for deletions +#[derive(Clone, Debug, Default)] +pub struct SimpleZSet { + data: HashMap, +} + +impl SimpleZSet { + pub fn new() -> Self { + Self { + data: HashMap::new(), + } + } + + pub fn insert(&mut self, item: T, weight: isize) { + let current = self.data.get(&item).copied().unwrap_or(0); + let new_weight = current + weight; + if new_weight == 0 { + self.data.remove(&item); + } else { + self.data.insert(item, new_weight); + } + } + + pub fn iter(&self) -> impl Iterator { + self.data.iter().map(|(k, &v)| (k, v)) + } + + /// Get all items with positive weights + pub fn to_vec(&self) -> Vec { + self.data + .iter() + .filter(|(_, &weight)| weight > 0) + .map(|(item, _)| item.clone()) + .collect() + } + + pub fn merge(&mut self, other: &SimpleZSet) { + for (item, weight) in other.iter() { + self.insert(item.clone(), weight); + } + } +} + +/// A simplified stream for incremental computation +#[derive(Clone, Debug)] +pub struct SimpleStream { + current: SimpleZSet, +} + +impl SimpleStream { + pub fn from_zset(zset: SimpleZSet) -> Self { + Self { current: zset } + } + + /// Apply a delta (change) to the stream + pub fn apply_delta(&mut self, delta: &SimpleZSet) { + self.current.merge(delta); + } + + /// Get the current state as a vector of items (only positive weights) + pub fn to_vec(&self) -> Vec { + self.current.to_vec() + } +} + +// Type aliases for convenience +use super::hashable_row::HashableRow; + +pub type RowKey = HashableRow; +pub type RowKeyZSet = SimpleZSet; +pub type RowKeyStream = SimpleStream; + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_zset_merge_with_weights() { + let mut zset1 = SimpleZSet::new(); + zset1.insert(1, 1); // Row 1 with weight +1 + zset1.insert(2, 1); // Row 2 with weight +1 + + let mut zset2 = SimpleZSet::new(); + zset2.insert(2, -1); // Row 2 with weight -1 (delete) + zset2.insert(3, 1); // Row 3 with weight +1 (insert) + + zset1.merge(&zset2); + + // Row 1: weight 1 (unchanged) + // Row 2: weight 0 (deleted) + // Row 3: weight 1 (inserted) + assert_eq!(zset1.iter().count(), 2); // Only rows 1 and 3 + assert!(zset1.iter().any(|(k, _)| *k == 1)); + assert!(zset1.iter().any(|(k, _)| *k == 3)); + assert!(!zset1.iter().any(|(k, _)| *k == 2)); // Row 2 removed + } + + #[test] + fn test_zset_represents_updates_as_delete_plus_insert() { + let mut zset = SimpleZSet::new(); + + // Initial state + zset.insert(1, 1); + + // Update row 1: delete old + insert new + zset.insert(1, -1); // Delete old version + zset.insert(1, 1); // Insert new version + + // Weight should be 1 (not 2) + let weight = zset.iter().find(|(k, _)| **k == 1).map(|(_, w)| w); + assert_eq!(weight, Some(1)); + } +} diff --git a/core/incremental/hashable_row.rs b/core/incremental/hashable_row.rs new file mode 100644 index 000000000..46be59bde --- /dev/null +++ b/core/incremental/hashable_row.rs @@ -0,0 +1,80 @@ +use crate::types::Value; +use std::collections::hash_map::DefaultHasher; +use std::hash::{Hash, Hasher}; + +// The DBSP paper uses as a key the whole record, with both the row key and the values. This is a +// bit confuses for us in databases, because when you say "key", it is easy to understand that as +// being the row key. +// +// Empirically speaking, using row keys as the ZSet keys will waste a competent but not brilliant +// engineer around 82 and 88 hours, depending on how you count. Hours that are never coming back. +// +// One of the situations in which using row keys completely breaks are table updates. If the "key" +// is the row key, let's say "5", then an update is a delete + insert. Imagine a table that had k = +// 5, v = 5, and a view that filters v > 2. +// +// Now we will do an update that changes v => 1. If the "key" is 5, then inside the Delta set, we +// will have (5, weight = -1), (5, weight = +1), and the whole thing just disappears. The Delta +// set, therefore, has to contain ((5, 5), weight = -1), ((5, 1), weight = +1). +// +// It is theoretically possible to use the rowkey in the ZSet and then use a hash of key -> +// Vec(changes) in the Delta set. But deviating from the paper here is just asking for trouble, as +// I am sure it would break somewhere else. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct HashableRow { + pub rowid: i64, + pub values: Vec, + // Pre-computed hash: DBSP rows are immutable and frequently hashed during joins, + // making caching worthwhile despite the memory overhead + cached_hash: u64, +} + +impl HashableRow { + pub fn new(rowid: i64, values: Vec) -> Self { + let cached_hash = Self::compute_hash(rowid, &values); + Self { + rowid, + values, + cached_hash, + } + } + + fn compute_hash(rowid: i64, values: &[Value]) -> u64 { + let mut hasher = DefaultHasher::new(); + + rowid.hash(&mut hasher); + + for value in values { + match value { + Value::Null => { + 0u8.hash(&mut hasher); + } + Value::Integer(i) => { + 1u8.hash(&mut hasher); + i.hash(&mut hasher); + } + Value::Float(f) => { + 2u8.hash(&mut hasher); + f.to_bits().hash(&mut hasher); + } + Value::Text(s) => { + 3u8.hash(&mut hasher); + s.value.hash(&mut hasher); + (s.subtype as u8).hash(&mut hasher); + } + Value::Blob(b) => { + 4u8.hash(&mut hasher); + b.hash(&mut hasher); + } + } + } + + hasher.finish() + } +} + +impl Hash for HashableRow { + fn hash(&self, state: &mut H) { + self.cached_hash.hash(state); + } +} diff --git a/core/incremental/mod.rs b/core/incremental/mod.rs new file mode 100644 index 000000000..ef5242330 --- /dev/null +++ b/core/incremental/mod.rs @@ -0,0 +1,3 @@ +pub mod dbsp; +pub mod hashable_row; +pub mod operator; diff --git a/core/incremental/operator.rs b/core/incremental/operator.rs new file mode 100644 index 000000000..194d86013 --- /dev/null +++ b/core/incremental/operator.rs @@ -0,0 +1,1889 @@ +#![allow(dead_code)] +// Operator DAG for DBSP-style incremental computation +// Based on Feldera DBSP design but adapted for Turso's architecture + +use crate::incremental::hashable_row::HashableRow; +use crate::types::Text; +use crate::Value; +use std::collections::{HashMap, HashSet}; +use std::fmt::{self, Debug, Display}; +use std::sync::Arc; +use std::sync::Mutex; + +/// Tracks computation counts to verify incremental behavior (for tests now), and in the future +/// should be used to provide statistics. +#[derive(Debug, Default, Clone)] +pub struct ComputationTracker { + pub filter_evaluations: usize, + pub project_operations: usize, + pub join_lookups: usize, + pub aggregation_updates: usize, + pub full_scans: usize, +} + +impl ComputationTracker { + pub fn new() -> Self { + Self::default() + } + + pub fn record_filter(&mut self) { + self.filter_evaluations += 1; + } + + pub fn record_project(&mut self) { + self.project_operations += 1; + } + + pub fn record_join_lookup(&mut self) { + self.join_lookups += 1; + } + + pub fn record_aggregation(&mut self) { + self.aggregation_updates += 1; + } + + pub fn record_full_scan(&mut self) { + self.full_scans += 1; + } + + pub fn total_computations(&self) -> usize { + self.filter_evaluations + + self.project_operations + + self.join_lookups + + self.aggregation_updates + } +} + +/// A delta represents ordered changes to data +#[derive(Debug, Clone, Default)] +pub struct Delta { + /// Ordered list of changes: (row, weight) where weight is +1 for insert, -1 for delete + /// It is crucial that this is ordered. Imagine the case of an update, which becomes a delete + + /// insert. If this is not ordered, it would be applied in arbitrary order and break the view. + pub changes: Vec<(HashableRow, isize)>, +} + +impl Delta { + pub fn new() -> Self { + Self { + changes: Vec::new(), + } + } + + pub fn insert(&mut self, row_key: i64, values: Vec) { + let row = HashableRow::new(row_key, values); + self.changes.push((row, 1)); + } + + pub fn delete(&mut self, row_key: i64, values: Vec) { + let row = HashableRow::new(row_key, values); + self.changes.push((row, -1)); + } + + pub fn is_empty(&self) -> bool { + self.changes.is_empty() + } + + pub fn len(&self) -> usize { + self.changes.len() + } + + /// Merge another delta into this one + /// This preserves the order of operations - no consolidation is done + /// to maintain the full history of changes + pub fn merge(&mut self, other: &Delta) { + // Simply append all changes from other, preserving order + self.changes.extend(other.changes.iter().cloned()); + } + + /// Consolidate changes by combining entries with the same HashableRow + pub fn consolidate(&mut self) { + if self.changes.is_empty() { + return; + } + + // Use a HashMap to accumulate weights + let mut consolidated: HashMap = HashMap::new(); + + for (row, weight) in self.changes.drain(..) { + *consolidated.entry(row).or_insert(0) += weight; + } + + // Convert back to vec, filtering out zero weights + self.changes = consolidated + .into_iter() + .filter(|(_, weight)| *weight != 0) + .collect(); + } +} + +#[cfg(test)] +mod hashable_row_tests { + use super::*; + + #[test] + fn test_hashable_row_delta_operations() { + let mut delta = Delta::new(); + + // Test INSERT + delta.insert(1, vec![Value::Integer(1), Value::Integer(100)]); + assert_eq!(delta.len(), 1); + + // Test UPDATE (DELETE + INSERT) - order matters! + delta.delete(1, vec![Value::Integer(1), Value::Integer(100)]); + delta.insert(1, vec![Value::Integer(1), Value::Integer(200)]); + assert_eq!(delta.len(), 3); // Should have 3 operations before consolidation + + // Verify order is preserved + let ops: Vec<_> = delta.changes.iter().collect(); + assert_eq!(ops[0].1, 1); // First insert + assert_eq!(ops[1].1, -1); // Delete + assert_eq!(ops[2].1, 1); // Second insert + + // Test consolidation + delta.consolidate(); + // After consolidation, the first insert and delete should cancel out + // leaving only the second insert + assert_eq!(delta.len(), 1); + + let final_row = &delta.changes[0]; + assert_eq!(final_row.0.rowid, 1); + assert_eq!( + final_row.0.values, + vec![Value::Integer(1), Value::Integer(200)] + ); + assert_eq!(final_row.1, 1); + } + + #[test] + fn test_duplicate_row_consolidation() { + let mut delta = Delta::new(); + + // Insert same row twice + delta.insert(2, vec![Value::Integer(2), Value::Integer(300)]); + delta.insert(2, vec![Value::Integer(2), Value::Integer(300)]); + + assert_eq!(delta.len(), 2); + + delta.consolidate(); + assert_eq!(delta.len(), 1); + + // Weight should be 2 (sum of both inserts) + let final_row = &delta.changes[0]; + assert_eq!(final_row.0.rowid, 2); + assert_eq!(final_row.1, 2); + } +} + +/// Represents an operator in the dataflow graph +#[derive(Debug, Clone)] +pub enum QueryOperator { + /// Table scan - source of data + TableScan { + table_name: String, + column_names: Vec, + }, + + /// Filter rows based on predicate + Filter { + predicate: FilterPredicate, + input: usize, // Index of input operator + }, + + /// Project columns (select specific columns) + Project { + columns: Vec, + input: usize, + }, + + /// Join two inputs + Join { + join_type: JoinType, + on_column: String, + left_input: usize, + right_input: usize, + }, + + /// Aggregate + Aggregate { + group_by: Vec, + aggregates: Vec, + input: usize, + }, +} + +#[derive(Debug, Clone)] +pub enum FilterPredicate { + /// Column = value + Equals { column: String, value: Value }, + /// Column != value + NotEquals { column: String, value: Value }, + /// Column > value + GreaterThan { column: String, value: Value }, + /// Column >= value + GreaterThanOrEqual { column: String, value: Value }, + /// Column < value + LessThan { column: String, value: Value }, + /// Column <= value + LessThanOrEqual { column: String, value: Value }, + /// Logical AND of two predicates + And(Box, Box), + /// Logical OR of two predicates + Or(Box, Box), + /// No predicate (accept all rows) + None, +} + +impl FilterPredicate { + /// Parse a SQL AST expression into a FilterPredicate + /// This centralizes all SQL-to-predicate parsing logic + pub fn from_sql_expr(expr: &turso_sqlite3_parser::ast::Expr) -> Self { + use turso_sqlite3_parser::ast::*; + + if let Expr::Binary(lhs, op, rhs) = expr { + // Handle AND/OR logical operators + match op { + Operator::And => { + let left = Self::from_sql_expr(lhs); + let right = Self::from_sql_expr(rhs); + return FilterPredicate::And(Box::new(left), Box::new(right)); + } + Operator::Or => { + let left = Self::from_sql_expr(lhs); + let right = Self::from_sql_expr(rhs); + return FilterPredicate::Or(Box::new(left), Box::new(right)); + } + _ => {} + } + + // Handle comparison operators + if let Expr::Id(column_name) = &**lhs { + let column = column_name.as_str().to_string(); + + // Parse the right-hand side value + let value = match &**rhs { + Expr::Literal(Literal::String(s)) => { + // Strip quotes from string literals + let cleaned = s.trim_matches('\'').trim_matches('"'); + Value::Text(Text::new(cleaned)) + } + Expr::Literal(Literal::Numeric(n)) => { + // Try to parse as integer first, then float + if let Ok(i) = n.parse::() { + Value::Integer(i) + } else if let Ok(f) = n.parse::() { + Value::Float(f) + } else { + return FilterPredicate::None; + } + } + Expr::Literal(Literal::Null) => Value::Null, + Expr::Literal(Literal::Blob(_)) => { + // Blob comparison not yet supported + return FilterPredicate::None; + } + _ => { + // Complex expressions not yet supported + return FilterPredicate::None; + } + }; + + // Create the appropriate predicate based on operator + match op { + Operator::Equals => { + return FilterPredicate::Equals { column, value }; + } + Operator::NotEquals => { + return FilterPredicate::NotEquals { column, value }; + } + Operator::Greater => { + return FilterPredicate::GreaterThan { column, value }; + } + Operator::GreaterEquals => { + return FilterPredicate::GreaterThanOrEqual { column, value }; + } + Operator::Less => { + return FilterPredicate::LessThan { column, value }; + } + Operator::LessEquals => { + return FilterPredicate::LessThanOrEqual { column, value }; + } + _ => {} + } + } + } + + // Default to None for unsupported expressions + FilterPredicate::None + } + + /// Parse a WHERE clause from a SELECT statement + pub fn from_select(select: &turso_sqlite3_parser::ast::Select) -> Self { + use turso_sqlite3_parser::ast::*; + + if let OneSelect::Select(select_stmt) = &*select.body.select { + if let Some(where_clause) = &select_stmt.where_clause { + return Self::from_sql_expr(where_clause); + } + } + + FilterPredicate::None + } +} + +#[derive(Debug, Clone)] +pub enum ProjectColumn { + /// Direct column reference + Column(String), + /// Computed expression (simplified for now) + Expression { + expr: String, // For now, just store as string + alias: Option, + }, +} + +#[derive(Debug, Clone)] +pub enum JoinType { + Inner, + Left, + Right, +} + +#[derive(Debug, Clone)] +pub enum AggregateFunction { + Count, + Sum(String), + Avg(String), + Min(String), + Max(String), +} + +impl Display for AggregateFunction { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + AggregateFunction::Count => write!(f, "COUNT(*)"), + AggregateFunction::Sum(col) => write!(f, "SUM({col})"), + AggregateFunction::Avg(col) => write!(f, "AVG({col})"), + AggregateFunction::Min(col) => write!(f, "MIN({col})"), + AggregateFunction::Max(col) => write!(f, "MAX({col})"), + } + } +} + +impl AggregateFunction { + /// Get the default output column name for this aggregate function + #[inline] + pub fn default_output_name(&self) -> String { + self.to_string() + } + + /// Create an AggregateFunction from a SQL function and its arguments + /// Returns None if the function is not a supported aggregate + pub fn from_sql_function( + func: &crate::function::Func, + input_column: Option, + ) -> Option { + use crate::function::{AggFunc, Func}; + + match func { + Func::Agg(agg_func) => { + match agg_func { + AggFunc::Count | AggFunc::Count0 => Some(AggregateFunction::Count), + AggFunc::Sum => input_column.map(AggregateFunction::Sum), + AggFunc::Avg => input_column.map(AggregateFunction::Avg), + AggFunc::Min => input_column.map(AggregateFunction::Min), + AggFunc::Max => input_column.map(AggregateFunction::Max), + _ => None, // Other aggregate functions not yet supported in DBSP + } + } + _ => None, // Not an aggregate function + } + } +} + +/// Operator DAG (Directed Acyclic Graph) +/// Base trait for incremental operators +pub trait IncrementalOperator: Debug { + /// Initialize with base data + fn initialize(&mut self, data: Delta); + + /// Process a delta (incremental update) + fn process_delta(&mut self, delta: Delta) -> Delta; + + /// Get current accumulated state + fn get_current_state(&self) -> Delta; + + /// Set computation tracker + fn set_tracker(&mut self, tracker: Arc>); +} + +/// Filter operator - filters rows based on predicate +#[derive(Debug)] +pub struct FilterOperator { + predicate: FilterPredicate, + current_state: Delta, + column_names: Vec, + tracker: Option>>, +} + +impl FilterOperator { + pub fn new(predicate: FilterPredicate, column_names: Vec) -> Self { + Self { + predicate, + current_state: Delta::new(), + column_names, + tracker: None, + } + } + + /// Get the predicate for this filter + pub fn predicate(&self) -> &FilterPredicate { + &self.predicate + } + + pub fn evaluate_predicate(&self, values: &[Value]) -> bool { + match &self.predicate { + FilterPredicate::None => true, + FilterPredicate::Equals { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + return v == value; + } + } + false + } + FilterPredicate::NotEquals { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + return v != value; + } + } + false + } + FilterPredicate::GreaterThan { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + // Compare based on value types + match (v, value) { + (Value::Integer(a), Value::Integer(b)) => return a > b, + (Value::Float(a), Value::Float(b)) => return a > b, + (Value::Text(a), Value::Text(b)) => return a.as_str() > b.as_str(), + _ => {} + } + } + } + false + } + FilterPredicate::GreaterThanOrEqual { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + match (v, value) { + (Value::Integer(a), Value::Integer(b)) => return a >= b, + (Value::Float(a), Value::Float(b)) => return a >= b, + (Value::Text(a), Value::Text(b)) => return a.as_str() >= b.as_str(), + _ => {} + } + } + } + false + } + FilterPredicate::LessThan { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + match (v, value) { + (Value::Integer(a), Value::Integer(b)) => return a < b, + (Value::Float(a), Value::Float(b)) => return a < b, + (Value::Text(a), Value::Text(b)) => return a.as_str() < b.as_str(), + _ => {} + } + } + } + false + } + FilterPredicate::LessThanOrEqual { column, value } => { + if let Some(idx) = self.column_names.iter().position(|c| c == column) { + if let Some(v) = values.get(idx) { + match (v, value) { + (Value::Integer(a), Value::Integer(b)) => return a <= b, + (Value::Float(a), Value::Float(b)) => return a <= b, + (Value::Text(a), Value::Text(b)) => return a.as_str() <= b.as_str(), + _ => {} + } + } + } + false + } + FilterPredicate::And(left, right) => { + // Temporarily create sub-filters to evaluate + let left_filter = FilterOperator::new((**left).clone(), self.column_names.clone()); + let right_filter = + FilterOperator::new((**right).clone(), self.column_names.clone()); + left_filter.evaluate_predicate(values) && right_filter.evaluate_predicate(values) + } + FilterPredicate::Or(left, right) => { + let left_filter = FilterOperator::new((**left).clone(), self.column_names.clone()); + let right_filter = + FilterOperator::new((**right).clone(), self.column_names.clone()); + left_filter.evaluate_predicate(values) || right_filter.evaluate_predicate(values) + } + } + } +} + +impl IncrementalOperator for FilterOperator { + fn initialize(&mut self, data: Delta) { + // Process initial data through filter + for (row, weight) in data.changes { + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_filter(); + } + + if self.evaluate_predicate(&row.values) { + self.current_state.changes.push((row, weight)); + } + } + } + + fn process_delta(&mut self, delta: Delta) -> Delta { + let mut output_delta = Delta::new(); + + // Process only the delta, not the entire state + for (row, weight) in delta.changes { + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_filter(); + } + + if self.evaluate_predicate(&row.values) { + output_delta.changes.push((row.clone(), weight)); + + // Update our state + self.current_state.changes.push((row, weight)); + } + } + + output_delta + } + + fn get_current_state(&self) -> Delta { + self.current_state.clone() + } + + fn set_tracker(&mut self, tracker: Arc>) { + self.tracker = Some(tracker); + } +} + +/// Project operator - selects/transforms columns +#[derive(Debug, Clone)] +pub struct ProjectOperator { + columns: Vec, + input_column_names: Vec, + output_column_names: Vec, + current_state: Delta, + tracker: Option>>, +} + +impl ProjectOperator { + pub fn new(columns: Vec, input_column_names: Vec) -> Self { + let output_column_names = columns + .iter() + .map(|c| match c { + ProjectColumn::Column(name) => name.clone(), + ProjectColumn::Expression { alias, .. } => { + alias.clone().unwrap_or_else(|| "expr".to_string()) + } + }) + .collect(); + + Self { + columns, + input_column_names, + output_column_names, + current_state: Delta::new(), + tracker: None, + } + } + + /// Get the columns for this projection + pub fn columns(&self) -> &[ProjectColumn] { + &self.columns + } + + fn project_values(&self, values: &[Value]) -> Vec { + let mut output = Vec::new(); + + for col in &self.columns { + match col { + ProjectColumn::Column(name) => { + if let Some(idx) = self.input_column_names.iter().position(|c| c == name) { + if let Some(v) = values.get(idx) { + output.push(v.clone()); + } else { + output.push(Value::Null); + } + } else { + output.push(Value::Null); + } + } + ProjectColumn::Expression { .. } => { + // For now, just pass through a null + // In a real implementation, we'd evaluate the expression + output.push(Value::Null); + } + } + } + + output + } +} + +impl IncrementalOperator for ProjectOperator { + fn initialize(&mut self, data: Delta) { + for (row, weight) in &data.changes { + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_project(); + } + + let projected = self.project_values(&row.values); + let projected_row = HashableRow::new(row.rowid, projected); + self.current_state.changes.push((projected_row, *weight)); + } + } + + fn process_delta(&mut self, delta: Delta) -> Delta { + let mut output_delta = Delta::new(); + + for (row, weight) in &delta.changes { + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_project(); + } + + let projected = self.project_values(&row.values); + let projected_row = HashableRow::new(row.rowid, projected); + + output_delta.changes.push((projected_row.clone(), *weight)); + self.current_state.changes.push((projected_row, *weight)); + } + + output_delta + } + + fn get_current_state(&self) -> Delta { + self.current_state.clone() + } + + fn set_tracker(&mut self, tracker: Arc>) { + self.tracker = Some(tracker); + } +} + +/// Join operator - performs incremental joins using DBSP formula +/// ∂(A ⋈ B) = A ⋈ ∂B + ∂A ⋈ B + ∂A ⋈ ∂B +#[derive(Debug)] +pub struct JoinOperator { + join_type: JoinType, + pub left_on_column: String, + pub right_on_column: String, + left_column_names: Vec, + right_column_names: Vec, + // Current accumulated state for both sides + left_state: Delta, + right_state: Delta, + // Index for efficient lookups: column_value_as_string -> vec of row_keys + // We use String representation of values since Value doesn't implement Hash + left_index: HashMap>, + right_index: HashMap>, + // Result state + current_state: Delta, + tracker: Option>>, + // For generating unique keys for join results + next_result_key: i64, +} + +impl JoinOperator { + pub fn new( + join_type: JoinType, + left_on_column: String, + right_on_column: String, + left_column_names: Vec, + right_column_names: Vec, + ) -> Self { + Self { + join_type, + left_on_column, + right_on_column, + left_column_names, + right_column_names, + left_state: Delta::new(), + right_state: Delta::new(), + left_index: HashMap::new(), + right_index: HashMap::new(), + current_state: Delta::new(), + tracker: None, + next_result_key: 0, + } + } + + pub fn set_tracker(&mut self, tracker: Arc>) { + self.tracker = Some(tracker); + } + + /// Build index for a side of the join + fn build_index( + state: &Delta, + column_names: &[String], + on_column: &str, + ) -> HashMap> { + let mut index = HashMap::new(); + + // Find the column index + let col_idx = column_names.iter().position(|c| c == on_column); + if col_idx.is_none() { + return index; + } + let col_idx = col_idx.unwrap(); + + // Build the index + for (row, weight) in &state.changes { + // Include rows with positive weight in the index + if *weight > 0 { + if let Some(key_value) = row.values.get(col_idx) { + // Convert value to string for indexing + let key_str = format!("{key_value:?}"); + index + .entry(key_str) + .or_insert_with(Vec::new) + .push(row.rowid); + } + } + } + + index + } + + /// Join two deltas + fn join_deltas(&self, left_delta: &Delta, right_delta: &Delta, next_key: &mut i64) -> Delta { + let mut result = Delta::new(); + + // Find column indices + let left_col_idx = self + .left_column_names + .iter() + .position(|c| c == &self.left_on_column) + .unwrap_or(0); + let right_col_idx = self + .right_column_names + .iter() + .position(|c| c == &self.right_on_column) + .unwrap_or(0); + + // For each row in left_delta + for (left_row, left_weight) in &left_delta.changes { + // Process both inserts and deletes + + let left_join_value = left_row.values.get(left_col_idx); + if left_join_value.is_none() { + continue; + } + let left_join_value = left_join_value.unwrap(); + + // Look up matching rows in right_delta + for (right_row, right_weight) in &right_delta.changes { + // Process both inserts and deletes + + let right_join_value = right_row.values.get(right_col_idx); + if right_join_value.is_none() { + continue; + } + let right_join_value = right_join_value.unwrap(); + + // Check if values match + if left_join_value == right_join_value { + // Record the join lookup + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_join_lookup(); + } + + // Create joined row + let mut joined_values = left_row.values.clone(); + joined_values.extend(right_row.values.clone()); + + // Generate a unique key for the result + let result_key = *next_key; + *next_key += 1; + + let joined_row = HashableRow::new(result_key, joined_values); + result + .changes + .push((joined_row, left_weight * right_weight)); + } + } + } + + result + } + + /// Join a delta with the full state using the index + fn join_delta_with_state( + &self, + delta: &Delta, + state: &Delta, + delta_on_left: bool, + next_key: &mut i64, + ) -> Delta { + let mut result = Delta::new(); + + let (delta_col_idx, state_col_names) = if delta_on_left { + ( + self.left_column_names + .iter() + .position(|c| c == &self.left_on_column) + .unwrap_or(0), + &self.right_column_names, + ) + } else { + ( + self.right_column_names + .iter() + .position(|c| c == &self.right_on_column) + .unwrap_or(0), + &self.left_column_names, + ) + }; + + // Use index for efficient lookup + let state_index = Self::build_index( + state, + state_col_names, + if delta_on_left { + &self.right_on_column + } else { + &self.left_on_column + }, + ); + + for (delta_row, delta_weight) in &delta.changes { + // Process both inserts and deletes + + let delta_join_value = delta_row.values.get(delta_col_idx); + if delta_join_value.is_none() { + continue; + } + let delta_join_value = delta_join_value.unwrap(); + + // Use index to find matching rows + let delta_key_str = format!("{delta_join_value:?}"); + if let Some(matching_keys) = state_index.get(&delta_key_str) { + for state_key in matching_keys { + // Look up in the state - find the row with this rowid + let state_row_opt = state + .changes + .iter() + .find(|(row, weight)| row.rowid == *state_key && *weight > 0); + + if let Some((state_row, state_weight)) = state_row_opt { + // Record the join lookup + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_join_lookup(); + } + + // Create joined row + let joined_values = if delta_on_left { + let mut v = delta_row.values.clone(); + v.extend(state_row.values.clone()); + v + } else { + let mut v = state_row.values.clone(); + v.extend(delta_row.values.clone()); + v + }; + + let result_key = *next_key; + *next_key += 1; + + let joined_row = HashableRow::new(result_key, joined_values); + result + .changes + .push((joined_row, delta_weight * state_weight)); + } + } + } + } + + result + } + + /// Initialize both sides of the join + pub fn initialize_both(&mut self, left_data: Delta, right_data: Delta) { + self.left_state = left_data.clone(); + self.right_state = right_data.clone(); + + // Build indices + self.left_index = Self::build_index( + &self.left_state, + &self.left_column_names, + &self.left_on_column, + ); + self.right_index = Self::build_index( + &self.right_state, + &self.right_column_names, + &self.right_on_column, + ); + + // Perform initial join + let mut next_key = self.next_result_key; + self.current_state = self.join_deltas(&self.left_state, &self.right_state, &mut next_key); + self.next_result_key = next_key; + } + + /// Process deltas for both sides using DBSP formula + /// ∂(A ⋈ B) = A ⋈ ∂B + ∂A ⋈ B + ∂A ⋈ ∂B + pub fn process_both_deltas(&mut self, left_delta: Delta, right_delta: Delta) -> Delta { + let mut result = Delta::new(); + let mut next_key = self.next_result_key; + + // A ⋈ ∂B (existing left with new right) + let a_join_db = + self.join_delta_with_state(&right_delta, &self.left_state, false, &mut next_key); + result.merge(&a_join_db); + + // ∂A ⋈ B (new left with existing right) + let da_join_b = + self.join_delta_with_state(&left_delta, &self.right_state, true, &mut next_key); + result.merge(&da_join_b); + + // ∂A ⋈ ∂B (new left with new right) + let da_join_db = self.join_deltas(&left_delta, &right_delta, &mut next_key); + result.merge(&da_join_db); + + // Update the next key counter + self.next_result_key = next_key; + + // Update state + self.left_state.merge(&left_delta); + self.right_state.merge(&right_delta); + self.current_state.merge(&result); + + // Rebuild indices if needed + self.left_index = Self::build_index( + &self.left_state, + &self.left_column_names, + &self.left_on_column, + ); + self.right_index = Self::build_index( + &self.right_state, + &self.right_column_names, + &self.right_on_column, + ); + + result + } + + pub fn get_current_state(&self) -> &Delta { + &self.current_state + } + + /// Process a delta from the left table only + pub fn process_left_delta(&mut self, left_delta: Delta) -> Delta { + let empty_delta = Delta::new(); + self.process_both_deltas(left_delta, empty_delta) + } + + /// Process a delta from the right table only + pub fn process_right_delta(&mut self, right_delta: Delta) -> Delta { + let empty_delta = Delta::new(); + self.process_both_deltas(empty_delta, right_delta) + } +} + +/// Aggregate operator - performs incremental aggregation with GROUP BY +/// Maintains running totals/counts that are updated incrementally +#[derive(Debug, Clone)] +pub struct AggregateOperator { + // GROUP BY columns + group_by: Vec, + // Aggregate functions to compute + aggregates: Vec, + // Column names from input + pub input_column_names: Vec, + // Aggregation state: group_key_str -> aggregate values + // For each group, we store the aggregate results + // We use String representation of group keys since Value doesn't implement Hash + group_states: HashMap, + // Map to keep track of actual group key values for output + group_key_values: HashMap>, + // Current output state as a Delta + current_state: Delta, + tracker: Option>>, +} + +/// State for a single group's aggregates +#[derive(Debug, Clone)] +struct AggregateState { + // For COUNT: just the count + count: i64, + // For SUM: column_name -> sum value + sums: HashMap, + // For AVG: column_name -> (sum, count) for computing average + avgs: HashMap, + // For MIN: column_name -> min value + mins: HashMap, + // For MAX: column_name -> max value + maxs: HashMap, +} + +impl AggregateState { + fn new() -> Self { + Self { + count: 0, + sums: HashMap::new(), + avgs: HashMap::new(), + mins: HashMap::new(), + maxs: HashMap::new(), + } + } + + /// Apply a delta to this aggregate state + fn apply_delta( + &mut self, + values: &[Value], + weight: isize, + aggregates: &[AggregateFunction], + column_names: &[String], + ) { + // Update COUNT + self.count += weight as i64; + + // Update other aggregates + for agg in aggregates { + match agg { + AggregateFunction::Count => { + // Already handled above + } + AggregateFunction::Sum(col_name) => { + if let Some(idx) = column_names.iter().position(|c| c == col_name) { + if let Some(val) = values.get(idx) { + let num_val = match val { + Value::Integer(i) => *i as f64, + Value::Float(f) => *f, + _ => 0.0, + }; + *self.sums.entry(col_name.clone()).or_insert(0.0) += + num_val * weight as f64; + } + } + } + AggregateFunction::Avg(col_name) => { + if let Some(idx) = column_names.iter().position(|c| c == col_name) { + if let Some(val) = values.get(idx) { + let num_val = match val { + Value::Integer(i) => *i as f64, + Value::Float(f) => *f, + _ => 0.0, + }; + let (sum, count) = + self.avgs.entry(col_name.clone()).or_insert((0.0, 0)); + *sum += num_val * weight as f64; + *count += weight as i64; + } + } + } + AggregateFunction::Min(col_name) => { + // MIN/MAX are more complex for incremental updates + // For now, we'll need to recompute from the full state + // This is a limitation we can improve later + if weight > 0 { + // Only update on insert + if let Some(idx) = column_names.iter().position(|c| c == col_name) { + if let Some(val) = values.get(idx) { + self.mins + .entry(col_name.clone()) + .and_modify(|existing| { + if val < existing { + *existing = val.clone(); + } + }) + .or_insert_with(|| val.clone()); + } + } + } + } + AggregateFunction::Max(col_name) => { + if weight > 0 { + // Only update on insert + if let Some(idx) = column_names.iter().position(|c| c == col_name) { + if let Some(val) = values.get(idx) { + self.maxs + .entry(col_name.clone()) + .and_modify(|existing| { + if val > existing { + *existing = val.clone(); + } + }) + .or_insert_with(|| val.clone()); + } + } + } + } + } + } + } + + /// Convert aggregate state to output values + fn to_values(&self, aggregates: &[AggregateFunction]) -> Vec { + let mut result = Vec::new(); + + for agg in aggregates { + match agg { + AggregateFunction::Count => { + result.push(Value::Integer(self.count)); + } + AggregateFunction::Sum(col_name) => { + let sum = self.sums.get(col_name).copied().unwrap_or(0.0); + // Return as integer if it's a whole number, otherwise as float + if sum.fract() == 0.0 { + result.push(Value::Integer(sum as i64)); + } else { + result.push(Value::Float(sum)); + } + } + AggregateFunction::Avg(col_name) => { + if let Some((sum, count)) = self.avgs.get(col_name) { + if *count > 0 { + result.push(Value::Float(sum / *count as f64)); + } else { + result.push(Value::Null); + } + } else { + result.push(Value::Null); + } + } + AggregateFunction::Min(col_name) => { + result.push(self.mins.get(col_name).cloned().unwrap_or(Value::Null)); + } + AggregateFunction::Max(col_name) => { + result.push(self.maxs.get(col_name).cloned().unwrap_or(Value::Null)); + } + } + } + + result + } +} + +impl AggregateOperator { + pub fn new( + group_by: Vec, + aggregates: Vec, + input_column_names: Vec, + ) -> Self { + Self { + group_by, + aggregates, + input_column_names, + group_states: HashMap::new(), + group_key_values: HashMap::new(), + current_state: Delta::new(), + tracker: None, + } + } + + pub fn set_tracker(&mut self, tracker: Arc>) { + self.tracker = Some(tracker); + } + + /// Extract group key values from a row + fn extract_group_key(&self, values: &[Value]) -> Vec { + let mut key = Vec::new(); + + for group_col in &self.group_by { + if let Some(idx) = self.input_column_names.iter().position(|c| c == group_col) { + if let Some(val) = values.get(idx) { + key.push(val.clone()); + } else { + key.push(Value::Null); + } + } else { + key.push(Value::Null); + } + } + + key + } + + /// Convert group key to string for indexing (since Value doesn't implement Hash) + fn group_key_to_string(key: &[Value]) -> String { + key.iter() + .map(|v| format!("{v:?}")) + .collect::>() + .join(",") + } + + /// Process a delta and update aggregate state incrementally + pub fn process_delta(&mut self, delta: Delta) -> Delta { + let mut output_delta = Delta::new(); + + // Track which groups were modified + let mut modified_groups = HashSet::new(); + + // Process each change in the delta + for (row, weight) in &delta.changes { + if let Some(tracker) = &self.tracker { + tracker.lock().unwrap().record_aggregation(); + } + + // Extract group key + let group_key = self.extract_group_key(&row.values); + let group_key_str = Self::group_key_to_string(&group_key); + modified_groups.insert(group_key_str.clone()); + + // Store the actual group key values + self.group_key_values + .insert(group_key_str.clone(), group_key.clone()); + + // Get or create aggregate state for this group + let state = self + .group_states + .entry(group_key_str.clone()) + .or_insert_with(AggregateState::new); + + // Apply the delta to the aggregate state + state.apply_delta( + &row.values, + *weight, + &self.aggregates, + &self.input_column_names, + ); + } + + // Generate output delta for modified groups + for group_key_str in modified_groups { + // Get the actual group key values + let group_key = self + .group_key_values + .get(&group_key_str) + .cloned() + .unwrap_or_default(); + + if let Some(state) = self.group_states.get(&group_key_str) { + // Build output row: group_by columns + aggregate values + let mut output_values = group_key.clone(); + output_values.extend(state.to_values(&self.aggregates)); + + // Generate a unique key for this group + // We use a hash of the group key to ensure consistency + let result_key = group_key_str + .bytes() + .fold(0i64, |acc, b| acc.wrapping_mul(31).wrapping_add(b as i64)); + + // Check if group should be removed (count is 0) + if state.count > 0 { + // Add to output delta with positive weight + let output_row = HashableRow::new(result_key, output_values.clone()); + output_delta.changes.push((output_row.clone(), 1)); + + // Update current state + self.current_state.changes.push((output_row, 1)); + } else { + // Add to output delta with negative weight (deletion) + let output_row = HashableRow::new(result_key, output_values); + output_delta.changes.push((output_row.clone(), -1)); + + // Mark for removal in current state + self.current_state.changes.push((output_row, -1)); + self.group_states.remove(&group_key_str); + self.group_key_values.remove(&group_key_str); + } + } + } + + // Consolidate current state to handle removals + self.current_state.consolidate(); + + output_delta + } + + pub fn get_current_state(&self) -> &Delta { + &self.current_state + } +} + +impl IncrementalOperator for AggregateOperator { + fn initialize(&mut self, data: Delta) { + // Process all initial data + self.process_delta(data); + } + + fn process_delta(&mut self, delta: Delta) -> Delta { + self.process_delta(delta) + } + + fn get_current_state(&self) -> Delta { + self.current_state.clone() + } + + fn set_tracker(&mut self, tracker: Arc>) { + self.tracker = Some(tracker); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::types::Text; + use crate::Value; + use std::sync::{Arc, Mutex}; + + /// Assert that we're doing incremental work, not full recomputation + fn assert_incremental(tracker: &ComputationTracker, expected_ops: usize, data_size: usize) { + assert!( + tracker.total_computations() <= expected_ops, + "Expected <= {} operations for incremental update, got {}", + expected_ops, + tracker.total_computations() + ); + assert!( + tracker.total_computations() < data_size, + "Computation count {} suggests full recomputation (data size: {})", + tracker.total_computations(), + data_size + ); + assert_eq!( + tracker.full_scans, 0, + "Incremental computation should not perform full scans" + ); + } + + // Join tests + #[test] + fn test_join_uses_delta_formula() { + let tracker = Arc::new(Mutex::new(ComputationTracker::new())); + + // Create join operator + let mut join = JoinOperator::new( + JoinType::Inner, + "user_id".to_string(), + "user_id".to_string(), + vec!["user_id".to_string(), "email".to_string()], + vec![ + "login_id".to_string(), + "user_id".to_string(), + "timestamp".to_string(), + ], + ); + join.set_tracker(tracker.clone()); + + // Initial data: emails table + let mut emails = Delta::new(); + emails.insert( + 1, + vec![ + Value::Integer(1), + Value::Text(Text::new("alice@example.com")), + ], + ); + emails.insert( + 2, + vec![Value::Integer(2), Value::Text(Text::new("bob@example.com"))], + ); + + // Initial data: logins table + let mut logins = Delta::new(); + logins.insert( + 1, + vec![Value::Integer(1), Value::Integer(1), Value::Integer(1000)], + ); + logins.insert( + 2, + vec![Value::Integer(2), Value::Integer(1), Value::Integer(2000)], + ); + + // Initialize join + join.initialize_both(emails.clone(), logins.clone()); + + // Reset tracker for delta processing + tracker.lock().unwrap().join_lookups = 0; + + // Add one login for bob (user_id=2) + let mut delta_logins = Delta::new(); + delta_logins.insert( + 3, + vec![Value::Integer(3), Value::Integer(2), Value::Integer(3000)], + ); + + // Process delta - should use incremental formula + let empty_delta = Delta::new(); + let output = join.process_both_deltas(empty_delta, delta_logins); + + // Should have one join result (bob's new login) + assert_eq!(output.len(), 1); + + // Verify we used index lookups, not nested loops + // Should have done 1 lookup (finding bob's email for the new login) + let lookups = tracker.lock().unwrap().join_lookups; + assert_eq!(lookups, 1, "Should use index lookup, not scan all emails"); + + // Verify incremental behavior - we processed only the delta + let t = tracker.lock().unwrap(); + assert_incremental(&t, 1, 3); // 1 operation for 3 total rows + } + + #[test] + fn test_join_maintains_index() { + // Create join operator + let mut join = JoinOperator::new( + JoinType::Inner, + "id".to_string(), + "ref_id".to_string(), + vec!["id".to_string(), "name".to_string()], + vec!["ref_id".to_string(), "value".to_string()], + ); + + // Initial data + let mut left = Delta::new(); + left.insert(1, vec![Value::Integer(1), Value::Text(Text::new("A"))]); + left.insert(2, vec![Value::Integer(2), Value::Text(Text::new("B"))]); + + let mut right = Delta::new(); + right.insert(1, vec![Value::Integer(1), Value::Integer(100)]); + + // Initialize - should build index + join.initialize_both(left.clone(), right.clone()); + + // Verify initial join worked + let state = join.get_current_state(); + assert_eq!(state.changes.len(), 1); // One match: id=1 + + // Add new item to left + let mut delta_left = Delta::new(); + delta_left.insert(3, vec![Value::Integer(3), Value::Text(Text::new("C"))]); + + // Add matching item to right + let mut delta_right = Delta::new(); + delta_right.insert(2, vec![Value::Integer(3), Value::Integer(300)]); + + // Process deltas + let output = join.process_both_deltas(delta_left, delta_right); + + // Should have new join result + assert_eq!(output.len(), 1); + + // Verify the join result has the expected values + assert!(!output.changes.is_empty()); + let (result, _weight) = &output.changes[0]; + assert_eq!(result.values.len(), 4); // id, name, ref_id, value + } + + #[test] + fn test_join_formula_correctness() { + // Test the DBSP formula: ∂(A ⋈ B) = A ⋈ ∂B + ∂A ⋈ B + ∂A ⋈ ∂B + let tracker = Arc::new(Mutex::new(ComputationTracker::new())); + + let mut join = JoinOperator::new( + JoinType::Inner, + "x".to_string(), + "x".to_string(), + vec!["x".to_string(), "a".to_string()], + vec!["x".to_string(), "b".to_string()], + ); + join.set_tracker(tracker.clone()); + + // Initial state A + let mut a = Delta::new(); + a.insert(1, vec![Value::Integer(1), Value::Text(Text::new("a1"))]); + a.insert(2, vec![Value::Integer(2), Value::Text(Text::new("a2"))]); + + // Initial state B + let mut b = Delta::new(); + b.insert(1, vec![Value::Integer(1), Value::Text(Text::new("b1"))]); + b.insert(2, vec![Value::Integer(2), Value::Text(Text::new("b2"))]); + + join.initialize_both(a.clone(), b.clone()); + + // Reset tracker + tracker.lock().unwrap().join_lookups = 0; + + // Delta for A (add x=3) + let mut delta_a = Delta::new(); + delta_a.insert(3, vec![Value::Integer(3), Value::Text(Text::new("a3"))]); + + // Delta for B (add x=3 and x=1) + let mut delta_b = Delta::new(); + delta_b.insert(3, vec![Value::Integer(3), Value::Text(Text::new("b3"))]); + delta_b.insert(4, vec![Value::Integer(1), Value::Text(Text::new("b1_new"))]); + + let output = join.process_both_deltas(delta_a, delta_b); + + // Expected results: + // A ⋈ ∂B: (1,a1) ⋈ (1,b1_new) = 1 result + // ∂A ⋈ B: (3,a3) ⋈ nothing = 0 results + // ∂A ⋈ ∂B: (3,a3) ⋈ (3,b3) = 1 result + // Total: 2 results + assert_eq!(output.len(), 2); + + // Verify we're doing incremental work + let lookups = tracker.lock().unwrap().join_lookups; + assert!(lookups <= 4, "Should use efficient index lookups"); + } + + // Aggregation tests + #[test] + fn test_count_increments_not_recounts() { + let tracker = Arc::new(Mutex::new(ComputationTracker::new())); + + // Create COUNT(*) GROUP BY category + let mut agg = AggregateOperator::new( + vec!["category".to_string()], + vec![AggregateFunction::Count], + vec![ + "item_id".to_string(), + "category".to_string(), + "price".to_string(), + ], + ); + agg.set_tracker(tracker.clone()); + + // Initial: 100 items in 10 categories (10 items each) + let mut initial = Delta::new(); + for i in 0..100 { + let category = format!("cat_{}", i / 10); + initial.insert( + i, + vec![ + Value::Integer(i), + Value::Text(Text::new(&category)), + Value::Integer(i * 10), + ], + ); + } + agg.initialize(initial); + + // Reset tracker for delta processing + tracker.lock().unwrap().aggregation_updates = 0; + + // Add one item to category 'cat_0' + let mut delta = Delta::new(); + delta.insert( + 100, + vec![ + Value::Integer(100), + Value::Text(Text::new("cat_0")), + Value::Integer(1000), + ], + ); + + let output = agg.process_delta(delta); + + // Should only update one group (cat_0), not recount all groups + assert_eq!(tracker.lock().unwrap().aggregation_updates, 1); + + // Output should show cat_0 now has count 11 + assert_eq!(output.len(), 1); + assert!(!output.changes.is_empty()); + let (change_row, _weight) = &output.changes[0]; + assert_eq!(change_row.values[0], Value::Text(Text::new("cat_0"))); + assert_eq!(change_row.values[1], Value::Integer(11)); + + // Verify incremental behavior + let t = tracker.lock().unwrap(); + assert_incremental(&t, 1, 101); + } + + #[test] + fn test_sum_updates_incrementally() { + let tracker = Arc::new(Mutex::new(ComputationTracker::new())); + + // Create SUM(amount) GROUP BY product + let mut agg = AggregateOperator::new( + vec!["product".to_string()], + vec![AggregateFunction::Sum("amount".to_string())], + vec![ + "sale_id".to_string(), + "product".to_string(), + "amount".to_string(), + ], + ); + agg.set_tracker(tracker.clone()); + + // Initial sales + let mut initial = Delta::new(); + initial.insert( + 1, + vec![ + Value::Integer(1), + Value::Text(Text::new("Widget")), + Value::Integer(100), + ], + ); + initial.insert( + 2, + vec![ + Value::Integer(2), + Value::Text(Text::new("Gadget")), + Value::Integer(200), + ], + ); + initial.insert( + 3, + vec![ + Value::Integer(3), + Value::Text(Text::new("Widget")), + Value::Integer(150), + ], + ); + agg.initialize(initial); + + // Check initial state: Widget=250, Gadget=200 + let state = agg.get_current_state(); + let widget_sum = state + .changes + .iter() + .find(|(c, _)| c.values[0] == Value::Text(Text::new("Widget"))) + .map(|(c, _)| c) + .unwrap(); + assert_eq!(widget_sum.values[1], Value::Integer(250)); + + // Reset tracker + tracker.lock().unwrap().aggregation_updates = 0; + + // Add sale of 50 for Widget + let mut delta = Delta::new(); + delta.insert( + 4, + vec![ + Value::Integer(4), + Value::Text(Text::new("Widget")), + Value::Integer(50), + ], + ); + + let output = agg.process_delta(delta); + + // Should only update Widget group + assert_eq!(tracker.lock().unwrap().aggregation_updates, 1); + assert_eq!(output.len(), 1); + + // Widget should now be 300 (250 + 50) + assert!(!output.changes.is_empty()); + let (change, _weight) = &output.changes[0]; + assert_eq!(change.values[0], Value::Text(Text::new("Widget"))); + assert_eq!(change.values[1], Value::Integer(300)); + } + + #[test] + fn test_count_and_sum_together() { + // Test the example from DBSP_ROADMAP: COUNT(*) and SUM(amount) GROUP BY user_id + let mut agg = AggregateOperator::new( + vec!["user_id".to_string()], + vec![ + AggregateFunction::Count, + AggregateFunction::Sum("amount".to_string()), + ], + vec![ + "order_id".to_string(), + "user_id".to_string(), + "amount".to_string(), + ], + ); + + // Initial orders + let mut initial = Delta::new(); + initial.insert( + 1, + vec![Value::Integer(1), Value::Integer(1), Value::Integer(100)], + ); + initial.insert( + 2, + vec![Value::Integer(2), Value::Integer(1), Value::Integer(200)], + ); + initial.insert( + 3, + vec![Value::Integer(3), Value::Integer(2), Value::Integer(150)], + ); + agg.initialize(initial); + + // Check initial state + // User 1: count=2, sum=300 + // User 2: count=1, sum=150 + let state = agg.get_current_state(); + assert_eq!(state.changes.len(), 2); + + let user1 = state + .changes + .iter() + .find(|(c, _)| c.values[0] == Value::Integer(1)) + .map(|(c, _)| c) + .unwrap(); + assert_eq!(user1.values[1], Value::Integer(2)); // count + assert_eq!(user1.values[2], Value::Integer(300)); // sum + + let user2 = state + .changes + .iter() + .find(|(c, _)| c.values[0] == Value::Integer(2)) + .map(|(c, _)| c) + .unwrap(); + assert_eq!(user2.values[1], Value::Integer(1)); // count + assert_eq!(user2.values[2], Value::Integer(150)); // sum + + // Add order for user 1 + let mut delta = Delta::new(); + delta.insert( + 4, + vec![Value::Integer(4), Value::Integer(1), Value::Integer(50)], + ); + let output = agg.process_delta(delta); + + // Should only update user 1 + assert_eq!(output.len(), 1); + assert!(!output.changes.is_empty()); + let (change, _weight) = &output.changes[0]; + assert_eq!(change.values[0], Value::Integer(1)); // user_id + assert_eq!(change.values[1], Value::Integer(3)); // count: 2 + 1 + assert_eq!(change.values[2], Value::Integer(350)); // sum: 300 + 50 + } + + #[test] + fn test_avg_maintains_sum_and_count() { + // Test AVG aggregation + let mut agg = AggregateOperator::new( + vec!["category".to_string()], + vec![AggregateFunction::Avg("value".to_string())], + vec![ + "id".to_string(), + "category".to_string(), + "value".to_string(), + ], + ); + + // Initial data + let mut initial = Delta::new(); + initial.insert( + 1, + vec![ + Value::Integer(1), + Value::Text(Text::new("A")), + Value::Integer(10), + ], + ); + initial.insert( + 2, + vec![ + Value::Integer(2), + Value::Text(Text::new("A")), + Value::Integer(20), + ], + ); + initial.insert( + 3, + vec![ + Value::Integer(3), + Value::Text(Text::new("B")), + Value::Integer(30), + ], + ); + agg.initialize(initial); + + // Check initial averages + // Category A: avg = (10 + 20) / 2 = 15 + // Category B: avg = 30 / 1 = 30 + let state = agg.get_current_state(); + let cat_a = state + .changes + .iter() + .find(|(c, _)| c.values[0] == Value::Text(Text::new("A"))) + .map(|(c, _)| c) + .unwrap(); + assert_eq!(cat_a.values[1], Value::Float(15.0)); + + let cat_b = state + .changes + .iter() + .find(|(c, _)| c.values[0] == Value::Text(Text::new("B"))) + .map(|(c, _)| c) + .unwrap(); + assert_eq!(cat_b.values[1], Value::Float(30.0)); + + // Add value to category A + let mut delta = Delta::new(); + delta.insert( + 4, + vec![ + Value::Integer(4), + Value::Text(Text::new("A")), + Value::Integer(30), + ], + ); + let output = agg.process_delta(delta); + + // Category A avg should now be (10 + 20 + 30) / 3 = 20 + assert!(!output.changes.is_empty()); + let (change, _weight) = &output.changes[0]; + assert_eq!(change.values[0], Value::Text(Text::new("A"))); + assert_eq!(change.values[1], Value::Float(20.0)); + } + + #[test] + fn test_delete_updates_aggregates() { + // Test that deletes (negative weights) properly update aggregates + let mut agg = AggregateOperator::new( + vec!["category".to_string()], + vec![ + AggregateFunction::Count, + AggregateFunction::Sum("value".to_string()), + ], + vec![ + "id".to_string(), + "category".to_string(), + "value".to_string(), + ], + ); + + // Initial data + let mut initial = Delta::new(); + initial.insert( + 1, + vec![ + Value::Integer(1), + Value::Text(Text::new("A")), + Value::Integer(100), + ], + ); + initial.insert( + 2, + vec![ + Value::Integer(2), + Value::Text(Text::new("A")), + Value::Integer(200), + ], + ); + agg.initialize(initial); + + // Check initial state: count=2, sum=300 + let state = agg.get_current_state(); + assert!(!state.changes.is_empty()); + let (row, _weight) = &state.changes[0]; + assert_eq!(row.values[1], Value::Integer(2)); // count + assert_eq!(row.values[2], Value::Integer(300)); // sum + + // Delete one row + let mut delta = Delta::new(); + delta.delete( + 1, + vec![ + Value::Integer(1), + Value::Text(Text::new("A")), + Value::Integer(100), + ], + ); + + let output = agg.process_delta(delta); + + // Should update to count=1, sum=200 + assert!(!output.changes.is_empty()); + let (change_row, _weight) = &output.changes[0]; + assert_eq!(change_row.values[0], Value::Text(Text::new("A"))); + assert_eq!(change_row.values[1], Value::Integer(1)); // count: 2 - 1 + assert_eq!(change_row.values[2], Value::Integer(200)); // sum: 300 - 100 + } +} diff --git a/core/lib.rs b/core/lib.rs index 648037c6a..94956f83a 100644 --- a/core/lib.rs +++ b/core/lib.rs @@ -6,6 +6,7 @@ mod ext; mod fast_lock; mod function; mod functions; +mod incremental; mod info; mod io; #[cfg(feature = "json")] From 145d6eede797df411b894b8f744973f000d3e8e3 Mon Sep 17 00:00:00 2001 From: Glauber Costa Date: Fri, 8 Aug 2025 13:36:34 -0500 Subject: [PATCH 03/10] Implement very basic views using DBSP This is just the bare minimum that I needed to convince myself that this approach will work. The only views that we support are slices of the main table: no aggregations, no joins, no projections. drop view is implemented. view population is implemented. deletes, inserts and updates are implemented. much like indexes before, a flag must be passed to enable views. --- bindings/go/rs_src/lib.rs | 2 +- bindings/python/src/lib.rs | 2 +- cli/app.rs | 74 +- core/incremental/mod.rs | 1 + core/incremental/view.rs | 1193 +++++++++++++++++ core/lib.rs | 127 +- core/schema.rs | 124 ++ core/storage/pager.rs | 2 +- core/translate/emitter.rs | 4 +- core/translate/mod.rs | 24 +- core/translate/planner.rs | 33 +- core/translate/pragma.rs | 82 +- core/translate/schema.rs | 2 + core/translate/view.rs | 221 +++ core/util.rs | 229 +++- core/vdbe/execute.rs | 178 ++- core/vdbe/explain.rs | 18 + core/vdbe/insn.rs | 16 + core/vdbe/mod.rs | 17 + core/vtab.rs | 30 +- core/vtab_view.rs | 101 ++ tests/integration/common.rs | 3 + .../query_processing/test_write_path.rs | 1 + 23 files changed, 2409 insertions(+), 75 deletions(-) create mode 100644 core/incremental/view.rs create mode 100644 core/translate/view.rs create mode 100644 core/vtab_view.rs diff --git a/bindings/go/rs_src/lib.rs b/bindings/go/rs_src/lib.rs index e226f9ca8..8bb96022a 100644 --- a/bindings/go/rs_src/lib.rs +++ b/bindings/go/rs_src/lib.rs @@ -20,7 +20,7 @@ pub unsafe extern "C" fn db_open(path: *const c_char) -> *mut c_void { } let path = unsafe { std::ffi::CStr::from_ptr(path) }; let path = path.to_str().unwrap(); - let Ok((io, conn)) = Connection::from_uri(path, false, false) else { + let Ok((io, conn)) = Connection::from_uri(path, false, false, false) else { panic!("Failed to open connection with path: {path}"); }; LimboConn::new(conn, io).to_ptr() diff --git a/bindings/python/src/lib.rs b/bindings/python/src/lib.rs index cac0a7280..5e4b51ca4 100644 --- a/bindings/python/src/lib.rs +++ b/bindings/python/src/lib.rs @@ -318,7 +318,7 @@ impl Drop for Connection { #[pyfunction(signature = (path, experimental_indexes=None))] pub fn connect(path: &str, experimental_indexes: Option) -> Result { let experimental_indexes = experimental_indexes.unwrap_or(true); - match turso_core::Connection::from_uri(path, experimental_indexes, false) { + match turso_core::Connection::from_uri(path, experimental_indexes, false, false) { Ok((io, conn)) => Ok(Connection { conn, _io: io }), Err(e) => Err(PyErr::new::(format!( "Failed to create connection: {e:?}" diff --git a/cli/app.rs b/cli/app.rs index 480e7e63f..8d82cd46a 100644 --- a/cli/app.rs +++ b/cli/app.rs @@ -60,6 +60,8 @@ pub struct Opts { pub readonly: bool, #[clap(long, help = "Enable experimental MVCC feature")] pub experimental_mvcc: bool, + #[clap(long, help = "Enable experimental views feature")] + pub experimental_views: bool, #[clap(long, help = "Enable experimental indexing feature")] pub experimental_indexes: Option, #[clap(short = 't', long, help = "specify output file for log traces")] @@ -121,7 +123,12 @@ impl Limbo { .map_or(":memory:".to_string(), |p| p.to_string_lossy().to_string()); let indexes_enabled = opts.experimental_indexes.unwrap_or(true); let (io, conn) = if db_file.contains([':', '?', '&', '#']) { - Connection::from_uri(&db_file, indexes_enabled, opts.experimental_mvcc)? + Connection::from_uri( + &db_file, + indexes_enabled, + opts.experimental_mvcc, + opts.experimental_views, + )? } else { let flags = if opts.readonly { OpenFlags::ReadOnly @@ -134,6 +141,7 @@ impl Limbo { flags, indexes_enabled, opts.experimental_mvcc, + opts.experimental_views, )?; let conn = db.connect()?; (io, conn) @@ -951,7 +959,11 @@ impl Limbo { } fn print_schema_entry(&mut self, db_display_name: &str, row: &turso_core::Row) -> bool { - if let Ok(Value::Text(schema)) = row.get::<&Value>(0) { + if let (Ok(Value::Text(schema)), Ok(Value::Text(obj_type)), Ok(Value::Text(obj_name))) = ( + row.get::<&Value>(0), + row.get::<&Value>(1), + row.get::<&Value>(2), + ) { let modified_schema = if db_display_name == "main" { schema.as_str().to_string() } else { @@ -982,12 +994,64 @@ impl Limbo { } }; let _ = self.write_fmt(format_args!("{modified_schema};")); + // For views, add the column comment like SQLite does + if obj_type.as_str() == "view" { + let columns = self + .get_view_columns(obj_name.as_str()) + .unwrap_or_else(|_| "x".to_string()); + let _ = self.write_fmt(format_args!("/* {}({}) */", obj_name.as_str(), columns)); + } true } else { false } } + /// Get column names for a view to generate the SQLite-compatible comment + fn get_view_columns(&mut self, view_name: &str) -> anyhow::Result { + // Get column information using PRAGMA table_info + let pragma_sql = format!("PRAGMA table_info({view_name})"); + + match self.conn.query(&pragma_sql) { + Ok(Some(ref mut rows)) => { + let mut columns = Vec::new(); + loop { + match rows.step()? { + StepResult::Row => { + let row = rows.row().unwrap(); + // Column name is in the second column (index 1) of PRAGMA table_info + if let Ok(Value::Text(col_name)) = row.get::<&Value>(1) { + columns.push(col_name.as_str().to_string()); + } + } + StepResult::IO => { + rows.run_once()?; + } + StepResult::Done => break, + StepResult::Interrupt => break, + StepResult::Busy => break, + } + } + + if columns.is_empty() { + anyhow::bail!("PRAGMA table_info returned no columns for view '{}'. The view may be corrupted or the database schema is invalid.", view_name); + } + + Ok(columns.join(",")) + } + Ok(None) => { + anyhow::bail!("PRAGMA table_info('{}') returned no results. The view may not exist or the database schema is invalid.", view_name); + } + Err(e) => { + anyhow::bail!( + "Failed to execute PRAGMA table_info for view '{}': {}", + view_name, + e + ); + } + } + } + fn query_one_table_schema( &mut self, db_prefix: &str, @@ -995,7 +1059,7 @@ impl Limbo { table_name: &str, ) -> anyhow::Result { let sql = format!( - "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index') AND tbl_name = '{table_name}' AND name NOT LIKE 'sqlite_%'" + "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND tbl_name = '{table_name}' OR name = '{table_name}' AND name NOT LIKE 'sqlite_%' ORDER BY type, name" ); let mut found = false; @@ -1028,9 +1092,7 @@ impl Limbo { db_prefix: &str, db_display_name: &str, ) -> anyhow::Result<()> { - let sql = format!( - "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index') AND name NOT LIKE 'sqlite_%'" - ); + let sql = format!("SELECT sql, type, name FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND name NOT LIKE 'sqlite_%' ORDER BY CASE type WHEN 'table' THEN 1 WHEN 'view' THEN 2 WHEN 'index' THEN 3 END, name"); match self.conn.query(&sql) { Ok(Some(ref mut rows)) => loop { diff --git a/core/incremental/mod.rs b/core/incremental/mod.rs index ef5242330..d80a09081 100644 --- a/core/incremental/mod.rs +++ b/core/incremental/mod.rs @@ -1,3 +1,4 @@ pub mod dbsp; pub mod hashable_row; pub mod operator; +pub mod view; diff --git a/core/incremental/view.rs b/core/incremental/view.rs new file mode 100644 index 000000000..89e9833ef --- /dev/null +++ b/core/incremental/view.rs @@ -0,0 +1,1193 @@ +use super::dbsp::{RowKeyStream, RowKeyZSet}; +use super::operator::{ + AggregateFunction, Delta, FilterOperator, FilterPredicate, ProjectColumn, ProjectOperator, +}; +use crate::schema::{BTreeTable, Column, Schema}; +use crate::types::{IOResult, Value}; +use crate::util::{extract_column_name_from_expr, extract_view_columns}; +use crate::{LimboError, Result, Statement}; +use fallible_iterator::FallibleIterator; +use std::collections::BTreeMap; +use std::fmt; +use std::sync::Arc; +use turso_sqlite3_parser::{ + ast::{Cmd, Stmt}, + lexer::sql::Parser, +}; + +/// State machine for populating a view from its source table +pub enum PopulateState { + /// Initial state - need to prepare the query + Start, + /// Actively processing rows from the query + Processing { + stmt: Box, + rows_processed: usize, + }, + /// Population complete + Done, +} + +impl fmt::Debug for PopulateState { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + PopulateState::Start => write!(f, "Start"), + PopulateState::Processing { rows_processed, .. } => f + .debug_struct("Processing") + .field("rows_processed", rows_processed) + .finish(), + PopulateState::Done => write!(f, "Done"), + } + } +} + +/// Per-connection transaction state for incremental views +#[derive(Debug, Clone, Default)] +pub struct ViewTransactionState { + // Per-connection delta for uncommitted changes (contains both weights and values) + pub delta: Delta, +} + +/// Incremental view that maintains a stream of row keys using DBSP-style computation +/// The actual row data is stored as transformed Values +/// +/// This version keeps everything in-memory. This is acceptable for small views, since DBSP +/// doesn't have to track the history of changes. Still for very large views (think of the result +/// of create view v as select * from tbl where x > 1; and that having 1B values. +/// +/// We should have a version of this that materializes the results. Materializing will also be good +/// for large aggregations, because then we don't have to re-compute when opening the database +/// again. +/// +/// Right now we are supporting the simplest views by keeping the operators in the view and +/// applying them in a sane order. But the general solution would turn this into a DBSP circuit. +#[derive(Debug)] +pub struct IncrementalView { + // Stream of row keys for this view + stream: RowKeyStream, + name: String, + // Store the actual row data as Values, keyed by row_key + // Using BTreeMap for ordered iteration + pub records: BTreeMap>, + // WHERE clause predicate for filtering (kept for compatibility) + pub where_predicate: FilterPredicate, + // The SELECT statement that defines how to transform input data + pub select_stmt: Box, + + // Internal filter operator for predicate evaluation + filter_operator: Option, + // Internal project operator for value transformation + project_operator: Option, + // Tables referenced by this view (extracted from FROM clause and JOINs) + base_table: Arc, + // The view's output columns with their types + pub columns: Vec, + // State machine for population + populate_state: PopulateState, +} + +impl IncrementalView { + /// Validate that a CREATE VIEW statement can be handled by IncrementalView + /// This should be called early, before updating sqlite_master + pub fn can_create_view( + select: &turso_sqlite3_parser::ast::Select, + schema: &Schema, + ) -> Result<()> { + // Check for aggregations + let (group_by_columns, aggregate_functions, _) = Self::extract_aggregation_info(select); + if !group_by_columns.is_empty() || !aggregate_functions.is_empty() { + return Err(LimboError::ParseError( + "aggregations in views are not yet supported".to_string(), + )); + } + + // Check for JOINs + let (join_tables, join_condition) = Self::extract_join_info(select); + if join_tables.is_some() || join_condition.is_some() { + return Err(LimboError::ParseError( + "JOINs in views are not yet supported".to_string(), + )); + } + + // Check that we have a base table + let base_table_name = Self::extract_base_table(select).ok_or_else(|| { + LimboError::ParseError("views without a base table not supported yet".to_string()) + })?; + + // Get the base table + let base_table = schema.get_btree_table(&base_table_name).ok_or_else(|| { + LimboError::ParseError(format!("Table '{base_table_name}' not found in schema")) + })?; + + // Get base table column names for validation + let base_table_column_names: Vec = base_table + .columns + .iter() + .enumerate() + .map(|(i, col)| col.name.clone().unwrap_or_else(|| format!("column_{i}"))) + .collect(); + + // Validate columns are a strict subset + Self::validate_view_columns(select, &base_table_column_names)?; + + Ok(()) + } + + /// Get an iterator over column names, using enumerated naming for unnamed columns + pub fn column_names(&self) -> impl Iterator + '_ { + self.columns.iter().enumerate().map(|(i, col)| { + col.name + .clone() + .unwrap_or_else(|| format!("column{}", i + 1)) + }) + } + + /// Check if this view has the same SQL definition as the provided SQL string + pub fn has_same_sql(&self, sql: &str) -> bool { + // Parse the SQL to extract just the SELECT statement + if let Ok(Some(Cmd::Stmt(Stmt::CreateView { select, .. }))) = + Parser::new(sql.as_bytes()).next() + { + // Compare the SELECT statements as SQL strings + use turso_sqlite3_parser::ast::fmt::ToTokens; + + // Format both SELECT statements and compare + if let (Ok(current_sql), Ok(provided_sql)) = + (self.select_stmt.format(), select.format()) + { + return current_sql == provided_sql; + } + } + false + } + + /// Apply filter operator to check if values pass the view's WHERE clause + fn apply_filter(&self, values: &[Value]) -> bool { + if let Some(ref filter_op) = self.filter_operator { + filter_op.evaluate_predicate(values) + } else { + true + } + } + pub fn from_sql(sql: &str, schema: &Schema) -> Result { + let mut parser = Parser::new(sql.as_bytes()); + let cmd = parser.next()?; + let cmd = cmd.expect("View is an empty statement"); + match cmd { + Cmd::Stmt(Stmt::CreateView { + temporary: _, + if_not_exists: _, + view_name, + columns: _, + select, + }) => IncrementalView::from_stmt(view_name, select, schema), + _ => Err(LimboError::ParseError(format!( + "View is not a CREATE VIEW statement: {sql}" + ))), + } + } + + pub fn from_stmt( + view_name: turso_sqlite3_parser::ast::QualifiedName, + select: Box, + schema: &Schema, + ) -> Result { + let name = view_name.name.as_str().to_string(); + + let where_predicate = FilterPredicate::from_select(&select); + + // Extract output columns using the shared function + let view_columns = extract_view_columns(&select, schema); + + // Extract GROUP BY columns and aggregate functions + let (group_by_columns, aggregate_functions, _old_output_names) = + Self::extract_aggregation_info(&select); + + if !group_by_columns.is_empty() || !aggregate_functions.is_empty() { + return Err(LimboError::ParseError( + "aggregations in views are not yet supported".to_string(), + )); + } + + let (join_tables, join_condition) = Self::extract_join_info(&select); + if join_tables.is_some() || join_condition.is_some() { + return Err(LimboError::ParseError( + "JOINs in views are not yet supported".to_string(), + )); + } + + // Get the base table from FROM clause (when no joins) + let base_table = if let Some(base_table_name) = Self::extract_base_table(&select) { + if let Some(table) = schema.get_btree_table(&base_table_name) { + table.clone() + } else { + return Err(LimboError::ParseError(format!( + "Table '{base_table_name}' not found in schema" + ))); + } + } else { + return Err(LimboError::ParseError( + "views without a base table not supported yet".to_string(), + )); + }; + + let base_table_column_names = base_table + .columns + .iter() + .enumerate() + .map(|(i, col)| col.name.clone().unwrap_or_else(|| format!("column_{i}"))) + .collect(); + + Ok(Self::new( + name, + Vec::new(), // Empty initial data + where_predicate, + select.clone(), + base_table, + base_table_column_names, + view_columns, + )) + } + + pub fn new( + name: String, + initial_data: Vec<(i64, Vec)>, + where_predicate: FilterPredicate, + select_stmt: Box, + base_table: Arc, + base_table_column_names: Vec, + columns: Vec, + ) -> Self { + let mut records = BTreeMap::new(); + + for (row_key, values) in initial_data { + records.insert(row_key, values); + } + + // Create initial stream with row keys + let mut zset = RowKeyZSet::new(); + for (row_key, values) in &records { + use crate::incremental::hashable_row::HashableRow; + let row = HashableRow::new(*row_key, values.clone()); + zset.insert(row, 1); + } + + // Create filter operator if we have a predicate + let filter_operator = if !matches!(where_predicate, FilterPredicate::None) { + Some(FilterOperator::new( + where_predicate.clone(), + base_table_column_names.clone(), + )) + } else { + None + }; + + let project_operator = { + let columns = Self::extract_project_columns(&select_stmt, &base_table_column_names) + .unwrap_or_else(|| { + // If we can't extract columns, default to projecting all columns + base_table_column_names + .iter() + .map(|name| ProjectColumn::Column(name.to_string())) + .collect() + }); + Some(ProjectOperator::new( + columns, + base_table_column_names.clone(), + )) + }; + + Self { + stream: RowKeyStream::from_zset(zset), + name, + records, + where_predicate, + select_stmt, + filter_operator, + project_operator, + base_table, + columns, + populate_state: PopulateState::Start, + } + } + + pub fn name(&self) -> &str { + &self.name + } + + /// Get all table names referenced by this view + pub fn get_referenced_table_names(&self) -> Vec { + vec![self.base_table.name.clone()] + } + + /// Get all tables referenced by this view + pub fn get_referenced_tables(&self) -> Vec> { + vec![self.base_table.clone()] + } + + /// Validate that view columns are a strict subset of the base table columns + /// No duplicates, no complex expressions, only simple column references + fn validate_view_columns( + select: &turso_sqlite3_parser::ast::Select, + base_table_column_names: &[String], + ) -> Result<()> { + if let turso_sqlite3_parser::ast::OneSelect::Select(ref select_stmt) = &*select.body.select + { + let mut seen_columns = std::collections::HashSet::new(); + + for result_col in &select_stmt.columns { + match result_col { + turso_sqlite3_parser::ast::ResultColumn::Expr( + turso_sqlite3_parser::ast::Expr::Id(name), + _, + ) => { + let col_name = name.as_str(); + + // Check for duplicates + if !seen_columns.insert(col_name) { + return Err(LimboError::ParseError(format!( + "Duplicate column '{col_name}' in view. Views must have columns as a strict subset of the base table (no duplicates)" + ))); + } + + // Check that column exists in base table + if !base_table_column_names.iter().any(|n| n == col_name) { + return Err(LimboError::ParseError(format!( + "Column '{col_name}' not found in base table. Views must have columns as a strict subset of the base table" + ))); + } + } + turso_sqlite3_parser::ast::ResultColumn::Star => { + // SELECT * is allowed - it's the full set + } + _ => { + // Any other expression is not allowed + return Err(LimboError::ParseError("Complex expressions, functions, or computed columns are not supported in views. Views must have columns as a strict subset of the base table".to_string())); + } + } + } + } + Ok(()) + } + + /// Extract the base table name from a SELECT statement (for non-join cases) + fn extract_base_table(select: &turso_sqlite3_parser::ast::Select) -> Option { + if let turso_sqlite3_parser::ast::OneSelect::Select(ref select_stmt) = &*select.body.select + { + if let Some(ref from) = &select_stmt.from { + if let Some(ref select_table) = &from.select { + if let turso_sqlite3_parser::ast::SelectTable::Table(name, _, _) = + &**select_table + { + return Some(name.name.as_str().to_string()); + } + } + } + } + None + } + + /// Generate the SQL query for populating the view from its source table + fn sql_for_populate(&self) -> crate::Result { + // Get the base table from referenced tables + let table = &self.base_table; + + // Build column list for SELECT clause + let select_columns = if let Some(ref project_op) = self.project_operator { + // Get the columns used by the projection operator + let mut columns = Vec::new(); + for col in project_op.columns() { + match col { + ProjectColumn::Column(name) => { + columns.push(name.clone()); + } + ProjectColumn::Expression { .. } => { + // For expressions, we need all columns (for now) + columns.clear(); + columns.push("*".to_string()); + break; + } + } + } + if columns.is_empty() || columns.contains(&"*".to_string()) { + "*".to_string() + } else { + // Add the columns and always include rowid + columns.join(", ").to_string() + } + } else { + // No projection, use all columns + "*".to_string() + }; + + // Build WHERE clause from filter operator + let where_clause = if let Some(ref filter_op) = self.filter_operator { + self.build_where_clause(filter_op.predicate())? + } else { + String::new() + }; + + // Construct the final query + let query = if where_clause.is_empty() { + format!("SELECT {}, rowid FROM {}", select_columns, table.name) + } else { + format!( + "SELECT {}, rowid FROM {} WHERE {}", + select_columns, table.name, where_clause + ) + }; + Ok(query) + } + + /// Build a WHERE clause from a FilterPredicate + fn build_where_clause(&self, predicate: &FilterPredicate) -> crate::Result { + match predicate { + FilterPredicate::None => Ok(String::new()), + FilterPredicate::Equals { column, value } => { + Ok(format!("{} = {}", column, self.value_to_sql(value))) + } + FilterPredicate::NotEquals { column, value } => { + Ok(format!("{} != {}", column, self.value_to_sql(value))) + } + FilterPredicate::GreaterThan { column, value } => { + Ok(format!("{} > {}", column, self.value_to_sql(value))) + } + FilterPredicate::GreaterThanOrEqual { column, value } => { + Ok(format!("{} >= {}", column, self.value_to_sql(value))) + } + FilterPredicate::LessThan { column, value } => { + Ok(format!("{} < {}", column, self.value_to_sql(value))) + } + FilterPredicate::LessThanOrEqual { column, value } => { + Ok(format!("{} <= {}", column, self.value_to_sql(value))) + } + FilterPredicate::And(left, right) => { + let left_clause = self.build_where_clause(left)?; + let right_clause = self.build_where_clause(right)?; + Ok(format!("({left_clause} AND {right_clause})")) + } + FilterPredicate::Or(left, right) => { + let left_clause = self.build_where_clause(left)?; + let right_clause = self.build_where_clause(right)?; + Ok(format!("({left_clause} OR {right_clause})")) + } + } + } + + /// Convert a Value to SQL literal representation + fn value_to_sql(&self, value: &Value) -> String { + match value { + Value::Null => "NULL".to_string(), + Value::Integer(i) => i.to_string(), + Value::Float(f) => f.to_string(), + Value::Text(t) => format!("'{}'", t.as_str().replace('\'', "''")), + Value::Blob(_) => "NULL".to_string(), // Blob literals not supported in WHERE clause yet + } + } + + /// Populate the view by scanning the source table using a state machine + /// This can be called multiple times and will resume from where it left off + pub fn populate_from_table( + &mut self, + conn: &std::sync::Arc, + ) -> crate::Result> { + // If already populated, return immediately + if matches!(self.populate_state, PopulateState::Done) { + return Ok(IOResult::Done(())); + } + + const BATCH_SIZE: usize = 100; // Process 100 rows at a time before yielding + + loop { + match &mut self.populate_state { + PopulateState::Start => { + // Generate the SQL query for populating the view + // It is best to use a standard query than a cursor for two reasons: + // 1) Using a sql query will allow us to be much more efficient in cases where we only want + // some rows, in particular for indexed filters + // 2) There are two types of cursors: index and table. In some situations (like for example + // if the table has an integer primary key), the key will be exclusively in the index + // btree and not in the table btree. Using cursors would force us to be aware of this + // distinction (and others), and ultimately lead to reimplementing the whole query + // machinery (next step is which index is best to use, etc) + let query = self.sql_for_populate()?; + + // Prepare the statement + let stmt = conn.prepare(&query)?; + + self.populate_state = PopulateState::Processing { + stmt: Box::new(stmt), + rows_processed: 0, + }; + // Continue to next state + } + + PopulateState::Processing { + stmt, + rows_processed, + } => { + // Process rows in batches to allow for IO interruption + let mut batch_count = 0; + + loop { + if batch_count >= BATCH_SIZE { + // Yield control after processing a batch + // The statement maintains its position, so we'll resume from here + return Ok(IOResult::IO); + } + + // This step() call resumes from where the statement left off + match stmt.step()? { + crate::vdbe::StepResult::Row => { + // Get the row + let row = stmt.row().unwrap(); + + // Extract values from the row + let all_values: Vec = + row.get_values().cloned().collect(); + + // The last value should be the rowid + let rowid = match all_values.last() { + Some(crate::types::Value::Integer(id)) => *id, + _ => { + // This shouldn't happen - rowid must be an integer + *rows_processed += 1; + batch_count += 1; + continue; + } + }; + + // Get all values except the rowid + let values = all_values[..all_values.len() - 1].to_vec(); + + // Apply filter if we have one + // Pure DBSP would ingest the entire stream and then apply filter operators. + // However, for initial population, we adopt a hybrid approach where we filter at + // the query result level for efficiency. This avoids reading millions of rows just + // to filter them down to a few. We only do this optimization for filters, not for + // other operators like projections or aggregations. + // TODO: We should further optimize by pushing the filter into the SQL WHERE clause. + + // Check filter first (we need to do this before accessing self mutably) + let passes_filter = + if let Some(ref filter_op) = self.filter_operator { + filter_op.evaluate_predicate(&values) + } else { + true + }; + + if passes_filter { + // Store the row with its original rowid + self.records.insert(rowid, values.clone()); + + // Update the ZSet stream with weight +1 + let mut delta = RowKeyZSet::new(); + use crate::incremental::hashable_row::HashableRow; + let row = HashableRow::new(rowid, values); + delta.insert(row, 1); + self.stream.apply_delta(&delta); + } + + *rows_processed += 1; + batch_count += 1; + } + crate::vdbe::StepResult::Done => { + // All rows processed, move to Done state + self.populate_state = PopulateState::Done; + return Ok(IOResult::Done(())); + } + crate::vdbe::StepResult::Interrupt | crate::vdbe::StepResult::Busy => { + return Err(LimboError::Busy); + } + crate::vdbe::StepResult::IO => { + // The Statement needs to wait for IO + // When we return here, the Statement maintains its position + return Ok(IOResult::IO); + } + } + } + } + + PopulateState::Done => { + // Already populated + return Ok(IOResult::Done(())); + } + } + } + } + + /// Extract GROUP BY columns and aggregate functions from SELECT statement + fn extract_aggregation_info( + select: &turso_sqlite3_parser::ast::Select, + ) -> (Vec, Vec, Vec) { + use turso_sqlite3_parser::ast::*; + + let mut group_by_columns = Vec::new(); + let mut aggregate_functions = Vec::new(); + let mut output_column_names = Vec::new(); + + if let OneSelect::Select(ref select_stmt) = &*select.body.select { + // Extract GROUP BY columns + if let Some(ref group_by) = select_stmt.group_by { + for expr in &group_by.exprs { + if let Some(col_name) = extract_column_name_from_expr(expr) { + group_by_columns.push(col_name); + } + } + } + + // Extract aggregate functions and column names/aliases from SELECT list + for result_col in &select_stmt.columns { + match result_col { + ResultColumn::Expr(expr, alias) => { + // Extract aggregate functions + let mut found_aggregates = Vec::new(); + Self::extract_aggregates_from_expr(expr, &mut found_aggregates); + + // Determine the output column name + let col_name = if let Some(As::As(alias_name)) = alias { + // Use the provided alias + alias_name.as_str().to_string() + } else if !found_aggregates.is_empty() { + // Use the default name from the aggregate function + found_aggregates[0].default_output_name() + } else if let Some(name) = extract_column_name_from_expr(expr) { + // Use the column name + name + } else { + // Fallback to a generic name + format!("column{}", output_column_names.len() + 1) + }; + + output_column_names.push(col_name); + aggregate_functions.extend(found_aggregates); + } + ResultColumn::Star => { + // For SELECT *, we'd need to know the base table columns + // This is handled elsewhere + } + ResultColumn::TableStar(_) => { + // Similar to Star, but for a specific table + } + } + } + } + + (group_by_columns, aggregate_functions, output_column_names) + } + + /// Recursively extract aggregate functions from an expression + fn extract_aggregates_from_expr( + expr: &turso_sqlite3_parser::ast::Expr, + aggregate_functions: &mut Vec, + ) { + use crate::function::Func; + use turso_sqlite3_parser::ast::*; + + match expr { + // Handle COUNT(*) and similar aggregate functions with * + Expr::FunctionCallStar { name, .. } => { + // FunctionCallStar is typically COUNT(*), which has 0 args + if let Ok(func) = Func::resolve_function(name.as_str(), 0) { + // Use the centralized mapping from operator.rs + // For COUNT(*), we pass None as the input column + if let Some(agg_func) = AggregateFunction::from_sql_function(&func, None) { + aggregate_functions.push(agg_func); + } + } + } + Expr::FunctionCall { name, args, .. } => { + // Regular function calls with arguments + let arg_count = args.as_ref().map_or(0, |a| a.len()); + + if let Ok(func) = Func::resolve_function(name.as_str(), arg_count) { + // Extract the input column if there's an argument + let input_column = if arg_count > 0 { + args.as_ref() + .and_then(|args| args.first()) + .and_then(extract_column_name_from_expr) + } else { + None + }; + + // Use the centralized mapping from operator.rs + if let Some(agg_func) = + AggregateFunction::from_sql_function(&func, input_column) + { + aggregate_functions.push(agg_func); + } + } + } + // Recursively check binary expressions, etc. + Expr::Binary(left, _, right) => { + Self::extract_aggregates_from_expr(left, aggregate_functions); + Self::extract_aggregates_from_expr(right, aggregate_functions); + } + _ => {} + } + } + + /// Extract JOIN information from SELECT statement + #[allow(clippy::type_complexity)] + pub fn extract_join_info( + select: &turso_sqlite3_parser::ast::Select, + ) -> (Option<(String, String)>, Option<(String, String)>) { + use turso_sqlite3_parser::ast::*; + + if let OneSelect::Select(ref select_stmt) = &*select.body.select { + if let Some(ref from) = &select_stmt.from { + // Check if there are any joins + if let Some(ref joins) = &from.joins { + if !joins.is_empty() { + // Get the first (left) table name + let left_table = if let Some(ref select_table) = &from.select { + match &**select_table { + SelectTable::Table(name, _, _) => { + Some(name.name.as_str().to_string()) + } + _ => None, + } + } else { + None + }; + + // Get the first join (right) table and condition + if let Some(first_join) = joins.first() { + let right_table = match &first_join.table { + SelectTable::Table(name, _, _) => { + Some(name.name.as_str().to_string()) + } + _ => None, + }; + + // Extract join condition (simplified - assumes single equality) + let join_condition = + if let Some(ref constraint) = &first_join.constraint { + match constraint { + JoinConstraint::On(expr) => { + Self::extract_join_columns_from_expr(expr) + } + _ => None, + } + } else { + None + }; + + if let (Some(left), Some(right)) = (left_table, right_table) { + return (Some((left, right)), join_condition); + } + } + } + } + } + } + + (None, None) + } + + /// Extract join column names from a join condition expression + fn extract_join_columns_from_expr( + expr: &turso_sqlite3_parser::ast::Expr, + ) -> Option<(String, String)> { + use turso_sqlite3_parser::ast::*; + + // Look for expressions like: t1.col = t2.col + if let Expr::Binary(left, op, right) = expr { + if matches!(op, Operator::Equals) { + // Extract column names from both sides + let left_col = match &**left { + Expr::Qualified(name, _) => Some(name.as_str().to_string()), + Expr::Id(name) => Some(name.as_str().to_string()), + _ => None, + }; + + let right_col = match &**right { + Expr::Qualified(name, _) => Some(name.as_str().to_string()), + Expr::Id(name) => Some(name.as_str().to_string()), + _ => None, + }; + + if let (Some(l), Some(r)) = (left_col, right_col) { + return Some((l, r)); + } + } + } + + None + } + + /// Extract projection columns from SELECT statement + fn extract_project_columns( + select: &turso_sqlite3_parser::ast::Select, + column_names: &[String], + ) -> Option> { + use turso_sqlite3_parser::ast::*; + + if let OneSelect::Select(ref select_stmt) = &*select.body.select { + let mut columns = Vec::new(); + + for result_col in &select_stmt.columns { + match result_col { + ResultColumn::Expr(Expr::Id(name), _) => { + columns.push(ProjectColumn::Column(name.as_str().to_string())); + } + ResultColumn::Star => { + // Select all columns + for name in column_names { + columns.push(ProjectColumn::Column(name.as_str().to_string())); + } + } + _ => { + // For now, skip complex expressions + // Could be extended to handle more cases + } + } + } + + if !columns.is_empty() { + return Some(columns); + } + } + + None + } + + /// Get the current records as an iterator - for cursor-based access + pub fn iter(&self) -> impl Iterator)> + '_ { + self.stream.to_vec().into_iter().filter_map(move |row| { + self.records + .get(&row.rowid) + .map(|values| (row.rowid, values.clone())) + }) + } + + /// Get current data merged with transaction state + pub fn current_data(&self, tx_state: Option<&ViewTransactionState>) -> Vec<(i64, Vec)> { + // Start with committed records + + if let Some(tx_state) = tx_state { + // processed_delta = input delta for now. Need to apply operations + let processed_delta = &tx_state.delta; + + // For non-aggregation views, merge the processed delta with committed records + let mut result_map: BTreeMap> = self.records.clone(); + + for (row, weight) in &processed_delta.changes { + if *weight > 0 && self.apply_filter(&row.values) { + result_map.insert(row.rowid, row.values.clone()); + } else if *weight < 0 { + result_map.remove(&row.rowid); + } + } + + result_map.into_iter().collect() + } else { + // No transaction state: return committed records + self.records.clone().into_iter().collect() + } + } + + /// Merge a delta of changes into the view's current state + pub fn merge_delta(&mut self, delta: &Delta) { + // Create a Z-set of changes to apply to the stream + let mut zset_delta = RowKeyZSet::new(); + + // Apply the delta changes to the records + for (row, weight) in &delta.changes { + if *weight > 0 { + // Insert + if self.apply_filter(&row.values) { + self.records.insert(row.rowid, row.values.clone()); + zset_delta.insert(row.clone(), 1); + } + } else if *weight < 0 { + // Delete + if self.records.remove(&row.rowid).is_some() { + zset_delta.insert(row.clone(), -1); + } + } + } + + // Apply all changes to the stream at once + self.stream.apply_delta(&zset_delta); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::incremental::operator::{Delta, IncrementalOperator}; + use crate::schema::{BTreeTable, Column, Schema, Type}; + use crate::types::Value; + use std::sync::Arc; + fn create_test_schema() -> Schema { + let mut schema = Schema::new(false); + let table = BTreeTable { + root_page: 1, + name: "t".to_string(), + columns: vec![ + Column { + name: Some("a".to_string()), + ty: Type::Integer, + ty_str: "INTEGER".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }, + Column { + name: Some("b".to_string()), + ty: Type::Integer, + ty_str: "INTEGER".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }, + Column { + name: Some("c".to_string()), + ty: Type::Integer, + ty_str: "INTEGER".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }, + ], + primary_key_columns: vec![], + has_rowid: true, + is_strict: false, + unique_sets: None, + }; + schema.add_btree_table(Arc::new(table)); + schema + } + + #[test] + fn test_projection_simple_columns() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a, b FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(10), Value::Integer(20), Value::Integer(30)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!(output.values, vec![Value::Integer(10), Value::Integer(20)]); + } + + #[test] + fn test_projection_arithmetic_expression() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a * 2 as doubled FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(4), Value::Integer(2), Value::Integer(0)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!(output.values, vec![Value::Integer(8)]); + } + + #[test] + fn test_projection_multiple_expressions() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a + b as sum, a - b as diff, c FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(10), Value::Integer(3), Value::Integer(7)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!( + output.values, + vec![Value::Integer(13), Value::Integer(7), Value::Integer(7),] + ); + } + + #[test] + fn test_projection_function_call() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT hex(a) as hex_a, b FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(255), Value::Integer(20), Value::Integer(30)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!( + output.values, + vec![Value::Text("FF".into()), Value::Integer(20),] + ); + } + + #[test] + fn test_projection_mixed_columns_and_expressions() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a, b * 2 as doubled, c, a + b + c as total FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(1), Value::Integer(5), Value::Integer(3)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!( + output.values, + vec![ + Value::Integer(1), + Value::Integer(10), + Value::Integer(3), + Value::Integer(9), + ] + ); + } + + #[test] + fn test_projection_complex_expression() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT (a * 2) + (b * 3) as weighted, c / 2 as half FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(5), Value::Integer(2), Value::Integer(10)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!(output.values, vec![Value::Integer(16), Value::Integer(5),]); + } + + #[test] + fn test_projection_with_where_clause() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a, a * 2 as doubled FROM t WHERE b > 2"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + assert!(view.filter_operator.is_some()); + + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(4), Value::Integer(3), Value::Integer(0)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + assert_eq!(output.values, vec![Value::Integer(4), Value::Integer(8),]); + } + + #[test] + fn test_projection_more_output_columns_than_input() { + let schema = create_test_schema(); + let sql = "CREATE VIEW v AS SELECT a, b, a * 2 as doubled_a, b * 3 as tripled_b, a + b as sum, hex(c) as hex_c FROM t"; + + let view = IncrementalView::from_sql(sql, &schema).unwrap(); + + assert!(view.project_operator.is_some()); + let project_op = view.project_operator.as_ref().unwrap(); + + let mut delta = Delta::new(); + delta.insert( + 1, + vec![Value::Integer(5), Value::Integer(2), Value::Integer(15)], + ); + + let mut temp_project = project_op.clone(); + temp_project.initialize(delta); + let result = temp_project.get_current_state(); + + let (output, _weight) = result.changes.first().unwrap(); + // 3 input columns -> 6 output columns + assert_eq!( + output.values, + vec![ + Value::Integer(5), // a + Value::Integer(2), // b + Value::Integer(10), // a * 2 + Value::Integer(6), // b * 3 + Value::Integer(7), // a + b + Value::Text("F".into()), // hex(15) + ] + ); + } +} diff --git a/core/lib.rs b/core/lib.rs index 94956f83a..eda318414 100644 --- a/core/lib.rs +++ b/core/lib.rs @@ -32,6 +32,7 @@ mod uuid; mod vdbe; mod vector; mod vtab; +mod vtab_view; #[cfg(feature = "fuzz")] pub mod numeric; @@ -39,6 +40,7 @@ pub mod numeric; #[cfg(not(feature = "fuzz"))] mod numeric; +use crate::incremental::view::ViewTransactionState; use crate::translate::optimizer::optimize_plan; use crate::translate::pragma::TURSO_CDC_DEFAULT_TABLE_NAME; #[cfg(all(feature = "fs", feature = "conn_raw_api"))] @@ -130,6 +132,7 @@ pub struct Database { init_lock: Arc>, open_flags: OpenFlags, builtin_syms: RefCell, + experimental_views: bool, } unsafe impl Send for Database {} @@ -190,7 +193,14 @@ impl Database { enable_mvcc: bool, enable_indexes: bool, ) -> Result> { - Self::open_file_with_flags(io, path, OpenFlags::default(), enable_mvcc, enable_indexes) + Self::open_file_with_flags( + io, + path, + OpenFlags::default(), + enable_mvcc, + enable_indexes, + false, + ) } #[cfg(feature = "fs")] @@ -200,10 +210,19 @@ impl Database { flags: OpenFlags, enable_mvcc: bool, enable_indexes: bool, + enable_views: bool, ) -> Result> { let file = io.open_file(path, flags, true)?; let db_file = Arc::new(DatabaseFile::new(file)); - Self::open_with_flags(io, path, db_file, flags, enable_mvcc, enable_indexes) + Self::open_with_flags( + io, + path, + db_file, + flags, + enable_mvcc, + enable_indexes, + enable_views, + ) } #[allow(clippy::arc_with_non_send_sync)] @@ -221,6 +240,7 @@ impl Database { OpenFlags::default(), enable_mvcc, enable_indexes, + false, ) } @@ -232,9 +252,18 @@ impl Database { flags: OpenFlags, enable_mvcc: bool, enable_indexes: bool, + enable_views: bool, ) -> Result> { if path == ":memory:" { - return Self::do_open_with_flags(io, path, db_file, flags, enable_mvcc, enable_indexes); + return Self::do_open_with_flags( + io, + path, + db_file, + flags, + enable_mvcc, + enable_indexes, + enable_views, + ); } let mut registry = DATABASE_MANAGER.lock().unwrap(); @@ -247,7 +276,15 @@ impl Database { if let Some(db) = registry.get(&canonical_path).and_then(Weak::upgrade) { return Ok(db); } - let db = Self::do_open_with_flags(io, path, db_file, flags, enable_mvcc, enable_indexes)?; + let db = Self::do_open_with_flags( + io, + path, + db_file, + flags, + enable_mvcc, + enable_indexes, + enable_views, + )?; registry.insert(canonical_path, Arc::downgrade(&db)); Ok(db) } @@ -260,6 +297,7 @@ impl Database { flags: OpenFlags, enable_mvcc: bool, enable_indexes: bool, + enable_views: bool, ) -> Result> { let wal_path = format!("{path}-wal"); let maybe_shared_wal = WalFileShared::open_shared_if_exists(&io, wal_path.as_str())?; @@ -294,6 +332,7 @@ impl Database { open_flags: flags, db_state: Arc::new(AtomicDbState::new(db_state)), init_lock: Arc::new(Mutex::new(())), + experimental_views: enable_views, }); db.register_global_builtin_extensions() .expect("unable to register global extensions"); @@ -325,6 +364,13 @@ impl Database { Ok(()) })?; } + // FIXME: the correct way to do this is to just materialize the view. + // But this will allow us to keep going. + let conn = db.connect()?; + let pager = conn.pager.borrow().clone(); + pager + .io + .block(|| conn.schema.borrow().populate_views(&conn))?; Ok(db) } @@ -369,6 +415,7 @@ impl Database { closed: Cell::new(false), attached_databases: RefCell::new(DatabaseCatalog::new()), query_only: Cell::new(false), + view_transaction_states: RefCell::new(HashMap::new()), }); let builtin_syms = self.builtin_syms.borrow(); // add built-in extensions symbols to the connection to prevent having to load each time @@ -460,6 +507,7 @@ impl Database { flags: OpenFlags, indexes: bool, mvcc: bool, + views: bool, ) -> Result<(Arc, Arc)> where S: AsRef + std::fmt::Display, @@ -486,7 +534,7 @@ impl Database { } }, }; - let db = Self::open_file_with_flags(io.clone(), path, flags, mvcc, indexes)?; + let db = Self::open_file_with_flags(io.clone(), path, flags, mvcc, indexes, views)?; Ok((io, db)) } None => { @@ -494,7 +542,7 @@ impl Database { MEMORY_PATH => Arc::new(MemoryIO::new()), _ => Arc::new(PlatformIO::new()?), }; - let db = Self::open_file_with_flags(io.clone(), path, flags, mvcc, indexes)?; + let db = Self::open_file_with_flags(io.clone(), path, flags, mvcc, indexes, views)?; Ok((io, db)) } } @@ -533,6 +581,10 @@ impl Database { pub fn get_mv_store(&self) -> Option<&Arc> { self.mv_store.as_ref() } + + pub fn experimental_views_enabled(&self) -> bool { + self.experimental_views + } } #[derive(Debug, Clone, Eq, PartialEq)] @@ -725,6 +777,10 @@ pub struct Connection { /// Attached databases attached_databases: RefCell, query_only: Cell, + + /// Per-connection view transaction states for uncommitted changes. This represents + /// one entry per view that was touched in the transaction. + view_transaction_states: RefCell>, } impl Connection { @@ -827,6 +883,10 @@ impl Connection { let mut fresh = Schema::new(self.schema.borrow().indexes_enabled); fresh.schema_version = cookie; + // Preserve existing views to avoid expensive repopulation. + // TODO: We may not need to do this if we materialize our views. + let existing_views = self.schema.borrow().views.clone(); + // TODO: this is hack to avoid a cyclical problem with schema reprepare // The problem here is that we prepare a statement here, but when the statement tries // to execute it, it first checks the schema cookie to see if it needs to reprepare the statement. @@ -839,11 +899,16 @@ impl Connection { let stmt = self.prepare("SELECT * FROM sqlite_schema")?; // TODO: This function below is synchronous, make it async - parse_schema_rows(stmt, &mut fresh, &self.syms.borrow(), None)?; + parse_schema_rows(stmt, &mut fresh, &self.syms.borrow(), None, existing_views)?; self.with_schema_mut(|schema| { *schema = fresh; }); + + { + let schema = self.schema.borrow(); + pager.io.block(|| schema.populate_views(self))?; + } Result::Ok(()) }; @@ -1069,18 +1134,32 @@ impl Connection { uri: &str, use_indexes: bool, mvcc: bool, + views: bool, ) -> Result<(Arc, Arc)> { use crate::util::MEMORY_PATH; let opts = OpenOptions::parse(uri)?; let flags = opts.get_flags()?; if opts.path == MEMORY_PATH || matches!(opts.mode, OpenMode::Memory) { let io = Arc::new(MemoryIO::new()); - let db = - Database::open_file_with_flags(io.clone(), MEMORY_PATH, flags, mvcc, use_indexes)?; + let db = Database::open_file_with_flags( + io.clone(), + MEMORY_PATH, + flags, + mvcc, + use_indexes, + views, + )?; let conn = db.connect()?; return Ok((io, conn)); } - let (io, db) = Database::open_new(&opts.path, opts.vfs.as_ref(), flags, use_indexes, mvcc)?; + let (io, db) = Database::open_new( + &opts.path, + opts.vfs.as_ref(), + flags, + use_indexes, + mvcc, + views, + )?; if let Some(modeof) = opts.modeof { let perms = std::fs::metadata(modeof)?; std::fs::set_permissions(&opts.path, perms.permissions())?; @@ -1090,13 +1169,24 @@ impl Connection { } #[cfg(feature = "fs")] - fn from_uri_attached(uri: &str, use_indexes: bool, use_mvcc: bool) -> Result> { + fn from_uri_attached( + uri: &str, + use_indexes: bool, + use_mvcc: bool, + use_views: bool, + ) -> Result> { let mut opts = OpenOptions::parse(uri)?; // FIXME: for now, only support read only attach opts.mode = OpenMode::ReadOnly; let flags = opts.get_flags()?; - let (_io, db) = - Database::open_new(&opts.path, opts.vfs.as_ref(), flags, use_indexes, use_mvcc)?; + let (_io, db) = Database::open_new( + &opts.path, + opts.vfs.as_ref(), + flags, + use_indexes, + use_mvcc, + use_views, + )?; if let Some(modeof) = opts.modeof { let perms = std::fs::metadata(modeof)?; std::fs::set_permissions(&opts.path, perms.permissions())?; @@ -1417,7 +1507,9 @@ impl Connection { .expect("query must be parsed to statement"); let syms = self.syms.borrow(); self.with_schema_mut(|schema| { - if let Err(LimboError::ExtensionError(e)) = parse_schema_rows(rows, schema, &syms, None) + let existing_views = schema.views.clone(); + if let Err(LimboError::ExtensionError(e)) = + parse_schema_rows(rows, schema, &syms, None, existing_views) { // this means that a vtab exists and we no longer have the module loaded. we print // a warning to the user to load the module @@ -1483,6 +1575,10 @@ impl Connection { Ok(results) } + pub fn experimental_views_enabled(&self) -> bool { + self._db.experimental_views_enabled() + } + /// Query the current value(s) of `pragma_name` associated to /// `pragma_value`. /// @@ -1578,8 +1674,9 @@ impl Connection { .map_err(|_| LimboError::SchemaLocked)? .indexes_enabled(); let use_mvcc = self._db.mv_store.is_some(); + let use_views = self._db.experimental_views_enabled(); - let db = Self::from_uri_attached(path, use_indexes, use_mvcc)?; + let db = Self::from_uri_attached(path, use_indexes, use_mvcc, use_views)?; let pager = Rc::new(db.init_pager(None)?); self.attached_databases diff --git a/core/schema.rs b/core/schema.rs index 68a605c60..ff469b912 100644 --- a/core/schema.rs +++ b/core/schema.rs @@ -1,3 +1,8 @@ +use crate::incremental::view::IncrementalView; +use crate::types::IOResult; + +/// Type alias for the views collection +pub type ViewsMap = HashMap>>; use crate::result::LimboResult; use crate::storage::btree::BTreeCursor; use crate::translate::collate::CollationSeq; @@ -13,6 +18,7 @@ use std::collections::{BTreeSet, HashMap}; use std::ops::Deref; use std::rc::Rc; use std::sync::Arc; +use std::sync::Mutex; use tracing::trace; use turso_sqlite3_parser::ast::{self, ColumnDefinition, Expr, Literal, SortOrder, TableOptions}; use turso_sqlite3_parser::{ @@ -26,11 +32,16 @@ const SCHEMA_TABLE_NAME_ALT: &str = "sqlite_master"; #[derive(Debug)] pub struct Schema { pub tables: HashMap>, + pub views: ViewsMap, + /// table_name to list of indexes for the table pub indexes: HashMap>>, pub has_indexes: std::collections::HashSet, pub indexes_enabled: bool, pub schema_version: u32, + + /// Mapping from table names to the views that depend on them + pub table_to_views: HashMap>, } impl Schema { @@ -49,12 +60,16 @@ impl Schema { Arc::new(Table::Virtual(Arc::new((*function).clone()))), ); } + let views: ViewsMap = HashMap::new(); + let table_to_views: HashMap> = HashMap::new(); Self { tables, + views, indexes, has_indexes, indexes_enabled, schema_version: 0, + table_to_views, } } @@ -64,6 +79,68 @@ impl Schema { .iter() .any(|idx| idx.1.iter().any(|i| i.name == name)) } + pub fn add_view(&mut self, view: IncrementalView) { + let name = normalize_ident(view.name()); + self.views.insert(name, Arc::new(Mutex::new(view))); + } + + pub fn get_view(&self, name: &str) -> Option>> { + let name = normalize_ident(name); + self.views.get(&name).cloned() + } + + pub fn remove_view(&mut self, name: &str) -> Option>> { + let name = normalize_ident(name); + + // Remove from table_to_views dependencies + for views in self.table_to_views.values_mut() { + views.retain(|v| v != &name); + } + + // Remove the view itself + self.views.remove(&name) + } + + /// Register that a view depends on a table + pub fn add_view_dependency(&mut self, table_name: &str, view_name: &str) { + let table_name = normalize_ident(table_name); + let view_name = normalize_ident(view_name); + + self.table_to_views + .entry(table_name) + .or_default() + .push(view_name); + } + + /// Get all views that depend on a given table + pub fn get_dependent_views(&self, table_name: &str) -> Vec { + let table_name = normalize_ident(table_name); + self.table_to_views + .get(&table_name) + .cloned() + .unwrap_or_default() + } + + /// Populate all views by scanning their source tables + /// Returns IOResult to support async execution + pub fn populate_views(&self, conn: &Arc) -> Result> { + for view in self.views.values() { + let mut view = view + .lock() + .map_err(|_| LimboError::InternalError("Failed to lock view".to_string()))?; + match view.populate_from_table(conn)? { + IOResult::Done(()) => { + // This view is done, continue to next + continue; + } + IOResult::IO => { + // This view needs more IO, return early + return Ok(IOResult::IO); + } + } + } + Ok(IOResult::Done(())) + } pub fn add_btree_table(&mut self, table: Arc) { let name = normalize_ident(&table.name); @@ -160,6 +237,9 @@ impl Schema { let mut automatic_indices: HashMap> = HashMap::with_capacity(10); + // Collect views for second pass to populate table_to_views mapping + let mut views_to_process: Vec<(String, Vec)> = Vec::new(); + if matches!(pager.begin_read_tx()?, LimboResult::Busy) { return Err(LimboError::Busy); } @@ -271,6 +351,35 @@ impl Schema { } } } + "view" => { + let name_value = record_cursor.get_value(&row, 1)?; + let RefValue::Text(name_text) = name_value else { + return Err(LimboError::ConversionError("Expected text value".into())); + }; + let name = name_text.as_str(); + + let sql_value = record_cursor.get_value(&row, 4)?; + let RefValue::Text(sql_text) = sql_value else { + return Err(LimboError::ConversionError("Expected text value".into())); + }; + let sql = sql_text.as_str(); + + // Create IncrementalView directly + if let Ok(incremental_view) = IncrementalView::from_sql(sql, self) { + // Get referenced table names before moving the view + let referenced_tables = incremental_view.get_referenced_table_names(); + let view_name = name.to_string(); + + // Add to schema (moves incremental_view) + self.add_view(incremental_view); + + // Store for second pass processing + views_to_process.push((view_name, referenced_tables)); + } else { + eprintln!("Warning: Could not create incremental view for: {name}"); + } + } + _ => {} }; drop(record_cursor); @@ -281,6 +390,14 @@ impl Schema { pager.end_read_tx()?; + // Second pass: populate table_to_views mapping + for (view_name, referenced_tables) in views_to_process { + // Register this view as dependent on each referenced table + for table_name in referenced_tables { + self.add_view_dependency(&table_name, &view_name); + } + } + for unparsed_sql_from_index in from_sql_indexes { if !self.indexes_enabled() { self.table_set_has_index(&unparsed_sql_from_index.table_name); @@ -357,12 +474,19 @@ impl Clone for Schema { (name.clone(), indexes) }) .collect(); + let views = self + .views + .iter() + .map(|(name, view)| (name.clone(), view.clone())) + .collect(); Self { tables, + views, indexes, has_indexes: self.has_indexes.clone(), indexes_enabled: self.indexes_enabled, schema_version: self.schema_version, + table_to_views: self.table_to_views.clone(), } } } diff --git a/core/storage/pager.rs b/core/storage/pager.rs index 11e5999b1..947f8487e 100644 --- a/core/storage/pager.rs +++ b/core/storage/pager.rs @@ -787,7 +787,7 @@ impl Pager { )?; turso_assert!( - ptrmap_page.get().id == ptrmap_pg_no as usize, + ptrmap_page.get().id == ptrmap_pg_no, "ptrmap page has unexpected number" ); self.add_dirty(&ptrmap_page); diff --git a/core/translate/emitter.rs b/core/translate/emitter.rs index e9b72735b..68219b52f 100644 --- a/core/translate/emitter.rs +++ b/core/translate/emitter.rs @@ -1189,9 +1189,9 @@ fn emit_update_insns( flag: if has_user_provided_rowid { // The previous Insn::NotExists and Insn::Delete seek to the old rowid, // so to insert a new user-provided rowid, we need to seek to the correct place. - InsertFlags::new().require_seek() + InsertFlags::new().require_seek().update() } else { - InsertFlags::new() + InsertFlags::new().update() }, table_name: table_ref.identifier.clone(), }); diff --git a/core/translate/mod.rs b/core/translate/mod.rs index 8b47760ce..440f253b4 100644 --- a/core/translate/mod.rs +++ b/core/translate/mod.rs @@ -34,6 +34,7 @@ pub(crate) mod subquery; pub(crate) mod transaction; pub(crate) mod update; mod values; +pub(crate) mod view; use crate::schema::Schema; use crate::storage::pager::Pager; @@ -127,6 +128,7 @@ pub fn translate_inner( | ast::Stmt::Delete(..) | ast::Stmt::DropIndex { .. } | ast::Stmt::DropTable { .. } + | ast::Stmt::DropView { .. } | ast::Stmt::Reindex { .. } | ast::Stmt::Update(..) | ast::Stmt::Insert(..) @@ -186,7 +188,16 @@ pub fn translate_inner( program, )?, ast::Stmt::CreateTrigger { .. } => bail_parse_error!("CREATE TRIGGER not supported yet"), - ast::Stmt::CreateView { .. } => bail_parse_error!("CREATE VIEW not supported yet"), + ast::Stmt::CreateView { + view_name, select, .. + } => view::translate_create_view( + schema, + view_name.name.as_str(), + &select, + connection.clone(), + syms, + program, + )?, ast::Stmt::CreateVirtualTable(vtab) => { translate_create_virtual_table(*vtab, schema, syms, program)? } @@ -230,7 +241,16 @@ pub fn translate_inner( tbl_name, } => translate_drop_table(tbl_name, if_exists, schema, syms, program)?, ast::Stmt::DropTrigger { .. } => bail_parse_error!("DROP TRIGGER not supported yet"), - ast::Stmt::DropView { .. } => bail_parse_error!("DROP VIEW not supported yet"), + ast::Stmt::DropView { + if_exists, + view_name, + } => view::translate_drop_view( + schema, + view_name.name.as_str(), + if_exists, + connection.clone(), + program, + )?, ast::Stmt::Pragma(..) => { bail_parse_error!("PRAGMA statement cannot be evaluated in a nested context") } diff --git a/core/translate/planner.rs b/core/translate/planner.rs index cd1850812..c3ecd7456 100644 --- a/core/translate/planner.rs +++ b/core/translate/planner.rs @@ -4,8 +4,8 @@ use super::{ expr::walk_expr, plan::{ Aggregate, ColumnUsedMask, Distinctness, EvalAt, JoinInfo, JoinOrderMember, JoinedTable, - Operation, OuterQueryReference, Plan, QueryDestination, ResultSetColumn, TableReferences, - WhereTerm, + Operation, OuterQueryReference, Plan, QueryDestination, ResultSetColumn, Scan, + TableReferences, WhereTerm, }, select::prepare_select_plan, SymbolTable, @@ -470,6 +470,35 @@ fn parse_table( return Ok(()); }; + let view = connection.with_schema(database_id, |schema| schema.get_view(table_name.as_str())); + if let Some(view) = view { + // Create a virtual table wrapper for the view + // We'll use the view's columns from the schema + let vtab = crate::vtab_view::create_view_virtual_table(table_name.as_str(), view.clone())?; + + let alias = maybe_alias + .map(|a| match a { + ast::As::As(id) => id, + ast::As::Elided(id) => id, + }) + .map(|a| a.as_str().to_string()); + + table_references.add_joined_table(JoinedTable { + op: Operation::Scan(Scan::VirtualTable { + idx_num: -1, + idx_str: None, + constraints: Vec::new(), + }), + table: Table::Virtual(vtab), + identifier: alias.unwrap_or(normalized_qualified_name), + internal_id: table_ref_counter.next(), + join_info: None, + col_used_mask: ColumnUsedMask::default(), + database_id, + }); + return Ok(()); + } + // CTEs are transformed into FROM clause subqueries. // If we find a CTE with this name in our outer query references, // we can use it as a joined table, but we must clone it since it's not MATERIALIZED. diff --git a/core/translate/pragma.rs b/core/translate/pragma.rs index 114a66bb1..935bb5b7e 100644 --- a/core/translate/pragma.rs +++ b/core/translate/pragma.rs @@ -419,46 +419,19 @@ fn query_pragma( Ok((program, TransactionMode::Read)) } PragmaName::TableInfo => { - let table = match value { - Some(ast::Expr::Name(name)) => { - let tbl = normalize_ident(name.as_str()); - schema.get_table(&tbl) - } + let name = match value { + Some(ast::Expr::Name(name)) => Some(normalize_ident(name.as_str())), _ => None, }; let base_reg = register; program.alloc_registers(5); - if let Some(table) = table { - // According to the SQLite documentation: "The 'cid' column should not be taken to - // mean more than 'rank within the current result set'." - // Therefore, we enumerate only after filtering out hidden columns. - for (i, column) in table.columns().iter().filter(|col| !col.hidden).enumerate() { - // cid - program.emit_int(i as i64, base_reg); - // name - program.emit_string8(column.name.clone().unwrap_or_default(), base_reg + 1); - - // type - program.emit_string8(column.ty_str.clone(), base_reg + 2); - - // notnull - program.emit_bool(column.notnull, base_reg + 3); - - // dflt_value - match &column.default { - None => { - program.emit_null(base_reg + 4, None); - } - Some(expr) => { - program.emit_string8(expr.to_string(), base_reg + 4); - } - } - - // pk - program.emit_bool(column.primary_key, base_reg + 5); - - program.emit_result_row(base_reg, 6); + if let Some(name) = name { + if let Some(table) = schema.get_table(&name) { + emit_columns_for_table_info(&mut program, table.columns(), base_reg); + } else if let Some(view_mutex) = schema.get_view(&name) { + let view = view_mutex.lock().unwrap(); + emit_columns_for_table_info(&mut program, &view.columns, base_reg); } } let col_names = ["cid", "name", "type", "notnull", "dflt_value", "pk"]; @@ -565,6 +538,45 @@ fn query_pragma( } } +/// Helper function to emit column information for PRAGMA table_info +/// Used by both tables and views since they now have the same column emission logic +fn emit_columns_for_table_info( + program: &mut ProgramBuilder, + columns: &[crate::schema::Column], + base_reg: usize, +) { + // According to the SQLite documentation: "The 'cid' column should not be taken to + // mean more than 'rank within the current result set'." + // Therefore, we enumerate only after filtering out hidden columns. + for (i, column) in columns.iter().filter(|col| !col.hidden).enumerate() { + // cid + program.emit_int(i as i64, base_reg); + // name + program.emit_string8(column.name.clone().unwrap_or_default(), base_reg + 1); + + // type + program.emit_string8(column.ty_str.clone(), base_reg + 2); + + // notnull + program.emit_bool(column.notnull, base_reg + 3); + + // dflt_value + match &column.default { + None => { + program.emit_null(base_reg + 4, None); + } + Some(expr) => { + program.emit_string8(expr.to_string(), base_reg + 4); + } + } + + // pk + program.emit_bool(column.primary_key, base_reg + 5); + + program.emit_result_row(base_reg, 6); + } +} + fn update_auto_vacuum_mode( auto_vacuum_mode: AutoVacuumMode, largest_root_page_number: u32, diff --git a/core/translate/schema.rs b/core/translate/schema.rs index 786e2684f..a30ddf2cc 100644 --- a/core/translate/schema.rs +++ b/core/translate/schema.rs @@ -182,6 +182,7 @@ pub fn translate_create_table( pub enum SchemaEntryType { Table, Index, + View, } impl SchemaEntryType { @@ -189,6 +190,7 @@ impl SchemaEntryType { match self { SchemaEntryType::Table => "table", SchemaEntryType::Index => "index", + SchemaEntryType::View => "view", } } } diff --git a/core/translate/view.rs b/core/translate/view.rs new file mode 100644 index 000000000..f9ab2e989 --- /dev/null +++ b/core/translate/view.rs @@ -0,0 +1,221 @@ +use crate::schema::Schema; +use crate::translate::emitter::Resolver; +use crate::translate::schema::{emit_schema_entry, SchemaEntryType, SQLITE_TABLEID}; +use crate::util::normalize_ident; +use crate::vdbe::builder::{CursorType, ProgramBuilder}; +use crate::vdbe::insn::{CmpInsFlags, Cookie, Insn}; +use crate::{Connection, Result, SymbolTable}; +use std::sync::Arc; +use turso_sqlite3_parser::ast::{self, fmt::ToTokens}; + +pub fn translate_create_view( + schema: &Schema, + view_name: &str, + select_stmt: &ast::Select, + connection: Arc, + syms: &SymbolTable, + mut program: ProgramBuilder, +) -> Result { + // Check if experimental views are enabled + if !connection.experimental_views_enabled() { + return Err(crate::LimboError::ParseError( + "CREATE VIEW is an experimental feature. Enable with --experimental-views flag" + .to_string(), + )); + } + + let normalized_view_name = normalize_ident(view_name); + + // Check if view already exists + if schema.get_view(&normalized_view_name).is_some() { + return Err(crate::LimboError::ParseError(format!( + "View {normalized_view_name} already exists" + ))); + } + + // Validate that this view can be created as an IncrementalView + // This validation happens before updating sqlite_master to prevent + // storing invalid view definitions + use crate::incremental::view::IncrementalView; + IncrementalView::can_create_view(select_stmt, schema)?; + + // Reconstruct the SQL string + let sql = create_view_to_str(view_name, select_stmt); + + // Open cursor to sqlite_schema table + let table = schema.get_btree_table(SQLITE_TABLEID).unwrap(); + let sqlite_schema_cursor_id = program.alloc_cursor_id(CursorType::BTreeTable(table.clone())); + program.emit_insn(Insn::OpenWrite { + cursor_id: sqlite_schema_cursor_id, + root_page: 1usize.into(), + db: 0, + }); + + // Add the view entry to sqlite_schema + let resolver = Resolver::new(schema, syms); + emit_schema_entry( + &mut program, + &resolver, + sqlite_schema_cursor_id, + None, // cdc_table_cursor_id, no cdc for views + SchemaEntryType::View, + &normalized_view_name, + &normalized_view_name, // for views, tbl_name is same as name + 0, // views don't have a root page + Some(sql.clone()), + )?; + + // Parse schema to load the new view + program.emit_insn(Insn::ParseSchema { + db: sqlite_schema_cursor_id, + where_clause: Some(format!("name = '{normalized_view_name}'")), + }); + + // Populate the new view + program.emit_insn(Insn::PopulateViews); + + program.epilogue(schema); + Ok(program) +} + +fn create_view_to_str(view_name: &str, select_stmt: &ast::Select) -> String { + format!( + "CREATE VIEW {} AS {}", + view_name, + select_stmt.format().unwrap() + ) +} + +pub fn translate_drop_view( + schema: &Schema, + view_name: &str, + if_exists: bool, + connection: Arc, + mut program: ProgramBuilder, +) -> Result { + // Check if experimental views are enabled + if !connection.experimental_views_enabled() { + return Err(crate::LimboError::ParseError( + "DROP VIEW is an experimental feature. Enable with --experimental-views flag" + .to_string(), + )); + } + + let normalized_view_name = normalize_ident(view_name); + + // Check if view exists + let view_exists = schema.get_view(&normalized_view_name).is_some(); + + if !view_exists && !if_exists { + return Err(crate::LimboError::ParseError(format!( + "no such view: {normalized_view_name}" + ))); + } + + if !view_exists && if_exists { + // View doesn't exist but IF EXISTS was specified, nothing to do + return Ok(program); + } + + // Open cursor to sqlite_schema table + let schema_table = schema.get_btree_table(SQLITE_TABLEID).unwrap(); + let sqlite_schema_cursor_id = + program.alloc_cursor_id(CursorType::BTreeTable(schema_table.clone())); + program.emit_insn(Insn::OpenWrite { + cursor_id: sqlite_schema_cursor_id, + root_page: 1usize.into(), + db: 0, + }); + + // Allocate registers for searching + let view_name_reg = program.alloc_register(); + let type_reg = program.alloc_register(); + let rowid_reg = program.alloc_register(); + + // Set the view name and type we're looking for + program.emit_insn(Insn::String8 { + dest: view_name_reg, + value: normalized_view_name.clone(), + }); + program.emit_insn(Insn::String8 { + dest: type_reg, + value: "view".to_string(), + }); + + // Start scanning from the beginning + let end_loop_label = program.allocate_label(); + let loop_start_label = program.allocate_label(); + + program.emit_insn(Insn::Rewind { + cursor_id: sqlite_schema_cursor_id, + pc_if_empty: end_loop_label, + }); + program.preassign_label_to_next_insn(loop_start_label); + + // Check if this row is the view we're looking for + // Column 0 is type, Column 1 is name, Column 2 is tbl_name + let col0_reg = program.alloc_register(); + let col1_reg = program.alloc_register(); + + program.emit_column(sqlite_schema_cursor_id, 0, col0_reg); + program.emit_column(sqlite_schema_cursor_id, 1, col1_reg); + + // Check if type == 'view' and name == view_name + let skip_delete_label = program.allocate_label(); + program.emit_insn(Insn::Ne { + lhs: col0_reg, + rhs: type_reg, + target_pc: skip_delete_label, + flags: CmpInsFlags::default(), + collation: program.curr_collation(), + }); + program.emit_insn(Insn::Ne { + lhs: col1_reg, + rhs: view_name_reg, + target_pc: skip_delete_label, + flags: CmpInsFlags::default(), + collation: program.curr_collation(), + }); + + // Get the rowid and delete this row + program.emit_insn(Insn::RowId { + cursor_id: sqlite_schema_cursor_id, + dest: rowid_reg, + }); + program.emit_insn(Insn::Delete { + cursor_id: sqlite_schema_cursor_id, + table_name: "sqlite_schema".to_string(), + }); + + program.resolve_label(skip_delete_label, program.offset()); + + // Move to next row + program.emit_insn(Insn::Next { + cursor_id: sqlite_schema_cursor_id, + pc_if_next: loop_start_label, + }); + + program.preassign_label_to_next_insn(end_loop_label); + + // Remove the view from the in-memory schema + program.emit_insn(Insn::DropView { + db: 0, + view_name: normalized_view_name.clone(), + }); + + // Update schema version (increment schema cookie) + let schema_version_reg = program.alloc_register(); + program.emit_insn(Insn::Integer { + dest: schema_version_reg, + value: (schema.schema_version + 1) as i64, + }); + program.emit_insn(Insn::SetCookie { + db: 0, + cookie: Cookie::SchemaVersion, + value: (schema.schema_version + 1) as i32, + p5: 1, // update version + }); + + program.epilogue(schema); + Ok(program) +} diff --git a/core/util.rs b/core/util.rs index 5534759d1..fafe606bb 100644 --- a/core/util.rs +++ b/core/util.rs @@ -2,17 +2,21 @@ use crate::translate::expr::WalkControl; use crate::types::IOResult; use crate::{ - schema::{self, Column, Schema, Type}, + schema::{self, Column, Schema, Type, ViewsMap}, translate::{collate::CollationSeq, expr::walk_expr, plan::JoinOrderMember}, types::{Value, ValueType}, LimboError, OpenFlags, Result, Statement, StepResult, SymbolTable, }; use crate::{Connection, IO}; -use std::{rc::Rc, sync::Arc}; +use std::{ + rc::Rc, + sync::{Arc, Mutex}, +}; use tracing::{instrument, Level}; use turso_sqlite3_parser::ast::{ - self, CreateTableBody, Expr, FunctionTail, Literal, UnaryOperator, + self, fmt::ToTokens, Cmd, CreateTableBody, Expr, FunctionTail, Literal, Stmt, UnaryOperator, }; +use turso_sqlite3_parser::lexer::sql::Parser; pub trait IOExt { fn block(&self, f: impl FnMut() -> Result>) -> Result; @@ -73,18 +77,22 @@ pub fn parse_schema_rows( schema: &mut Schema, syms: &SymbolTable, mv_tx_id: Option, + mut existing_views: ViewsMap, ) -> Result<()> { rows.set_mv_tx_id(mv_tx_id); // TODO: if we IO, this unparsed indexes is lost. Will probably need some state between // IO runs let mut from_sql_indexes = Vec::with_capacity(10); let mut automatic_indices = std::collections::HashMap::with_capacity(10); + + // Collect views for second pass to populate table_to_views mapping + let mut views_to_process: Vec<(String, Vec)> = Vec::new(); loop { match rows.step()? { StepResult::Row => { let row = rows.row().unwrap(); let ty = row.get::<&str>(0)?; - if !["table", "index"].contains(&ty) { + if !["table", "index", "view"].contains(&ty) { continue; } match ty { @@ -141,6 +149,65 @@ pub fn parse_schema_rows( } } } + "view" => { + use crate::incremental::view::IncrementalView; + use fallible_iterator::FallibleIterator; + + let name: &str = row.get::<&str>(1)?; + let sql: &str = row.get::<&str>(4)?; + let view_name = name.to_string(); + + // Try to remove and potentially reuse an existing view with this name. + // Note: After this function completes, any views not reused are discarded, + // as they are no longer relevant in the new schema. + let should_create_new = + if let Some(existing_view) = existing_views.remove(&view_name) { + // Check if we can reuse this view (same SQL definition) + let can_reuse = if let Ok(view_guard) = existing_view.lock() { + view_guard.has_same_sql(sql) + } else { + false + }; + + if can_reuse { + // Reuse the existing view - it's already populated! + let referenced_tables = + if let Ok(view_guard) = existing_view.lock() { + view_guard.get_referenced_table_names() + } else { + vec![] + }; + + // Add the existing view to the new schema + schema.views.insert(view_name.clone(), existing_view); + + // Store for second pass processing + views_to_process.push((view_name.clone(), referenced_tables)); + false // Don't create new + } else { + true // SQL changed, need to create new + } + } else { + true // No existing view, need to create new + }; + + if should_create_new { + // Create a new IncrementalView + match IncrementalView::from_sql(sql, schema) { + Ok(incremental_view) => { + let referenced_tables = + incremental_view.get_referenced_table_names(); + schema.add_view(incremental_view); + views_to_process.push((view_name, referenced_tables)); + } + Err(e) => { + eprintln!( + "Warning: Could not create incremental view for {name}: {e:?}" + ); + } + } + } + } _ => continue, } } @@ -183,6 +250,15 @@ pub fn parse_schema_rows( } } } + + // Second pass: populate table_to_views mapping + for (view_name, referenced_tables) in views_to_process { + // Register this view as dependent on each referenced table + for table_name in referenced_tables { + schema.add_view_dependency(&table_name, &view_name); + } + } + Ok(()) } @@ -1094,6 +1170,151 @@ pub fn parse_pragma_bool(expr: &Expr) -> Result { )) } +/// Extract column name from an expression (e.g., for SELECT clauses) +pub fn extract_column_name_from_expr(expr: &ast::Expr) -> Option { + match expr { + ast::Expr::Id(name) => Some(name.as_str().to_string()), + ast::Expr::Qualified(_, name) => Some(name.as_str().to_string()), + _ => None, + } +} + +/// Extract column information from a SELECT statement for view creation +pub fn extract_view_columns(select_stmt: &ast::Select, schema: &Schema) -> Vec { + let mut columns = Vec::new(); + // Navigate to the first SELECT in the statement + if let ast::OneSelect::Select(select_core) = select_stmt.body.select.as_ref() { + // First, we need to figure out which table(s) are being selected from + let table_name = if let Some(from) = &select_core.from { + if let Some(ast::SelectTable::Table(qualified_name, _, _)) = from.select.as_deref() { + Some(normalize_ident(qualified_name.name.as_str())) + } else { + None + } + } else { + None + }; + // Get the table for column resolution + let _table = table_name.as_ref().and_then(|name| schema.get_table(name)); + // Process each column in the SELECT list + for (i, result_col) in select_core.columns.iter().enumerate() { + match result_col { + ast::ResultColumn::Expr(expr, alias) => { + let name = alias + .as_ref() + .map(|a| match a { + ast::As::Elided(name) => name.as_str().to_string(), + ast::As::As(name) => name.as_str().to_string(), + }) + .or_else(|| extract_column_name_from_expr(expr)) + .unwrap_or_else(|| { + // If we can't extract a simple column name, use the expression itself + expr.format().unwrap_or_else(|_| format!("column_{i}")) + }); + columns.push(Column { + name: Some(name), + ty: Type::Text, // Default to TEXT, could be refined with type analysis + ty_str: "TEXT".to_string(), + primary_key: false, // Views don't have primary keys + is_rowid_alias: false, + notnull: false, // Views typically don't enforce NOT NULL + default: None, // Views don't have default values + unique: false, + collation: None, + hidden: false, + }); + } + ast::ResultColumn::Star => { + // For SELECT *, expand to all columns from the table + if let Some(ref table_name) = table_name { + if let Some(table) = schema.get_table(table_name) { + // Copy all columns from the table, but adjust for view constraints + for table_column in table.columns() { + columns.push(Column { + name: table_column.name.clone(), + ty: table_column.ty, + ty_str: table_column.ty_str.clone(), + primary_key: false, // Views don't have primary keys + is_rowid_alias: false, + notnull: false, // Views typically don't enforce NOT NULL + default: None, // Views don't have default values + unique: false, + collation: table_column.collation, + hidden: false, + }); + } + } else { + // Table not found, create placeholder + columns.push(Column { + name: Some("*".to_string()), + ty: Type::Text, + ty_str: "TEXT".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }); + } + } else { + // No FROM clause or couldn't determine table, create placeholder + columns.push(Column { + name: Some("*".to_string()), + ty: Type::Text, + ty_str: "TEXT".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }); + } + } + ast::ResultColumn::TableStar(table_name) => { + // For table.*, expand to all columns from the specified table + let table_name_str = normalize_ident(table_name.as_str()); + if let Some(table) = schema.get_table(&table_name_str) { + // Copy all columns from the table, but adjust for view constraints + for table_column in table.columns() { + columns.push(Column { + name: table_column.name.clone(), + ty: table_column.ty, + ty_str: table_column.ty_str.clone(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: table_column.collation, + hidden: false, + }); + } + } else { + // Table not found, create placeholder + columns.push(Column { + name: Some(format!("{table_name_str}.*")), + ty: Type::Text, + ty_str: "TEXT".to_string(), + primary_key: false, + is_rowid_alias: false, + notnull: false, + default: None, + unique: false, + collation: None, + hidden: false, + }); + } + } + } + } + } + columns +} + #[cfg(test)] pub mod tests { use super::*; diff --git a/core/vdbe/execute.rs b/core/vdbe/execute.rs index 856e9fcbb..ad233b0f3 100644 --- a/core/vdbe/execute.rs +++ b/core/vdbe/execute.rs @@ -5121,7 +5121,7 @@ pub fn op_insert( key_reg, record_reg, flag, - table_name: _, + table_name, }, insn ); @@ -5164,6 +5164,72 @@ pub fn op_insert( Register::Aggregate(..) => unreachable!("Cannot insert an aggregate value."), }; + // Update dependent views for incremental computation + let schema = program.connection.schema.borrow(); + let dependent_views = schema.get_dependent_views(table_name); + + if !dependent_views.is_empty() { + // If this is an UPDATE operation, first capture and delete the old row data + if flag.has(InsertFlags::UPDATE) { + // Get the old record before it's overwritten + let old_record_values = if let Some(old_record) = return_if_io!(cursor.record()) { + let mut values = old_record + .get_values() + .into_iter() + .map(|v| v.to_owned()) + .collect::>(); + + // Fix rowid alias columns: replace Null with actual rowid value + let schema = program.connection.schema.borrow(); + if let Some(table) = schema.get_table(table_name) { + for (i, col) in table.columns().iter().enumerate() { + if col.is_rowid_alias && i < values.len() { + values[i] = Value::Integer(key); + } + } + } + drop(schema); + + Some(values) + } else { + None + }; + + // Add deletion of old row to view deltas + if let Some(old_values) = old_record_values { + let mut tx_states = program.connection.view_transaction_states.borrow_mut(); + for view_name in &dependent_views { + let tx_state = tx_states.entry(view_name.clone()).or_default(); + tx_state.delta.delete(key, old_values.clone()); + } + } + } + + // Add insertion of new row to view deltas + let mut new_values = record + .get_values() + .into_iter() + .map(|v| v.to_owned()) + .collect::>(); + + // Fix rowid alias columns: replace Null with actual rowid value + let schema = program.connection.schema.borrow(); + if let Some(table) = schema.get_table(table_name) { + for (i, col) in table.columns().iter().enumerate() { + if col.is_rowid_alias && i < new_values.len() { + new_values[i] = Value::Integer(key); + } + } + } + drop(schema); + + let mut tx_states = program.connection.view_transaction_states.borrow_mut(); + for view_name in dependent_views { + let tx_state = tx_states.entry(view_name.clone()).or_default(); + tx_state.delta.insert(key, new_values.clone()); + } + } + // In a table insert, if the caller does not pass InsertFlags::REQUIRE_SEEK, they must ensure that a seek has already happened to the correct location. // This typically happens by invoking either Insn::NewRowid or Insn::NotExists, because: // 1. op_new_rowid() seeks to the end of the table, which is the correct insertion position. @@ -5220,14 +5286,59 @@ pub fn op_delete( load_insn!( Delete { cursor_id, - table_name: _ + table_name }, insn ); - { + + // Capture row data before deletion for view updates + let (key, record_values) = { let mut cursor = state.get_cursor(*cursor_id); let cursor = cursor.as_btree_mut(); + + // Get the current key + let maybe_key = return_if_io!(cursor.rowid()); + let key = maybe_key.ok_or_else(|| { + LimboError::InternalError("Cannot delete: no current row".to_string()) + })?; + + // Get the current record before deletion and extract values + let record_values = if let Some(record) = return_if_io!(cursor.record()) { + let mut values = record + .get_values() + .into_iter() + .map(|v| v.to_owned()) + .collect::>(); + + // Fix rowid alias columns: replace Null with actual rowid value + let schema = program.connection.schema.borrow(); + if let Some(table) = schema.get_table(table_name) { + for (i, col) in table.columns().iter().enumerate() { + if col.is_rowid_alias && i < values.len() { + values[i] = Value::Integer(key); + } + } + } + Some(values) + } else { + None + }; + + // Now perform the deletion return_if_io!(cursor.delete()); + + (key, record_values) + }; + + // Update dependent views for incremental computation + if let Some(values) = record_values { + let schema = program.connection.schema.borrow(); + let dependent_views = schema.get_dependent_views(table_name); + let mut tx_states = program.connection.view_transaction_states.borrow_mut(); + for view_name in dependent_views { + let tx_state = tx_states.entry(view_name).or_default(); + tx_state.delta.delete(key, values.clone()); + } } let prev_changes = program.n_change.get(); program.n_change.set(prev_changes + 1); @@ -6040,6 +6151,26 @@ pub fn op_drop_table( Ok(InsnFunctionStepResult::Step) } +pub fn op_drop_view( + program: &Program, + state: &mut ProgramState, + insn: &Insn, + _pager: &Rc, + _mv_store: Option<&Arc>, +) -> Result { + load_insn!(DropView { db, view_name }, insn); + if *db > 0 { + todo!("temp databases not implemented yet"); + } + let conn = program.connection.clone(); + conn.with_schema_mut(|schema| { + schema.remove_view(view_name); + Ok::<(), crate::LimboError>(()) + })?; + state.pc += 1; + Ok(InsnFunctionStepResult::Step) +} + pub fn op_close( program: &Program, state: &mut ProgramState, @@ -6140,14 +6271,28 @@ pub fn op_parse_schema( conn.with_schema_mut(|schema| { // TODO: This function below is synchronous, make it async - parse_schema_rows(stmt, schema, &conn.syms.borrow(), state.mv_tx_id) + let existing_views = schema.views.clone(); + parse_schema_rows( + stmt, + schema, + &conn.syms.borrow(), + state.mv_tx_id, + existing_views, + ) })?; } else { let stmt = conn.prepare("SELECT * FROM sqlite_schema")?; conn.with_schema_mut(|schema| { // TODO: This function below is synchronous, make it async - parse_schema_rows(stmt, schema, &conn.syms.borrow(), state.mv_tx_id) + let existing_views = schema.views.clone(); + parse_schema_rows( + stmt, + schema, + &conn.syms.borrow(), + state.mv_tx_id, + existing_views, + ) })?; } conn.auto_commit.set(previous_auto_commit); @@ -6155,6 +6300,29 @@ pub fn op_parse_schema( Ok(InsnFunctionStepResult::Step) } +pub fn op_populate_views( + program: &Program, + state: &mut ProgramState, + _insn: &Insn, + _pager: &Rc, + _mv_store: Option<&Arc>, +) -> Result { + let conn = program.connection.clone(); + let schema = conn.schema.borrow(); + + match schema.populate_views(&conn)? { + IOResult::Done(()) => { + // All views populated, advance to next instruction + state.pc += 1; + Ok(InsnFunctionStepResult::Step) + } + IOResult::IO => { + // Need more IO, stay on this instruction + Ok(InsnFunctionStepResult::IO) + } + } +} + pub fn op_read_cookie( program: &Program, state: &mut ProgramState, diff --git a/core/vdbe/explain.rs b/core/vdbe/explain.rs index 6f2f4352e..a54b01ef9 100644 --- a/core/vdbe/explain.rs +++ b/core/vdbe/explain.rs @@ -1280,6 +1280,15 @@ pub fn insn_to_str( 0, format!("DROP TABLE {table_name}"), ), + Insn::DropView { db, view_name } => ( + "DropView", + *db as i32, + 0, + 0, + Value::build_text(view_name), + 0, + format!("DROP VIEW {view_name}"), + ), Insn::DropIndex { db: _, index } => ( "DropIndex", 0, @@ -1328,6 +1337,15 @@ pub fn insn_to_str( 0, where_clause.clone().unwrap_or("NULL".to_string()), ), + Insn::PopulateViews => ( + "PopulateViews", + 0, + 0, + 0, + Value::Null, + 0, + "".to_string(), + ), Insn::Prev { cursor_id, pc_if_prev, diff --git a/core/vdbe/insn.rs b/core/vdbe/insn.rs index dce8ea938..4b835f1d9 100644 --- a/core/vdbe/insn.rs +++ b/core/vdbe/insn.rs @@ -122,6 +122,11 @@ impl InsertFlags { self.0 |= InsertFlags::REQUIRE_SEEK; self } + + pub fn update(mut self) -> Self { + self.0 |= InsertFlags::UPDATE; + self + } } #[derive(Clone, Copy, Debug)] @@ -845,6 +850,12 @@ pub enum Insn { // The name of the table being dropped table_name: String, }, + DropView { + /// The database within which this view needs to be dropped + db: usize, + /// The name of the view being dropped + view_name: String, + }, DropIndex { /// The database within which this index needs to be dropped (P1). db: usize, @@ -886,6 +897,9 @@ pub enum Insn { where_clause: Option, }, + /// Populate all views after schema parsing + PopulateViews, + /// Place the result of lhs >> rhs in dest register. ShiftRight { lhs: usize, @@ -1161,10 +1175,12 @@ impl Insn { Insn::Destroy { .. } => execute::op_destroy, Insn::DropTable { .. } => execute::op_drop_table, + Insn::DropView { .. } => execute::op_drop_view, Insn::Close { .. } => execute::op_close, Insn::IsNull { .. } => execute::op_is_null, Insn::CollSeq { .. } => execute::op_coll_seq, Insn::ParseSchema { .. } => execute::op_parse_schema, + Insn::PopulateViews => execute::op_populate_views, Insn::ShiftRight { .. } => execute::op_shift_right, Insn::ShiftLeft { .. } => execute::op_shift_left, Insn::AddImm { .. } => execute::op_add_imm, diff --git a/core/vdbe/mod.rs b/core/vdbe/mod.rs index 99d115095..a31105c1d 100644 --- a/core/vdbe/mod.rs +++ b/core/vdbe/mod.rs @@ -443,6 +443,21 @@ impl Program { } #[instrument(skip_all, level = Level::DEBUG)] + fn apply_view_deltas(&self, rollback: bool) { + let tx_states = self.connection.view_transaction_states.take(); + + if !rollback { + let schema = self.connection.schema.borrow(); + + for (view_name, tx_state) in tx_states.iter() { + if let Some(view_mutex) = schema.get_view(view_name) { + let mut view = view_mutex.lock().unwrap(); + view.merge_delta(&tx_state.delta); + } + } + } + } + pub fn commit_txn( &self, pager: Rc, @@ -450,6 +465,8 @@ impl Program { mv_store: Option<&Arc>, rollback: bool, ) -> Result { + self.apply_view_deltas(rollback); + if self.connection.transaction_state.get() == TransactionState::None && mv_store.is_none() { // No need to do any work here if not in tx. Current MVCC logic doesn't work with this assumption, // hence the mv_store.is_none() check. diff --git a/core/vtab.rs b/core/vtab.rs index f26ca1e8e..686158e57 100644 --- a/core/vtab.rs +++ b/core/vtab.rs @@ -6,7 +6,7 @@ use fallible_iterator::FallibleIterator; use std::ffi::c_void; use std::ptr::NonNull; use std::rc::Rc; -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use turso_ext::{ConstraintInfo, IndexInfo, OrderByInfo, ResultCode, VTabKind, VTabModuleImpl}; use turso_sqlite3_parser::{ast, lexer::sql::Parser}; @@ -14,6 +14,7 @@ use turso_sqlite3_parser::{ast, lexer::sql::Parser}; pub(crate) enum VirtualTableType { Pragma(PragmaVirtualTable), External(ExtVirtualTable), + View(crate::vtab_view::ViewVirtualTable), } #[derive(Clone, Debug)] @@ -29,6 +30,7 @@ impl VirtualTable { match &self.vtab_type { VirtualTableType::Pragma(_) => true, VirtualTableType::External(table) => table.readonly(), + VirtualTableType::View(_) => true, } } @@ -86,6 +88,21 @@ impl VirtualTable { Ok(Arc::new(vtab)) } + /// Create a virtual table for a view + pub(crate) fn view( + view_name: &str, + columns: Vec, + view: Arc>, + ) -> crate::Result> { + let vtab = VirtualTable { + name: view_name.to_owned(), + columns, + kind: VTabKind::VirtualTable, + vtab_type: VirtualTableType::View(crate::vtab_view::ViewVirtualTable { view }), + }; + Ok(Arc::new(vtab)) + } + fn resolve_columns(schema: String) -> crate::Result> { let mut parser = Parser::new(schema.as_bytes()); if let ast::Cmd::Stmt(ast::Stmt::CreateTable { body, .. }) = parser.next()?.ok_or( @@ -107,6 +124,9 @@ impl VirtualTable { VirtualTableType::External(table) => { Ok(VirtualTableCursor::External(table.open(conn.clone())?)) } + VirtualTableType::View(table) => { + Ok(VirtualTableCursor::View(Box::new(table.open(conn)?))) + } } } @@ -114,6 +134,7 @@ impl VirtualTable { match &self.vtab_type { VirtualTableType::Pragma(_) => Err(LimboError::ReadOnly), VirtualTableType::External(table) => table.update(args), + VirtualTableType::View(_) => Err(LimboError::ReadOnly), } } @@ -121,6 +142,7 @@ impl VirtualTable { match &self.vtab_type { VirtualTableType::Pragma(_) => Ok(()), VirtualTableType::External(table) => table.destroy(), + VirtualTableType::View(_) => Ok(()), } } @@ -132,6 +154,7 @@ impl VirtualTable { match &self.vtab_type { VirtualTableType::Pragma(table) => table.best_index(constraints), VirtualTableType::External(table) => table.best_index(constraints, order_by), + VirtualTableType::View(view) => view.best_index(), } } } @@ -139,6 +162,7 @@ impl VirtualTable { pub enum VirtualTableCursor { Pragma(Box), External(ExtVirtualTableCursor), + View(Box), } impl VirtualTableCursor { @@ -146,6 +170,7 @@ impl VirtualTableCursor { match self { VirtualTableCursor::Pragma(cursor) => cursor.next(), VirtualTableCursor::External(cursor) => cursor.next(), + VirtualTableCursor::View(cursor) => cursor.next(), } } @@ -153,6 +178,7 @@ impl VirtualTableCursor { match self { VirtualTableCursor::Pragma(cursor) => cursor.rowid(), VirtualTableCursor::External(cursor) => cursor.rowid(), + VirtualTableCursor::View(cursor) => cursor.rowid(), } } @@ -160,6 +186,7 @@ impl VirtualTableCursor { match self { VirtualTableCursor::Pragma(cursor) => cursor.column(column), VirtualTableCursor::External(cursor) => cursor.column(column), + VirtualTableCursor::View(cursor) => cursor.column(column), } } @@ -175,6 +202,7 @@ impl VirtualTableCursor { VirtualTableCursor::External(cursor) => { cursor.filter(idx_num, idx_str, arg_count, args) } + VirtualTableCursor::View(cursor) => cursor.filter(args), } } } diff --git a/core/vtab_view.rs b/core/vtab_view.rs new file mode 100644 index 000000000..4b44f0592 --- /dev/null +++ b/core/vtab_view.rs @@ -0,0 +1,101 @@ +use crate::incremental::view::IncrementalView; +use crate::{Connection, LimboError, Value, VirtualTable}; +use std::sync::{Arc, Mutex}; + +/// Create a virtual table wrapper for a view +pub fn create_view_virtual_table( + view_name: &str, + view: Arc>, +) -> crate::Result> { + // Use the VirtualTable::view method we added + let view_locked = view.lock().map_err(|_| { + LimboError::InternalError("Failed to lock view for virtual table creation".to_string()) + })?; + let columns = view_locked.columns.clone(); + drop(view_locked); // Release the lock before passing the Arc + VirtualTable::view(view_name, columns, view) +} + +/// Virtual table wrapper for incremental views +#[derive(Clone, Debug)] +pub struct ViewVirtualTable { + pub view: Arc>, +} + +impl ViewVirtualTable { + pub fn best_index(&self) -> Result { + // Views don't use indexes - return a simple index info + Ok(turso_ext::IndexInfo { + idx_num: 0, + idx_str: None, + order_by_consumed: false, + estimated_cost: 1000000.0, + estimated_rows: 1000, + constraint_usages: Vec::new(), + }) + } + + pub fn open(&self, conn: Arc) -> crate::Result { + // Views are now populated during schema parsing (in parse_schema_rows) + // so we just get the current data from the view. + + let view = self.view.lock().map_err(|_| { + LimboError::InternalError("Failed to lock view for reading".to_string()) + })?; + + let tx_states = conn.view_transaction_states.borrow(); + let tx_state = tx_states.get(view.name()); + + let data: Vec<(i64, Vec)> = view.current_data(tx_state); + Ok(ViewVirtualTableCursor { + data, + current_pos: 0, + }) + } +} + +/// Cursor for iterating over view data +pub struct ViewVirtualTableCursor { + data: Vec<(i64, Vec)>, + current_pos: usize, +} + +impl ViewVirtualTableCursor { + pub fn next(&mut self) -> crate::Result { + if self.current_pos < self.data.len() { + self.current_pos += 1; + Ok(self.current_pos < self.data.len()) + } else { + Ok(false) + } + } + + pub fn rowid(&self) -> i64 { + if self.current_pos < self.data.len() { + self.data[self.current_pos].0 + } else { + -1 + } + } + + pub fn column(&self, column: usize) -> crate::Result { + if self.current_pos >= self.data.len() { + return Ok(Value::Null); + } + + let (_row_key, values) = &self.data[self.current_pos]; + + // Return the value at the requested column index + if let Some(value) = values.get(column) { + Ok(value.clone()) + } else { + Ok(Value::Null) + } + } + + pub fn filter(&mut self, _args: Vec) -> crate::Result { + // Reset to beginning for new filter + self.current_pos = 0; + Ok(!self.data.is_empty()) + } +} diff --git a/tests/integration/common.rs b/tests/integration/common.rs index 8d494c68a..058d44787 100644 --- a/tests/integration/common.rs +++ b/tests/integration/common.rs @@ -31,6 +31,7 @@ impl TempDatabase { turso_core::OpenFlags::default(), false, enable_indexes, + false, ) .unwrap(); Self { path, io, db } @@ -56,6 +57,7 @@ impl TempDatabase { flags, false, enable_indexes, + false, ) .unwrap(); Self { @@ -85,6 +87,7 @@ impl TempDatabase { turso_core::OpenFlags::default(), false, enable_indexes, + false, ) .unwrap(); diff --git a/tests/integration/query_processing/test_write_path.rs b/tests/integration/query_processing/test_write_path.rs index fc1ff8976..96d051894 100644 --- a/tests/integration/query_processing/test_write_path.rs +++ b/tests/integration/query_processing/test_write_path.rs @@ -706,6 +706,7 @@ fn test_wal_bad_frame() -> anyhow::Result<()> { turso_core::OpenFlags::default(), false, false, + false, ) .unwrap(); let tmp_db = TempDatabase { From 87322ad1e4a247010655745d0a443f1997955549 Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 08:23:53 +0300 Subject: [PATCH 04/10] core/incremental: Evaluate view expressions ...tests were failing because we are testing with expressions, but didn't support them. --- core/incremental/operator.rs | 152 +++++++++++++++++++++++++++++++++-- core/incremental/view.rs | 24 +++++- 2 files changed, 166 insertions(+), 10 deletions(-) diff --git a/core/incremental/operator.rs b/core/incremental/operator.rs index 194d86013..a2f4c4b93 100644 --- a/core/incremental/operator.rs +++ b/core/incremental/operator.rs @@ -335,9 +335,9 @@ impl FilterPredicate { pub enum ProjectColumn { /// Direct column reference Column(String), - /// Computed expression (simplified for now) + /// Computed expression Expression { - expr: String, // For now, just store as string + expr: turso_sqlite3_parser::ast::Expr, alias: Option, }, } @@ -625,16 +625,156 @@ impl ProjectOperator { output.push(Value::Null); } } - ProjectColumn::Expression { .. } => { - // For now, just pass through a null - // In a real implementation, we'd evaluate the expression - output.push(Value::Null); + ProjectColumn::Expression { expr, .. } => { + // Evaluate the expression + let result = self.evaluate_expression(expr, values); + output.push(result); } } } output } + + fn evaluate_expression( + &self, + expr: &turso_sqlite3_parser::ast::Expr, + values: &[Value], + ) -> Value { + use turso_sqlite3_parser::ast::*; + + match expr { + Expr::Id(name) => { + if let Some(idx) = self + .input_column_names + .iter() + .position(|c| c == name.as_str()) + { + if let Some(v) = values.get(idx) { + return v.clone(); + } + } + Value::Null + } + Expr::Literal(lit) => { + match lit { + Literal::Numeric(n) => { + if let Ok(i) = n.parse::() { + Value::Integer(i) + } else if let Ok(f) = n.parse::() { + Value::Float(f) + } else { + Value::Null + } + } + Literal::String(s) => { + let cleaned = s.trim_matches('\'').trim_matches('"'); + Value::Text(Text::new(cleaned)) + } + Literal::Null => Value::Null, + Literal::Blob(_) + | Literal::Keyword(_) + | Literal::CurrentDate + | Literal::CurrentTime + | Literal::CurrentTimestamp => Value::Null, // Not supported yet + } + } + Expr::Binary(left, op, right) => { + let left_val = self.evaluate_expression(left, values); + let right_val = self.evaluate_expression(right, values); + + match op { + Operator::Add => match (&left_val, &right_val) { + (Value::Integer(a), Value::Integer(b)) => Value::Integer(a + b), + (Value::Float(a), Value::Float(b)) => Value::Float(a + b), + (Value::Integer(a), Value::Float(b)) => Value::Float(*a as f64 + b), + (Value::Float(a), Value::Integer(b)) => Value::Float(a + *b as f64), + _ => Value::Null, + }, + Operator::Subtract => match (&left_val, &right_val) { + (Value::Integer(a), Value::Integer(b)) => Value::Integer(a - b), + (Value::Float(a), Value::Float(b)) => Value::Float(a - b), + (Value::Integer(a), Value::Float(b)) => Value::Float(*a as f64 - b), + (Value::Float(a), Value::Integer(b)) => Value::Float(a - *b as f64), + _ => Value::Null, + }, + Operator::Multiply => match (&left_val, &right_val) { + (Value::Integer(a), Value::Integer(b)) => Value::Integer(a * b), + (Value::Float(a), Value::Float(b)) => Value::Float(a * b), + (Value::Integer(a), Value::Float(b)) => Value::Float(*a as f64 * b), + (Value::Float(a), Value::Integer(b)) => Value::Float(a * *b as f64), + _ => Value::Null, + }, + Operator::Divide => match (&left_val, &right_val) { + (Value::Integer(a), Value::Integer(b)) => { + if *b != 0 { + Value::Integer(a / b) + } else { + Value::Null + } + } + (Value::Float(a), Value::Float(b)) => { + if *b != 0.0 { + Value::Float(a / b) + } else { + Value::Null + } + } + (Value::Integer(a), Value::Float(b)) => { + if *b != 0.0 { + Value::Float(*a as f64 / b) + } else { + Value::Null + } + } + (Value::Float(a), Value::Integer(b)) => { + if *b != 0 { + Value::Float(a / *b as f64) + } else { + Value::Null + } + } + _ => Value::Null, + }, + _ => Value::Null, // Other operators not supported yet + } + } + Expr::FunctionCall { name, args, .. } => { + match name.as_str().to_lowercase().as_str() { + "hex" => { + if let Some(arg_list) = args { + if arg_list.len() == 1 { + let arg_val = self.evaluate_expression(&arg_list[0], values); + match arg_val { + Value::Integer(i) => { + Value::Text(Text::new(&format!("{:X}", i))) + } + _ => Value::Null, + } + } else { + Value::Null + } + } else { + Value::Null + } + } + _ => Value::Null, // Other functions not supported yet + } + } + Expr::Parenthesized(inner) => { + assert!( + inner.len() <= 1, + "Parenthesized expressions with multiple elements are not supported" + ); + if !inner.is_empty() { + self.evaluate_expression(&inner[0], values) + } else { + Value::Null + } + } + _ => Value::Null, // Other expression types not supported yet + } + } } impl IncrementalOperator for ProjectOperator { diff --git a/core/incremental/view.rs b/core/incremental/view.rs index 89e9833ef..e9908f8b5 100644 --- a/core/incremental/view.rs +++ b/core/incremental/view.rs @@ -828,8 +828,25 @@ impl IncrementalView { for result_col in &select_stmt.columns { match result_col { - ResultColumn::Expr(Expr::Id(name), _) => { - columns.push(ProjectColumn::Column(name.as_str().to_string())); + ResultColumn::Expr(expr, alias) => { + match expr { + Expr::Id(name) => { + // Simple column reference + columns.push(ProjectColumn::Column(name.as_str().to_string())); + } + _ => { + // Expression - store it for evaluation + let alias_str = if let Some(As::As(alias_name)) = alias { + Some(alias_name.as_str().to_string()) + } else { + None + }; + columns.push(ProjectColumn::Expression { + expr: expr.clone(), + alias: alias_str, + }); + } + } } ResultColumn::Star => { // Select all columns @@ -838,8 +855,7 @@ impl IncrementalView { } } _ => { - // For now, skip complex expressions - // Could be extended to handle more cases + // For now, skip TableStar and other cases } } } From 62f1fd2038336859a9db216f0e4436452fbe5b81 Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 08:36:53 +0300 Subject: [PATCH 05/10] core/incremental: Make clippy happy --- core/incremental/operator.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/incremental/operator.rs b/core/incremental/operator.rs index a2f4c4b93..9bc50a444 100644 --- a/core/incremental/operator.rs +++ b/core/incremental/operator.rs @@ -746,9 +746,7 @@ impl ProjectOperator { if arg_list.len() == 1 { let arg_val = self.evaluate_expression(&arg_list[0], values); match arg_val { - Value::Integer(i) => { - Value::Text(Text::new(&format!("{:X}", i))) - } + Value::Integer(i) => Value::Text(Text::new(&format!("{i:X}"))), _ => Value::Null, } } else { From 6f3278f6ed96bac25a77aee1d03506e381560008 Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 10:26:16 +0300 Subject: [PATCH 06/10] core/vdbe: Perform view processing in op_delete only if views are enabled Like in op_insert(), let's perform view processing only if views are actually enabled. --- core/vdbe/execute.rs | 33 +++++++++++++++++++-------------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/core/vdbe/execute.rs b/core/vdbe/execute.rs index 4934e4f2f..c3c03acf7 100644 --- a/core/vdbe/execute.rs +++ b/core/vdbe/execute.rs @@ -5301,24 +5301,29 @@ pub fn op_delete( LimboError::InternalError("Cannot delete: no current row".to_string()) })?; - // Get the current record before deletion and extract values - let record_values = if let Some(record) = return_if_io!(cursor.record()) { - let mut values = record - .get_values() - .into_iter() - .map(|v| v.to_owned()) - .collect::>(); + let schema = program.connection.schema.borrow(); + let dependent_views = schema.get_dependent_views(table_name); + let record_values = if !dependent_views.is_empty() { + // Get the current record before deletion and extract values + if let Some(record) = return_if_io!(cursor.record()) { + let mut values = record + .get_values() + .into_iter() + .map(|v| v.to_owned()) + .collect::>(); - // Fix rowid alias columns: replace Null with actual rowid value - let schema = program.connection.schema.borrow(); - if let Some(table) = schema.get_table(table_name) { - for (i, col) in table.columns().iter().enumerate() { - if col.is_rowid_alias && i < values.len() { - values[i] = Value::Integer(key); + // Fix rowid alias columns: replace Null with actual rowid value + if let Some(table) = schema.get_table(table_name) { + for (i, col) in table.columns().iter().enumerate() { + if col.is_rowid_alias && i < values.len() { + values[i] = Value::Integer(key); + } } } + Some(values) + } else { + None } - Some(values) } else { None }; From 3aa885690ca1254442c24fc44d72f360c251072f Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 10:56:58 +0300 Subject: [PATCH 07/10] core/vdbe: Don't call rowid() unless view processing is enabled Currently, the simulator complains of the following error: ``` Error: failed with error: 'attempt to multiply with overflow' ``` However, we don't enable views in the simulator so -- despite being an issue -- we should never see this. Let's fix `op_delete()` some more not to not even call rowid() unless view processing is enabled. --- core/vdbe/execute.rs | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/core/vdbe/execute.rs b/core/vdbe/execute.rs index c3c03acf7..673d85a67 100644 --- a/core/vdbe/execute.rs +++ b/core/vdbe/execute.rs @@ -5291,19 +5291,18 @@ pub fn op_delete( ); // Capture row data before deletion for view updates - let (key, record_values) = { + let record_key_and_values = { let mut cursor = state.get_cursor(*cursor_id); let cursor = cursor.as_btree_mut(); - // Get the current key - let maybe_key = return_if_io!(cursor.rowid()); - let key = maybe_key.ok_or_else(|| { - LimboError::InternalError("Cannot delete: no current row".to_string()) - })?; - let schema = program.connection.schema.borrow(); let dependent_views = schema.get_dependent_views(table_name); - let record_values = if !dependent_views.is_empty() { + let result = if !dependent_views.is_empty() { + // Get the current key + let maybe_key = return_if_io!(cursor.rowid()); + let key = maybe_key.ok_or_else(|| { + LimboError::InternalError("Cannot delete: no current row".to_string()) + })?; // Get the current record before deletion and extract values if let Some(record) = return_if_io!(cursor.record()) { let mut values = record @@ -5320,7 +5319,7 @@ pub fn op_delete( } } } - Some(values) + Some((key, values)) } else { None } @@ -5331,11 +5330,11 @@ pub fn op_delete( // Now perform the deletion return_if_io!(cursor.delete()); - (key, record_values) + result }; // Update dependent views for incremental computation - if let Some(values) = record_values { + if let Some((key, values)) = record_key_and_values { let schema = program.connection.schema.borrow(); let dependent_views = schema.get_dependent_views(table_name); let mut tx_states = program.connection.view_transaction_states.borrow_mut(); From 4923d95ca390768597c886b4bbfb6c4d2752dee3 Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 11:43:23 +0300 Subject: [PATCH 08/10] cli: Fix schema object ordering SQLite orders schema objects by rowid, not by name. --- cli/app.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cli/app.rs b/cli/app.rs index 8d82cd46a..940baef9b 100644 --- a/cli/app.rs +++ b/cli/app.rs @@ -1059,7 +1059,7 @@ impl Limbo { table_name: &str, ) -> anyhow::Result { let sql = format!( - "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND tbl_name = '{table_name}' OR name = '{table_name}' AND name NOT LIKE 'sqlite_%' ORDER BY type, name" + "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND tbl_name = '{table_name}' OR name = '{table_name}' AND name NOT LIKE 'sqlite_%' ORDER BY type, rowid" ); let mut found = false; @@ -1092,7 +1092,7 @@ impl Limbo { db_prefix: &str, db_display_name: &str, ) -> anyhow::Result<()> { - let sql = format!("SELECT sql, type, name FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND name NOT LIKE 'sqlite_%' ORDER BY CASE type WHEN 'table' THEN 1 WHEN 'view' THEN 2 WHEN 'index' THEN 3 END, name"); + let sql = format!("SELECT sql, type, name FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND name NOT LIKE 'sqlite_%' ORDER BY CASE type WHEN 'table' THEN 1 WHEN 'view' THEN 2 WHEN 'index' THEN 3 END, rowid"); match self.conn.query(&sql) { Ok(Some(ref mut rows)) => loop { From e829201fdd266d31d629ea1f07a389b1c6f61c6b Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 11:52:57 +0300 Subject: [PATCH 09/10] cli: Fix `.schema` SQL statement some more We need to fetch name and type too, but also sort in both places consistently. --- cli/app.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cli/app.rs b/cli/app.rs index 940baef9b..8bb391c47 100644 --- a/cli/app.rs +++ b/cli/app.rs @@ -1059,7 +1059,7 @@ impl Limbo { table_name: &str, ) -> anyhow::Result { let sql = format!( - "SELECT sql FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND tbl_name = '{table_name}' OR name = '{table_name}' AND name NOT LIKE 'sqlite_%' ORDER BY type, rowid" + "SELECT sql, type, name FROM {db_prefix}.sqlite_schema WHERE type IN ('table', 'index', 'view') AND (tbl_name = '{table_name}' OR name = '{table_name}') AND name NOT LIKE 'sqlite_%' ORDER BY CASE type WHEN 'table' THEN 1 WHEN 'view' THEN 2 WHEN 'index' THEN 3 END, rowid" ); let mut found = false; From cdaea7f27440535c851c2a18a3eeee263aaff8c5 Mon Sep 17 00:00:00 2001 From: Pekka Enberg Date: Mon, 11 Aug 2025 12:26:11 +0300 Subject: [PATCH 10/10] core/vdbe: Make apply_view_deltas() return early if views are disabled Currently, we have a borrow problem because parse_schema_rows() already borrows `schema`, but then `apply_view_deltas` does the same: ``` thread 'main' panicked at core/vdbe/mod.rs:450:49: already mutably borrowed: BorrowError stack backtrace: 0: __rustc::rust_begin_unwind at /rustc/6b00bc3880198600130e1cf62b8f8a93494488cc/library/std/src/panicking.rs:697:5 1: core::panicking::panic_fmt at /rustc/6b00bc3880198600130e1cf62b8f8a93494488cc/library/core/src/panicking.rs:75:14 2: core::cell::panic_already_mutably_borrowed at /rustc/6b00bc3880198600130e1cf62b8f8a93494488cc/library/core/src/cell.rs:799:5 3: core::cell::RefCell::borrow at /rustc/6b00bc3880198600130e1cf62b8f8a93494488cc/library/core/src/cell.rs:987:25 4: turso_core::vdbe::Program::apply_view_deltas at ./core/vdbe/mod.rs:450:26 5: turso_core::vdbe::Program::commit_txn at ./core/vdbe/mod.rs:468:9 6: turso_core::vdbe::execute::op_halt at ./core/vdbe/execute.rs:1954:15 7: turso_core::vdbe::Program::step at ./core/vdbe/mod.rs:430:19 8: turso_core::Statement::step at ./core/lib.rs:1914:23 9: turso_core::util::parse_schema_rows at ./core/util.rs:91:15 10: turso_core::Connection::parse_schema_rows::{{closure}} at ./core/lib.rs:1518:17 11: turso_core::Connection::with_schema_mut at ./core/lib.rs:1625:9 12: turso_core::Connection::parse_schema_rows at ./core/lib.rs:1515:9 ``` However, this is a read transaction and views are not even enabled, let's just make `apply_view_deltas()` return early if there's no processing needed, to skip the schema borrow altogether. --- core/vdbe/mod.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/vdbe/mod.rs b/core/vdbe/mod.rs index a31105c1d..259767031 100644 --- a/core/vdbe/mod.rs +++ b/core/vdbe/mod.rs @@ -444,6 +444,10 @@ impl Program { #[instrument(skip_all, level = Level::DEBUG)] fn apply_view_deltas(&self, rollback: bool) { + if self.connection.view_transaction_states.borrow().is_empty() { + return; + } + let tx_states = self.connection.view_transaction_states.take(); if !rollback {