use std::{cmp::Ordering, sync::Arc}; use turso_parser::ast::{self, FrameBound, FrameClause, FrameExclude, FrameMode, SortOrder}; use crate::{ function::AggFunc, schema::{BTreeTable, Column, FromClauseSubquery, Index, Schema, Table}, translate::{ collate::get_collseq_from_expr, emitter::UpdateRowSource, optimizer::constraints::SeekRangeConstraint, }, vdbe::{ builder::{CursorKey, CursorType, ProgramBuilder}, insn::{IdxInsertFlags, Insn}, BranchOffset, CursorID, }, Result, VirtualTable, }; use crate::{schema::Type, types::SeekOp}; use turso_parser::ast::TableInternalId; use super::{emitter::OperationMode, planner::determine_where_to_eval_term}; #[derive(Debug, Clone)] pub struct ResultSetColumn { pub expr: ast::Expr, pub alias: Option, // TODO: encode which aggregates (e.g. index bitmask of plan.aggregates) are present in this column pub contains_aggregates: bool, } impl ResultSetColumn { pub fn name<'a>(&'a self, tables: &'a TableReferences) -> Option<&'a str> { if let Some(alias) = &self.alias { return Some(alias); } match &self.expr { ast::Expr::Column { table, column, .. } => { let table_ref = tables.find_table_by_internal_id(*table).unwrap(); table_ref.get_column_at(*column).unwrap().name.as_deref() } ast::Expr::RowId { table, .. } => { // If there is a rowid alias column, use its name let table_ref = tables.find_table_by_internal_id(*table).unwrap(); if let Table::BTree(table) = &table_ref { if let Some(rowid_alias_column) = table.get_rowid_alias_column() { if let Some(name) = &rowid_alias_column.1.name { return Some(name); } } } // If there is no rowid alias, use "rowid". Some("rowid") } _ => None, } } } #[derive(Debug, Clone)] pub struct GroupBy { pub exprs: Vec, /// sort order, if a sorter is required (= the columns aren't already in the correct order) pub sort_order: Option>, /// having clause split into a vec at 'AND' boundaries. pub having: Option>, } /// In a query plan, WHERE clause conditions and JOIN conditions are all folded into a vector of WhereTerm. /// This is done so that we can evaluate the conditions at the correct loop depth. /// We also need to keep track of whether the condition came from an OUTER JOIN. Take this example: /// SELECT * FROM users u LEFT JOIN products p ON u.id = 5. /// Even though the condition only refers to 'u', we CANNOT evaluate it at the users loop, because we need to emit NULL /// values for the columns of 'p', for EVERY row in 'u', instead of completely skipping any rows in 'u' where the condition is false. #[derive(Debug, Clone)] pub struct WhereTerm { /// The original condition expression. pub expr: ast::Expr, /// For normal JOIN conditions (ON or WHERE clauses), we break them up into individual [WhereTerm] conditions /// and let the optimizer determine when each should be evaluated based on the tables they reference. /// See e.g. [EvalAt]. /// For example, in "SELECT * FROM x JOIN y WHERE x.a = 2", we want to evaluate x.a = 2 right after opening x /// since it only depends on x. /// /// However, OUTER JOIN conditions require special handling. Consider: /// SELECT * FROM t LEFT JOIN s ON t.a = 2 /// /// Even though t.a = 2 only references t, we cannot evaluate it during t's loop and skip rows where t.a != 2. /// Instead, we must: /// 1. Process ALL rows from t /// 2. For each t row where t.a != 2, emit NULL values for s's columns /// 3. For each t row where t.a = 2, emit the actual s values /// /// This means the condition must be evaluated during s's loop, regardless of which tables it references. /// We track this requirement using [WhereTerm::from_outer_join], which contains the [TableInternalId] of the /// right-side table of the OUTER JOIN (in this case, s). When evaluating conditions, if [WhereTerm::from_outer_join] /// is set, we force evaluation to happen during that table's loop. pub from_outer_join: Option, /// Whether the condition has been consumed by the optimizer in some way, and it should not be evaluated /// in the normal place where WHERE terms are evaluated. /// A term may have been consumed e.g. if: /// - it has been converted into a constraint in a seek key /// - it has been removed due to being trivially true or false pub consumed: bool, } impl WhereTerm { pub fn should_eval_before_loop(&self, join_order: &[JoinOrderMember]) -> bool { if self.consumed { return false; } let Ok(eval_at) = self.eval_at(join_order) else { return false; }; eval_at == EvalAt::BeforeLoop } pub fn should_eval_at_loop(&self, loop_idx: usize, join_order: &[JoinOrderMember]) -> bool { if self.consumed { return false; } let Ok(eval_at) = self.eval_at(join_order) else { return false; }; eval_at == EvalAt::Loop(loop_idx) } fn eval_at(&self, join_order: &[JoinOrderMember]) -> Result { determine_where_to_eval_term(self, join_order) } } impl From for WhereTerm { fn from(value: Expr) -> Self { Self { expr: value, from_outer_join: None, consumed: false, } } } use crate::ast::Expr; use crate::util::exprs_are_equivalent; /// The loop index where to evaluate the condition. /// For example, in `SELECT * FROM u JOIN p WHERE u.id = 5`, the condition can already be evaluated at the first loop (idx 0), /// because that is the rightmost table that it references. /// /// Conditions like 1=2 can be evaluated before the main loop is opened, because they are constant. /// In theory we should be able to statically analyze them all and reduce them to a single boolean value, /// but that is not implemented yet. #[derive(Debug, Clone, PartialEq, Eq, Copy)] pub enum EvalAt { Loop(usize), BeforeLoop, } #[allow(clippy::non_canonical_partial_ord_impl)] impl PartialOrd for EvalAt { fn partial_cmp(&self, other: &Self) -> Option { match (self, other) { (EvalAt::Loop(a), EvalAt::Loop(b)) => a.partial_cmp(b), (EvalAt::BeforeLoop, EvalAt::BeforeLoop) => Some(Ordering::Equal), (EvalAt::BeforeLoop, _) => Some(Ordering::Less), (_, EvalAt::BeforeLoop) => Some(Ordering::Greater), } } } impl Ord for EvalAt { fn cmp(&self, other: &Self) -> Ordering { self.partial_cmp(other) .expect("total ordering not implemented for EvalAt") } } /// A query plan is either a SELECT or a DELETE (for now) #[derive(Debug, Clone)] pub enum Plan { Select(SelectPlan), CompoundSelect { left: Vec<(SelectPlan, ast::CompoundOperator)>, right_most: SelectPlan, limit: Option>, offset: Option>, order_by: Option>, }, Delete(DeletePlan), Update(UpdatePlan), } /// The destination of the results of a query. /// Typically, the results of a query are returned to the caller. /// However, there are some cases where the results are not returned to the caller, /// but rather are yielded to a parent query via coroutine, or stored in a temp table, /// later used by the parent query. #[derive(Debug, Clone)] pub enum QueryDestination { /// The results of the query are returned to the caller. ResultRows, /// The results of the query are yielded to a parent query via coroutine. CoroutineYield { /// The register that holds the program offset that handles jumping to/from the coroutine. yield_reg: usize, /// The index of the first instruction in the bytecode that implements the coroutine. coroutine_implementation_start: BranchOffset, }, /// The results of the query are stored in an ephemeral index, /// later used by the parent query. EphemeralIndex { /// The cursor ID of the ephemeral index that will be used to store the results. cursor_id: CursorID, /// The index that will be used to store the results. index: Arc, /// Whether this is a delete operation that will remove the index entries is_delete: bool, }, /// The results of the query are stored in an ephemeral table, /// later used by the parent query. EphemeralTable { /// The cursor ID of the ephemeral table that will be used to store the results. cursor_id: CursorID, /// The table that will be used to store the results. table: Arc, }, } impl QueryDestination { pub fn placeholder_for_subquery() -> Self { QueryDestination::CoroutineYield { yield_reg: usize::MAX, // will be set later in bytecode emission coroutine_implementation_start: BranchOffset::Placeholder, // will be set later in bytecode emission } } } #[derive(Debug, Default, Clone, Copy, PartialEq, Eq)] pub struct JoinOrderMember { /// The internal ID of the[TableReference] pub table_id: TableInternalId, /// The index of the table in the original join order. /// This is used to index into e.g. [TableReferences::joined_tables()] pub original_idx: usize, /// Whether this member is the right side of an OUTER JOIN pub is_outer: bool, } #[derive(Debug, Clone, PartialEq)] /// Whether a column is DISTINCT or not. pub enum Distinctness { /// The column is not a DISTINCT column. NonDistinct, /// The column is a DISTINCT column, /// and includes a translation context for handling duplicates. Distinct { ctx: Option }, } impl Distinctness { pub fn from_ast(distinctness: Option<&ast::Distinctness>) -> Self { match distinctness { Some(ast::Distinctness::Distinct) => Self::Distinct { ctx: None }, Some(ast::Distinctness::All) => Self::NonDistinct, None => Self::NonDistinct, } } pub fn is_distinct(&self) -> bool { matches!(self, Distinctness::Distinct { .. }) } } /// Translation context for handling DISTINCT columns. #[derive(Debug, Clone, PartialEq)] pub struct DistinctCtx { /// The cursor ID for the ephemeral index opened for the purpose of deduplicating results. pub cursor_id: usize, /// The index name for the ephemeral index, needed to lookup the cursor ID. pub ephemeral_index_name: String, /// The label for the on conflict branch. /// When a duplicate is found, the program will jump to the offset this label points to. pub label_on_conflict: BranchOffset, } impl DistinctCtx { pub fn emit_deduplication_insns( &self, program: &mut ProgramBuilder, num_regs: usize, start_reg: usize, ) { program.emit_insn(Insn::Found { cursor_id: self.cursor_id, target_pc: self.label_on_conflict, record_reg: start_reg, num_regs, }); let record_reg = program.alloc_register(); program.emit_insn(Insn::MakeRecord { start_reg, count: num_regs, dest_reg: record_reg, index_name: Some(self.ephemeral_index_name.to_string()), affinity_str: None, }); program.emit_insn(Insn::IdxInsert { cursor_id: self.cursor_id, record_reg, unpacked_start: None, unpacked_count: None, flags: IdxInsertFlags::new(), }); } } #[derive(Debug, Clone)] pub struct SelectPlan { pub table_references: TableReferences, /// The order in which the tables are joined. Tables have usize Ids (their index in joined_tables) pub join_order: Vec, /// the columns inside SELECT ... FROM pub result_columns: Vec, /// where clause split into a vec at 'AND' boundaries. all join conditions also get shoved in here, /// and we keep track of which join they came from (mainly for OUTER JOIN processing) pub where_clause: Vec, /// group by clause pub group_by: Option, /// order by clause pub order_by: Vec<(Box, SortOrder)>, /// all the aggregates collected from the result columns, order by, and (TODO) having clauses pub aggregates: Vec, /// limit clause pub limit: Option>, /// offset clause pub offset: Option>, /// query contains a constant condition that is always false pub contains_constant_false_condition: bool, /// the destination of the resulting rows from this plan. pub query_destination: QueryDestination, /// whether the query is DISTINCT pub distinctness: Distinctness, /// values: https://sqlite.org/syntax/select-core.html pub values: Vec>, /// The window definition and all window functions associated with it. There is at most one /// window per SELECT. If the original query contains more, they are pushed down into subqueries. pub window: Option, } impl SelectPlan { pub fn joined_tables(&self) -> &[JoinedTable] { self.table_references.joined_tables() } pub fn agg_args_count(&self) -> usize { self.aggregates.iter().map(|agg| agg.args.len()).sum() } /// Reference: https://github.com/sqlite/sqlite/blob/5db695197b74580c777b37ab1b787531f15f7f9f/src/select.c#L8613 /// /// Checks to see if the query is of the format `SELECT count(*) FROM ` pub fn is_simple_count(&self) -> bool { if !self.where_clause.is_empty() || self.aggregates.len() != 1 || matches!( self.query_destination, QueryDestination::CoroutineYield { .. } ) || self.table_references.joined_tables().len() != 1 || !self.table_references.outer_query_refs().is_empty() || self.result_columns.len() != 1 || self.group_by.is_some() || self.contains_constant_false_condition // TODO: (pedrocarlo) maybe can optimize to use the count optmization with more columns { return false; } let table_ref = self.table_references.joined_tables().first().unwrap(); if !matches!(table_ref.table, crate::schema::Table::BTree(..)) { return false; } let agg = self.aggregates.first().unwrap(); if !matches!(agg.func, AggFunc::Count0) { return false; } let count = ast::Expr::FunctionCall { name: ast::Name::exact("count".to_string()), distinctness: None, args: vec![], order_by: vec![], filter_over: ast::FunctionTail { filter_clause: None, over_clause: None, }, }; let count_star = ast::Expr::FunctionCallStar { name: ast::Name::exact("count".to_string()), filter_over: ast::FunctionTail { filter_clause: None, over_clause: None, }, }; let result_col_expr = &self.result_columns.first().unwrap().expr; if *result_col_expr != count && *result_col_expr != count_star { return false; } true } } #[allow(dead_code)] #[derive(Debug, Clone)] pub struct DeletePlan { pub table_references: TableReferences, /// the columns inside SELECT ... FROM pub result_columns: Vec, /// where clause split into a vec at 'AND' boundaries. pub where_clause: Vec, /// order by clause pub order_by: Vec<(Box, SortOrder)>, /// limit clause pub limit: Option>, /// offset clause pub offset: Option>, /// query contains a constant condition that is always false pub contains_constant_false_condition: bool, /// Indexes that must be updated by the delete operation. pub indexes: Vec>, } #[derive(Debug, Clone)] pub struct UpdatePlan { pub table_references: TableReferences, // (colum index, new value) pairs pub set_clauses: Vec<(usize, Box)>, pub where_clause: Vec, pub order_by: Vec<(Box, SortOrder)>, pub limit: Option>, pub offset: Option>, // TODO: optional RETURNING clause pub returning: Option>, // whether the WHERE clause is always false pub contains_constant_false_condition: bool, pub indexes_to_update: Vec>, // If the UPDATE modifies any column that is present in the key of the btree used to iterate over the table (either the table itself or an index), // gather all the target rowids into an ephemeral table, and then use that table as the single JoinedTable for the actual UPDATE loop. // This ensures the keys of the btree used to iterate cannot be changed during the UPDATE loop itself, ensuring all the intended rows actually get // updated and none are skipped. pub ephemeral_plan: Option, // For ALTER TABLE turso-db emits appropriate DDL statement in the "updates" cell of CDC table // This field is present only for update plan created for ALTER TABLE when CDC mode has "updates" values pub cdc_update_alter_statement: Option, } #[derive(Clone, Copy, Debug, PartialEq, Eq)] pub enum IterationDirection { Forwards, Backwards, } pub fn select_star(tables: &[JoinedTable], out_columns: &mut Vec) { for table in tables.iter() { out_columns.extend( table .columns() .iter() .enumerate() .filter(|(_, col)| !col.hidden) .filter(|(_, col)| { // If we are joining with USING, we need to deduplicate the columns from the right table // that are also present in the USING clause. if let Some(join_info) = &table.join_info { !join_info.using.iter().any(|using_col| { col.name .as_ref() .is_some_and(|name| name.eq_ignore_ascii_case(using_col.as_str())) }) } else { true } }) .map(|(i, col)| ResultSetColumn { alias: None, expr: ast::Expr::Column { database: None, table: table.internal_id, column: i, is_rowid_alias: col.is_rowid_alias, }, contains_aggregates: false, }), ); } } /// Join information for a table reference. #[derive(Debug, Clone)] pub struct JoinInfo { /// Whether this is an OUTER JOIN. pub outer: bool, /// The USING clause for the join, if any. NATURAL JOIN is transformed into USING (col1, col2, ...). pub using: Vec, } /// A joined table in the query plan. /// For example, /// ```sql /// SELECT * FROM users u JOIN products p JOIN (SELECT * FROM users) sub; /// ``` /// has three table references where /// - all have [Operation::Scan] /// - identifiers are `t`, `p`, `sub` /// - `t` and `p` are [Table::BTree] while `sub` is [Table::FromClauseSubquery] /// - join_info is None for the first table reference, and Some(JoinInfo { outer: false, using: None }) for the second and third table references #[derive(Debug, Clone)] pub struct JoinedTable { /// The operation that this table reference performs. pub op: Operation, /// Table object, which contains metadata about the table, e.g. columns. pub table: Table, /// The name of the table as referred to in the query, either the literal name or an alias e.g. "users" or "u" pub identifier: String, /// Internal ID of the table reference, used in e.g. [Expr::Column] to refer to this table. pub internal_id: TableInternalId, /// The join info for this table reference, if it is the right side of a join (which all except the first table reference have) pub join_info: Option, /// Bitmask of columns that are referenced in the query. /// Used to decide whether a covering index can be used. pub col_used_mask: ColumnUsedMask, /// The index of the database. "main" is always zero. pub database_id: usize, } #[derive(Debug, Clone)] pub struct OuterQueryReference { /// The name of the table as referred to in the query, either the literal name or an alias e.g. "users" or "u" pub identifier: String, /// Internal ID of the table reference, used in e.g. [Expr::Column] to refer to this table. pub internal_id: TableInternalId, /// Table object, which contains metadata about the table, e.g. columns. pub table: Table, /// Bitmask of columns that are referenced in the query. /// Used to track dependencies, so that it can be resolved /// when a WHERE clause subquery should be evaluated; /// i.e., if the subquery depends on tables T and U, /// then both T and U need to be in scope for the subquery to be evaluated. pub col_used_mask: ColumnUsedMask, } impl OuterQueryReference { /// Returns the columns of the table that this outer query reference refers to. pub fn columns(&self) -> &[Column] { self.table.columns() } /// Marks a column as used; used means that the column is referenced in the query. pub fn mark_column_used(&mut self, column_index: usize) { self.col_used_mask.set(column_index); } /// Whether the OuterQueryReference is used by the current query scope. /// This is used primarily to determine at what loop depth a subquery should be evaluated. pub fn is_used(&self) -> bool { !self.col_used_mask.is_empty() } } #[derive(Debug, Clone)] /// A collection of table references in a given SQL statement. /// /// `TableReferences::joined_tables` is the list of tables that are joined together. /// Example: SELECT * FROM t JOIN u JOIN v -- the joined tables are t, u and v. /// /// `TableReferences::outer_query_refs` are references to tables outside the current scope. /// Example: SELECT * FROM t WHERE EXISTS (SELECT * FROM u WHERE u.foo = t.foo) /// -- here, 'u' is an outer query reference for the subquery (SELECT * FROM u WHERE u.foo = t.foo), /// since that query does not declare 't' in its FROM clause. /// /// /// Typically a query will only have joined tables, but the following may have outer query references: /// - CTEs that refer to other preceding CTEs /// - Correlated subqueries, i.e. subqueries that depend on the outer scope pub struct TableReferences { /// Tables that are joined together in this query scope. joined_tables: Vec, /// Tables from outer scopes that are referenced in this query scope. outer_query_refs: Vec, } impl TableReferences { /// The maximum number of tables that can be joined together in a query. /// This limit is arbitrary, although we currently use a u128 to represent the [crate::translate::planner::TableMask], /// which can represent up to 128 tables. /// Even at 63 tables we currently cannot handle the optimization performantly, hence the arbitrary cap. pub const MAX_JOINED_TABLES: usize = 63; pub fn new( joined_tables: Vec, outer_query_refs: Vec, ) -> Self { Self { joined_tables, outer_query_refs, } } pub fn new_empty() -> Self { Self { joined_tables: Vec::new(), outer_query_refs: Vec::new(), } } pub fn is_empty(&self) -> bool { self.joined_tables.is_empty() && self.outer_query_refs.is_empty() } /// Add a new [JoinedTable] to the query plan. pub fn add_joined_table(&mut self, joined_table: JoinedTable) { self.joined_tables.push(joined_table); } /// Add a new [OuterQueryReference] to the query plan. pub fn add_outer_query_reference(&mut self, outer_query_reference: OuterQueryReference) { self.outer_query_refs.push(outer_query_reference); } /// Returns an immutable reference to the [JoinedTable]s in the query plan. pub fn joined_tables(&self) -> &[JoinedTable] { &self.joined_tables } /// Returns a mutable reference to the [JoinedTable]s in the query plan. pub fn joined_tables_mut(&mut self) -> &mut Vec { &mut self.joined_tables } /// Returns an immutable reference to the [OuterQueryReference]s in the query plan. pub fn outer_query_refs(&self) -> &[OuterQueryReference] { &self.outer_query_refs } /// Returns an immutable reference to the [OuterQueryReference] with the given internal ID. pub fn find_outer_query_ref_by_internal_id( &self, internal_id: TableInternalId, ) -> Option<&OuterQueryReference> { self.outer_query_refs .iter() .find(|t| t.internal_id == internal_id) } /// Returns a mutable reference to the [OuterQueryReference] with the given internal ID. pub fn find_outer_query_ref_by_internal_id_mut( &mut self, internal_id: TableInternalId, ) -> Option<&mut OuterQueryReference> { self.outer_query_refs .iter_mut() .find(|t| t.internal_id == internal_id) } /// Returns an immutable reference to the [Table] with the given internal ID. pub fn find_table_by_internal_id(&self, internal_id: TableInternalId) -> Option<&Table> { self.joined_tables .iter() .find(|t| t.internal_id == internal_id) .map(|t| &t.table) .or_else(|| { self.outer_query_refs .iter() .find(|t| t.internal_id == internal_id) .map(|t| &t.table) }) } /// Returns an immutable reference to the [Table] with the given identifier, /// where identifier is either the literal name of the table or an alias. pub fn find_table_by_identifier(&self, identifier: &str) -> Option<&Table> { self.joined_tables .iter() .find(|t| t.identifier == identifier) .map(|t| &t.table) .or_else(|| { self.outer_query_refs .iter() .find(|t| t.identifier == identifier) .map(|t| &t.table) }) } /// Returns an immutable reference to the [OuterQueryReference] with the given identifier, /// where identifier is either the literal name of the table or an alias. pub fn find_outer_query_ref_by_identifier( &self, identifier: &str, ) -> Option<&OuterQueryReference> { self.outer_query_refs .iter() .find(|t| t.identifier == identifier) } /// Returns the internal ID and immutable reference to the [Table] with the given identifier, pub fn find_table_and_internal_id_by_identifier( &self, identifier: &str, ) -> Option<(TableInternalId, &Table)> { self.joined_tables .iter() .find(|t| t.identifier == identifier) .map(|t| (t.internal_id, &t.table)) .or_else(|| { self.outer_query_refs .iter() .find(|t| t.identifier == identifier) .map(|t| (t.internal_id, &t.table)) }) } /// Returns an immutable reference to the [JoinedTable] with the given internal ID. pub fn find_joined_table_by_internal_id( &self, internal_id: TableInternalId, ) -> Option<&JoinedTable> { self.joined_tables .iter() .find(|t| t.internal_id == internal_id) } /// Returns a mutable reference to the [JoinedTable] with the given internal ID. pub fn find_joined_table_by_internal_id_mut( &mut self, internal_id: TableInternalId, ) -> Option<&mut JoinedTable> { self.joined_tables .iter_mut() .find(|t| t.internal_id == internal_id) } /// Marks a column as used; used means that the column is referenced in the query. pub fn mark_column_used(&mut self, internal_id: TableInternalId, column_index: usize) { if let Some(joined_table) = self.find_joined_table_by_internal_id_mut(internal_id) { joined_table.mark_column_used(column_index); } else if let Some(outer_query_ref) = self.find_outer_query_ref_by_internal_id_mut(internal_id) { outer_query_ref.mark_column_used(column_index); } else { panic!("table with internal id {internal_id} not found in table references"); } } pub fn contains_table(&self, table: &Table) -> bool { self.joined_tables .iter() .map(|t| &t.table) .chain(self.outer_query_refs.iter().map(|t| &t.table)) .any(|t| match t { Table::FromClauseSubquery(subquery_table) => { subquery_table.plan.table_references.contains_table(table) } _ => t == table, }) } pub fn extend(&mut self, other: TableReferences) { self.joined_tables.extend(other.joined_tables); self.outer_query_refs.extend(other.outer_query_refs); } } #[derive(Clone, Debug, Default, PartialEq)] #[repr(transparent)] pub struct ColumnUsedMask(roaring::RoaringBitmap); impl ColumnUsedMask { pub fn set(&mut self, index: usize) { self.0.insert(index as u32); } pub fn get(&self, index: usize) -> bool { self.0.contains(index as u32) } pub fn contains_all_set_bits_of(&self, other: &Self) -> bool { other.0.is_subset(&self.0) } pub fn is_empty(&self) -> bool { self.0.is_empty() } } #[derive(Clone, Debug)] #[allow(clippy::large_enum_variant)] pub enum Operation { // Scan operation // This operation is used to scan a table. Scan(Scan), // Search operation // This operation is used to search for a row in a table using an index // (i.e. a primary key or a secondary index) Search(Search), } impl Operation { pub fn default_scan_for(table: &Table) -> Self { match table { Table::BTree(_) => Operation::Scan(Scan::BTreeTable { iter_dir: IterationDirection::Forwards, index: None, }), Table::Virtual(_) => Operation::Scan(Scan::VirtualTable { idx_num: -1, idx_str: None, constraints: Vec::new(), }), Table::FromClauseSubquery(_) => Operation::Scan(Scan::Subquery), } } pub fn index(&self) -> Option<&Arc> { match self { Operation::Scan(Scan::BTreeTable { index, .. }) => index.as_ref(), Operation::Scan(_) => None, Operation::Search(Search::RowidEq { .. }) => None, Operation::Search(Search::Seek { index, .. }) => index.as_ref(), } } } impl JoinedTable { /// Returns the btree table for this table reference, if it is a BTreeTable. pub fn btree(&self) -> Option> { match &self.table { Table::BTree(_) => self.table.btree(), _ => None, } } pub fn virtual_table(&self) -> Option> { match &self.table { Table::Virtual(_) => self.table.virtual_table(), _ => None, } } /// Creates a new TableReference for a subquery. pub fn new_subquery( identifier: String, plan: SelectPlan, join_info: Option, internal_id: TableInternalId, ) -> Result { let mut columns = plan .result_columns .iter() .map(|rc| Column { name: rc.name(&plan.table_references).map(String::from), ty: Type::Blob, // FIXME: infer proper type ty_str: "BLOB".to_string(), is_rowid_alias: false, primary_key: false, notnull: false, default: None, unique: false, collation: None, hidden: false, }) .collect::>(); for (i, column) in columns.iter_mut().enumerate() { column.collation = get_collseq_from_expr(&plan.result_columns[i].expr, &plan.table_references)?; } let table = Table::FromClauseSubquery(FromClauseSubquery { name: identifier.clone(), plan: Box::new(plan), columns, result_columns_start_reg: None, }); Ok(Self { op: Operation::default_scan_for(&table), table, identifier, internal_id, join_info, col_used_mask: ColumnUsedMask::default(), database_id: 0, }) } pub fn columns(&self) -> &[Column] { self.table.columns() } /// Mark a column as used in the query. /// This is used to determine whether a covering index can be used. pub fn mark_column_used(&mut self, index: usize) { self.col_used_mask.set(index); } /// Open the necessary cursors for this table reference. /// Generally a table cursor is always opened unless a SELECT query can use a covering index. /// An index cursor is opened if an index is used in any way for reading data from the table. pub fn open_cursors( &self, program: &mut ProgramBuilder, mode: OperationMode, schema: &Schema, ) -> Result<(Option, Option)> { let index = self.op.index(); match &self.table { Table::BTree(btree) => { let use_covering_index = self.utilizes_covering_index(); let index_is_ephemeral = index.is_some_and(|index| index.ephemeral); let table_not_required = matches!(mode, OperationMode::SELECT) && use_covering_index && !index_is_ephemeral; let table_cursor_id = if table_not_required { None } else if let OperationMode::UPDATE(UpdateRowSource::PrebuiltEphemeralTable { target_table, .. }) = &mode { // The cursor for the ephemeral table was already allocated earlier. Let's allocate one for the target table, // in case it wasn't already allocated when populating the ephemeral table. Some(program.alloc_cursor_id_keyed_if_not_exists( CursorKey::table(target_table.internal_id), match &target_table.table { Table::BTree(btree) => CursorType::BTreeTable(btree.clone()), Table::Virtual(virtual_table) => { CursorType::VirtualTable(virtual_table.clone()) } _ => unreachable!("target table must be a btree or virtual table"), }, )) } else { // Check if this is a materialized view let cursor_type = if let Some(view_mutex) = schema.get_materialized_view(&btree.name) { CursorType::MaterializedView(btree.clone(), view_mutex) } else { CursorType::BTreeTable(btree.clone()) }; Some( program .alloc_cursor_id_keyed(CursorKey::table(self.internal_id), cursor_type), ) }; let index_cursor_id = index.map(|index| { program.alloc_cursor_id_keyed( CursorKey::index(self.internal_id, index.clone()), CursorType::BTreeIndex(index.clone()), ) }); Ok((table_cursor_id, index_cursor_id)) } Table::Virtual(virtual_table) => { let table_cursor_id = Some(program.alloc_cursor_id_keyed( CursorKey::table(self.internal_id), CursorType::VirtualTable(virtual_table.clone()), )); let index_cursor_id = None; Ok((table_cursor_id, index_cursor_id)) } Table::FromClauseSubquery(..) => Ok((None, None)), } } /// Resolve the already opened cursors for this table reference. pub fn resolve_cursors( &self, program: &mut ProgramBuilder, mode: OperationMode, ) -> Result<(Option, Option)> { let index = self.op.index(); let table_cursor_id = if let OperationMode::UPDATE(UpdateRowSource::PrebuiltEphemeralTable { target_table, .. }) = &mode { program.resolve_cursor_id_safe(&CursorKey::table(target_table.internal_id)) } else { program.resolve_cursor_id_safe(&CursorKey::table(self.internal_id)) }; let index_cursor_id = index.map(|index| { program.resolve_cursor_id(&CursorKey::index(self.internal_id, index.clone())) }); Ok((table_cursor_id, index_cursor_id)) } /// Returns true if a given index is a covering index for this [TableReference]. pub fn index_is_covering(&self, index: &Index) -> bool { let Table::BTree(btree) = &self.table else { return false; }; if self.col_used_mask.is_empty() { return false; } let mut index_cols_mask = ColumnUsedMask::default(); for col in index.columns.iter() { index_cols_mask.set(col.pos_in_table); } // If a table has a rowid (i.e. is not a WITHOUT ROWID table), the index is guaranteed to contain the rowid as well. if btree.has_rowid { if let Some(pos_of_rowid_alias_col) = btree.get_rowid_alias_column().map(|(pos, _)| pos) { let mut empty_mask = ColumnUsedMask::default(); empty_mask.set(pos_of_rowid_alias_col); if self.col_used_mask == empty_mask { // However if the index would be ONLY used for the rowid, then let's not bother using it to cover the query. // Example: if the query is SELECT id FROM t, and id is a rowid alias, then let's rather just scan the table // instead of an index. return false; } index_cols_mask.set(pos_of_rowid_alias_col); } } index_cols_mask.contains_all_set_bits_of(&self.col_used_mask) } /// Returns true if the index selected for use with this [TableReference] is a covering index, /// meaning that it contains all the columns that are referenced in the query. pub fn utilizes_covering_index(&self) -> bool { let Some(index) = self.op.index() else { return false; }; self.index_is_covering(index.as_ref()) } pub fn column_is_used(&self, index: usize) -> bool { self.col_used_mask.get(index) } } /// A definition of a rowid/index search. /// /// [SeekKey] is the condition that is used to seek to a specific row in a table/index. /// [SeekKey] also used to represent range scan termination condition. #[derive(Debug, Clone)] pub struct SeekDef { /// Common prefix of the key which is shared between start/end fields /// For example, given: /// - CREATE INDEX i ON t (x, y desc) /// - SELECT * FROM t WHERE x = 1 AND y >= 30 /// /// Then, prefix=[(eq=1, ASC)], start=Some((ge, Expr(30))), end=Some((gt, Sentinel)) pub prefix: Vec, /// The condition to use when seeking. See [SeekKey] for more details. pub start: SeekKey, /// The condition to use when terminating the scan that follows the seek. See [SeekKey] for more details. pub end: SeekKey, /// The direction of the scan that follows the seek. pub iter_dir: IterationDirection, } pub struct SeekDefKeyIterator<'a> { seek_def: &'a SeekDef, seek_key: &'a SeekKey, pos: usize, } impl<'a> Iterator for SeekDefKeyIterator<'a> { type Item = SeekKeyComponent<&'a ast::Expr>; fn next(&mut self) -> Option { let result = if self.pos < self.seek_def.prefix.len() { Some(SeekKeyComponent::Expr( &self.seek_def.prefix[self.pos].eq.as_ref().unwrap().1, )) } else if self.pos == self.seek_def.prefix.len() { match &self.seek_key.last_component { SeekKeyComponent::Expr(expr) => Some(SeekKeyComponent::Expr(expr)), SeekKeyComponent::None => None, } } else { None }; self.pos += 1; result } } impl SeekDef { /// returns amount of values in the given seek key /// - so, for SELECT * FROM t WHERE x = 10 AND y = 20 AND y >= 30 there will be 3 values (10, 20, 30) pub fn size(&self, key: &SeekKey) -> usize { self.prefix.len() + match key.last_component { SeekKeyComponent::Expr(_) => 1, SeekKeyComponent::None => 0, } } /// iterate over value expressions in the given seek key pub fn iter<'a>(&'a self, key: &'a SeekKey) -> SeekDefKeyIterator<'a> { SeekDefKeyIterator { seek_def: self, seek_key: key, pos: 0, } } } /// [SeekKeyComponent] enum represents optional last_component of the [SeekKey] /// /// This component represented by separate enum instead of Option because before there were third Sentinel value /// For now - we don't need this and it's enough to just either use some user-provided expression or omit last component of the key completely /// But as separate enum is almost never a harm - I decided to keep it here. /// /// This enum accepts generic argument E in order to use both SeekKeyComponent and SeekKeyComponent<&ast::Expr> #[derive(Debug, Clone)] pub enum SeekKeyComponent { Expr(E), None, } /// A condition to use when seeking. #[derive(Debug, Clone)] pub struct SeekKey { /// Complete key must be constructed from common [SeekDef::prefix] and optional last_component pub last_component: SeekKeyComponent, /// The comparison operator to use when seeking. pub op: SeekOp, } /// Represents the type of table scan performed during query execution. #[derive(Clone, Debug)] pub enum Scan { /// A scan of a B-tree–backed table, optionally using an index, and with an iteration direction. BTreeTable { /// The iter_dir is used to indicate the direction of the iterator. iter_dir: IterationDirection, /// The index that we are using to scan the table, if any. index: Option>, }, /// A scan of a virtual table, delegated to the table’s `filter` and related methods. VirtualTable { /// Index identifier returned by the table's `best_index` method. idx_num: i32, /// Optional index name returned by the table’s `best_index` method. idx_str: Option, /// Constraining expressions to be passed to the table’s `filter` method. /// The order of expressions matches the argument order expected by the virtual table. constraints: Vec, }, /// A scan of a subquery in the `FROM` clause. Subquery, } /// An enum that represents a search operation that can be used to search for a row in a table using an index /// (i.e. a primary key or a secondary index) #[allow(clippy::large_enum_variant)] #[derive(Clone, Debug)] pub enum Search { /// A rowid equality point lookup. This is a special case that uses the SeekRowid bytecode instruction and does not loop. RowidEq { cmp_expr: ast::Expr }, /// A search on a table btree (via `rowid`) or a secondary index search. Uses bytecode instructions like SeekGE, SeekGT etc. Seek { index: Option>, seek_def: SeekDef, }, } #[derive(Debug, Clone, PartialEq)] pub struct Aggregate { pub func: AggFunc, pub args: Vec, pub original_expr: ast::Expr, pub distinctness: Distinctness, } impl Aggregate { pub fn new(func: AggFunc, args: &[Box], expr: &Expr, distinctness: Distinctness) -> Self { Aggregate { func, args: args.iter().map(|arg| *arg.clone()).collect(), original_expr: expr.clone(), distinctness, } } pub fn is_distinct(&self) -> bool { self.distinctness.is_distinct() } } /// Represents the window definition and all window functions associated with a single SELECT. #[derive(Debug, Clone)] pub struct Window { /// The window name, either provided in the original statement or synthetically generated by /// the planner. This is optional because it can be assigned at different stages of query /// processing, but it should eventually always be set. pub name: Option, /// Expressions from the PARTITION BY clause. pub partition_by: Vec, /// The number of unique expressions in the PARTITION BY clause. This determines how many of /// the leftmost columns in the subquery output make up the partition key. pub deduplicated_partition_by_len: Option, /// Expressions from the ORDER BY clause. pub order_by: Vec<(Expr, SortOrder)>, /// All window functions associated with this window. pub functions: Vec, } impl Window { const DEFAULT_SORT_ORDER: SortOrder = SortOrder::Asc; pub fn new(name: Option, ast: &ast::Window) -> Result { if !Self::is_default_frame_spec(&ast.frame_clause) { crate::bail_parse_error!("Custom frame specifications are not supported yet"); } Ok(Window { name, partition_by: ast.partition_by.iter().map(|arg| *arg.clone()).collect(), deduplicated_partition_by_len: None, order_by: ast .order_by .iter() .map(|col| { ( *col.expr.clone(), col.order.unwrap_or(Self::DEFAULT_SORT_ORDER), ) }) .collect(), functions: vec![], }) } pub fn is_equivalent(&self, ast: &ast::Window) -> bool { if !Self::is_default_frame_spec(&ast.frame_clause) { return false; } if self.partition_by.len() != ast.partition_by.len() { return false; } if !self .partition_by .iter() .zip(&ast.partition_by) .all(|(a, b)| exprs_are_equivalent(a, b)) { return false; } if self.order_by.len() != ast.order_by.len() { return false; } self.order_by .iter() .zip(&ast.order_by) .all(|((expr_a, order_a), col_b)| { exprs_are_equivalent(expr_a, &col_b.expr) && *order_a == col_b.order.unwrap_or(Self::DEFAULT_SORT_ORDER) }) } fn is_default_frame_spec(frame: &Option) -> bool { if let Some(frame_clause) = frame { let FrameClause { mode, start, end, exclude, } = frame_clause; if *mode != FrameMode::Range { return false; } if *start != FrameBound::UnboundedPreceding { return false; } if *end != Some(FrameBound::CurrentRow) { return false; } if let Some(exclude) = exclude { if *exclude != FrameExclude::NoOthers { return false; } } } true } } #[derive(Debug, Clone)] pub struct WindowFunction { /// The resolved function. Currently, only regular aggregate functions are supported. /// In the future, more specialized window functions such as `RANK()`, `ROW_NUMBER()`, etc. will be added. pub func: AggFunc, /// The expression from which the function was resolved. pub original_expr: Expr, } #[cfg(test)] mod tests { use super::*; use rand_chacha::{ rand_core::{RngCore, SeedableRng}, ChaCha8Rng, }; #[test] fn test_column_used_mask_empty() { let mask = ColumnUsedMask::default(); assert!(mask.is_empty()); let mut mask2 = ColumnUsedMask::default(); mask2.set(0); assert!(!mask2.is_empty()); } #[test] fn test_column_used_mask_set_and_get() { let mut mask = ColumnUsedMask::default(); let max_columns = 10000; let mut set_indices = Vec::new(); let mut rng = ChaCha8Rng::seed_from_u64( std::time::SystemTime::now() .duration_since(std::time::UNIX_EPOCH) .unwrap() .as_secs(), ); for i in 0..max_columns { if rng.next_u32() % 3 == 0 { set_indices.push(i); mask.set(i); } } // Verify set bits are present for &i in &set_indices { assert!(mask.get(i), "Expected bit {i} to be set"); } // Verify unset bits are not present for i in 0..max_columns { if !set_indices.contains(&i) { assert!(!mask.get(i), "Expected bit {i} to not be set"); } } } #[test] fn test_column_used_mask_subset_relationship() { let mut full_mask = ColumnUsedMask::default(); let mut subset_mask = ColumnUsedMask::default(); let max_columns = 5000; let mut rng = ChaCha8Rng::seed_from_u64( std::time::SystemTime::now() .duration_since(std::time::UNIX_EPOCH) .unwrap() .as_secs(), ); // Create a pattern where subset has fewer bits for i in 0..max_columns { if rng.next_u32() % 5 == 0 { full_mask.set(i); if i % 2 == 0 { subset_mask.set(i); } } } // full_mask contains all bits of subset_mask assert!(full_mask.contains_all_set_bits_of(&subset_mask)); // subset_mask does not contain all bits of full_mask assert!(!subset_mask.contains_all_set_bits_of(&full_mask)); // A mask contains itself assert!(full_mask.contains_all_set_bits_of(&full_mask)); assert!(subset_mask.contains_all_set_bits_of(&subset_mask)); } #[test] fn test_column_used_mask_empty_subset() { let mut mask = ColumnUsedMask::default(); for i in (0..1000).step_by(7) { mask.set(i); } let empty_mask = ColumnUsedMask::default(); // Empty mask is subset of everything assert!(mask.contains_all_set_bits_of(&empty_mask)); assert!(empty_mask.contains_all_set_bits_of(&empty_mask)); } #[test] fn test_column_used_mask_sparse_indices() { let mut sparse_mask = ColumnUsedMask::default(); // Test with very sparse, large indices let sparse_indices = vec![0, 137, 1042, 5389, 10000, 50000, 100000, 500000, 1000000]; for &idx in &sparse_indices { sparse_mask.set(idx); } for &idx in &sparse_indices { assert!(sparse_mask.get(idx), "Expected bit {idx} to be set"); } // Check some indices that shouldn't be set let unset_indices = vec![1, 100, 1000, 5000, 25000, 75000, 250000, 750000]; for &idx in &unset_indices { assert!(!sparse_mask.get(idx), "Expected bit {idx} to not be set"); } assert!(!sparse_mask.is_empty()); } }