diff --git a/Cargo.toml b/Cargo.toml index 7438cc3a..1173be7c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -2,7 +2,7 @@ [package] name = "fnck_sql" -version = "0.0.1-alpha.11" +version = "0.0.1-alpha.11.fix2" edition = "2021" authors = ["Kould ", "Xwg "] description = "Fast Insert OLTP SQL DBMS" diff --git a/src/binder/select.rs b/src/binder/select.rs index 0fa37f48..c0d6a1fc 100644 --- a/src/binder/select.rs +++ b/src/binder/select.rs @@ -26,7 +26,7 @@ use crate::planner::operator::sort::{SortField, SortOperator}; use crate::planner::operator::union::UnionOperator; use crate::planner::LogicalPlan; use crate::storage::Transaction; -use crate::types::tuple::Schema; +use crate::types::tuple::{Schema, SchemaRef}; use crate::types::LogicalType; use itertools::Itertools; use sqlparser::ast::{ @@ -232,36 +232,24 @@ impl<'a, T: Transaction> Binder<'a, T> { } let TableWithJoins { relation, joins } = &from[0]; + let mut plan = self.bind_single_table_ref(relation, None)?; - let (left_name, mut plan) = self.bind_single_table_ref(relation, None)?; - - if !joins.is_empty() { - let left_name = Self::unpack_name(left_name, true); - - for join in joins { - plan = self.bind_join(left_name.clone(), plan, join)?; - } + for join in joins { + plan = self.bind_join(plan, join)?; } Ok(plan) } - fn unpack_name(table_name: Option, is_left: bool) -> TableName { - let title = if is_left { "Left" } else { "Right" }; - table_name.unwrap_or_else(|| panic!("{}: Table is not named", title)) - } - fn bind_single_table_ref( &mut self, table: &TableFactor, joint_type: Option, - ) -> Result<(Option, LogicalPlan), DatabaseError> { - let plan_with_name = match table { + ) -> Result { + let plan = match table { TableFactor::Table { name, alias, .. } => { let table_name = lower_case_name(name)?; - let (table, plan) = - self._bind_single_table_ref(joint_type, &table_name, alias.as_ref())?; - (Some(table), plan) + self._bind_single_table_ref(joint_type, &table_name, alias.as_ref())? } TableFactor::Derived { subquery, alias, .. @@ -284,16 +272,13 @@ impl<'a, T: Transaction> Binder<'a, T> { table_alias.to_string(), tables.pop().unwrap(), )?; - - (Some(table_alias), plan) - } else { - ((tables.len() > 1).then(|| tables.pop()).flatten(), plan) } + plan } _ => unimplemented!(), }; - Ok(plan_with_name) + Ok(plan) } fn register_alias( @@ -332,20 +317,17 @@ impl<'a, T: Transaction> Binder<'a, T> { join_type: Option, table: &str, alias: Option<&TableAlias>, - ) -> Result<(Arc, LogicalPlan), DatabaseError> { + ) -> Result { let table_name = Arc::new(table.to_string()); let table_catalog = self.context.table_and_bind(table_name.clone(), join_type)?; let scan_op = ScanOperator::build(table_name.clone(), table_catalog); if let Some(TableAlias { name, columns }) = alias { - let alias = lower_ident(name); - self.register_alias(columns, alias.clone(), table_name.clone())?; - - return Ok((Arc::new(alias), scan_op)); + self.register_alias(columns, lower_ident(name), table_name.clone())?; } - Ok((table_name, scan_op)) + Ok(scan_op) } /// Normalize select item. @@ -428,8 +410,7 @@ impl<'a, T: Transaction> Binder<'a, T> { fn bind_join( &mut self, - left_table: TableName, - left: LogicalPlan, + mut left: LogicalPlan, join: &Join, ) -> Result { let Join { @@ -445,11 +426,12 @@ impl<'a, T: Transaction> Binder<'a, T> { JoinOperator::CrossJoin => (JoinType::Cross, None), _ => unimplemented!(), }; - let (right_table, right) = self.bind_single_table_ref(relation, Some(join_type))?; - let right_table = Self::unpack_name(right_table, false); + let mut right = self.bind_single_table_ref(relation, Some(join_type))?; let on = match joint_condition { - Some(constraint) => self.bind_join_constraint(left_table, right_table, constraint)?, + Some(constraint) => { + self.bind_join_constraint(left.output_schema(), right.output_schema(), constraint)? + } None => JoinCondition::None, }; @@ -639,8 +621,8 @@ impl<'a, T: Transaction> Binder<'a, T> { fn bind_join_constraint( &mut self, - left_table: TableName, - right_table: TableName, + left_schema: &SchemaRef, + right_schema: &SchemaRef, constraint: &JoinConstraint, ) -> Result { match constraint { @@ -651,21 +633,12 @@ impl<'a, T: Transaction> Binder<'a, T> { let mut filter = vec![]; let expr = self.bind_expr(expr)?; - let left_table = self - .context - .table(left_table) - .ok_or(DatabaseError::TableNotFound)?; - let right_table = self - .context - .table(right_table) - .ok_or(DatabaseError::TableNotFound)?; - Self::extract_join_keys( expr, &mut on_keys, &mut filter, - left_table.schema_ref(), - right_table.schema_ref(), + left_schema, + right_schema, )?; // combine multiple filter exprs into one BinaryExpr @@ -691,19 +664,11 @@ impl<'a, T: Transaction> Binder<'a, T> { .find(|column| column.name() == lower_ident(ident)) .map(|column| ScalarExpression::ColumnRef(column.clone())) }; - let left_table = self - .context - .table(left_table) - .ok_or(DatabaseError::TableNotFound)?; - let right_table = self - .context - .table(right_table) - .ok_or(DatabaseError::TableNotFound)?; for ident in idents { if let (Some(left_column), Some(right_column)) = ( - fn_column(left_table.schema_ref(), ident), - fn_column(right_table.schema_ref(), ident), + fn_column(left_schema, ident), + fn_column(right_schema, ident), ) { on_keys.push((left_column, right_column)); } else { @@ -823,6 +788,9 @@ impl<'a, T: Transaction> Binder<'a, T> { right_schema, )?; } + BinaryOperator::Or => { + todo!("`NestLoopJoin` is not supported yet") + } _ => { if left_expr.referenced_columns(true).iter().all(|column| { fn_or_contains(left_schema, right_schema, column.summary()) diff --git a/src/execution/volcano/dql/index_scan.rs b/src/execution/volcano/dql/index_scan.rs index aa36d034..1f2c0399 100644 --- a/src/execution/volcano/dql/index_scan.rs +++ b/src/execution/volcano/dql/index_scan.rs @@ -10,15 +10,15 @@ use futures_async_stream::try_stream; pub(crate) struct IndexScan { op: ScanOperator, index_by: IndexMetaRef, - binaries: Vec, + ranges: Vec, } impl From<(ScanOperator, IndexMetaRef, Vec)> for IndexScan { - fn from((op, index_by, binaries): (ScanOperator, IndexMetaRef, Vec)) -> Self { + fn from((op, index_by, ranges): (ScanOperator, IndexMetaRef, Vec)) -> Self { IndexScan { op, index_by, - binaries, + ranges, } } } @@ -39,7 +39,7 @@ impl IndexScan { .. } = self.op; let mut iter = - transaction.read_by_index(table_name, limit, columns, self.index_by, self.binaries)?; + transaction.read_by_index(table_name, limit, columns, self.index_by, self.ranges)?; while let Some(tuple) = iter.next_tuple()? { yield tuple; diff --git a/src/execution/volcano/dql/join/hash_join.rs b/src/execution/volcano/dql/join/hash_join.rs index 96e28272..9840d865 100644 --- a/src/execution/volcano/dql/join/hash_join.rs +++ b/src/execution/volcano/dql/join/hash_join.rs @@ -72,6 +72,11 @@ impl HashJoinStatus { JoinCondition::On { on, filter } => (on.into_iter().unzip(), filter), JoinCondition::None => unreachable!("HashJoin must has on condition"), }; + if on_left_keys.is_empty() || on_right_keys.is_empty() { + todo!("`NestLoopJoin` should be used when there is no equivalent condition") + } + assert!(!on_left_keys.is_empty()); + assert!(!on_right_keys.is_empty()); let fn_process = |schema: &mut Vec, force_nullable| { for column in schema.iter_mut() { diff --git a/src/execution/volcano/mod.rs b/src/execution/volcano/mod.rs index 3169b99b..7d736ca9 100644 --- a/src/execution/volcano/mod.rs +++ b/src/execution/volcano/mod.rs @@ -84,10 +84,10 @@ pub fn build_read(plan: LogicalPlan, transaction: &T) -> BoxedEx Operator::Scan(op) => { if let Some(PhysicalOption::IndexScan(IndexInfo { meta, - binaries: Some(binaries), + ranges: Some(ranges), })) = plan.physical_option { - IndexScan::from((op, meta, binaries)).execute(transaction) + IndexScan::from((op, meta, ranges)).execute(transaction) } else { SeqScan::from(op).execute(transaction) } diff --git a/src/expression/simplify.rs b/src/expression/simplify.rs index ca95e5c9..dc684af9 100644 --- a/src/expression/simplify.rs +++ b/src/expression/simplify.rs @@ -56,6 +56,17 @@ impl ConstantBinary { if binaries.is_empty() { return Ok(vec![]); } + if binaries.len() == 1 + && matches!( + binaries[0], + ConstantBinary::Scope { + min: Bound::Unbounded, + max: Bound::Unbounded + } + ) + { + return Ok(binaries); + } let mut condition_binaries = Vec::new(); @@ -140,27 +151,23 @@ impl ConstantBinary { // `Or` is allowed to contain And, `Scope`, `Eq/NotEq` // Tips: Only single-level `And` ConstantBinary::Or(binaries) => { - let mut or_binaries = Vec::new(); - for binary in binaries { + for binary in mem::take(binaries) { match binary { - ConstantBinary::And(and_binaries) => { - or_binaries.append(&mut Self::and_scope_aggregation(and_binaries)?); + ConstantBinary::And(mut and_binaries) => { + Self::and_scope_aggregation(&mut and_binaries)?; + binaries.append(&mut and_binaries); } ConstantBinary::Or(_) => { unreachable!("`Or` does not allow nested `Or`") } - cb => { - or_binaries.push(cb.clone()); - } + binary => binaries.push(binary), } } - let or_binaries = Self::or_scope_aggregation(&or_binaries); - let _ = mem::replace(self, ConstantBinary::Or(or_binaries)); + Self::or_scope_aggregation(binaries); } // `And` is allowed to contain Scope, `Eq/NotEq` ConstantBinary::And(binaries) => { - let and_binaries = Self::and_scope_aggregation(binaries)?; - let _ = mem::replace(self, ConstantBinary::And(and_binaries)); + Self::and_scope_aggregation(binaries)?; } _ => (), } @@ -197,11 +204,9 @@ impl ConstantBinary { } // Tips: It only makes sense if the condition is and aggregation - fn and_scope_aggregation( - binaries: &[ConstantBinary], - ) -> Result, DatabaseError> { + fn and_scope_aggregation(binaries: &mut Vec) -> Result<(), DatabaseError> { if binaries.is_empty() { - return Ok(vec![]); + return Ok(()); } let mut scope_min = Bound::Unbounded; @@ -219,9 +224,8 @@ impl ConstantBinary { for binary in binaries.iter().sorted_by_key(sort_op) { match binary { ConstantBinary::Scope { min, max } => { - // Skip if eq or noteq exists - if !eq_set.is_empty() { - continue; + if eq_set.len() == 1 { + break; } if let Some(order) = Self::bound_compared(&scope_min, min, true) { @@ -238,9 +242,20 @@ impl ConstantBinary { } ConstantBinary::Eq(val) => { let _ = eq_set.insert(val.clone()); + + // when there are multiple inconsistent eq conditions for the same field in And, + // then no row can meet the conditions. + // e.g. `select * from t1 where c1 = 0 and c1 = 1` no data can be both 0 and 1 at the same time + if eq_set.len() > 1 { + binaries.clear(); + return Ok(()); + } } ConstantBinary::NotEq(val) => { - let _ = eq_set.remove(val); + if eq_set.contains(val) { + binaries.clear(); + return Ok(()); + } } ConstantBinary::Or(_) | ConstantBinary::And(_) => { return Err(DatabaseError::InvalidType) @@ -248,40 +263,52 @@ impl ConstantBinary { } } - let eq_option = eq_set - .into_iter() - .sorted_by(|a, b| a.partial_cmp(b).unwrap_or(Ordering::Equal)) - .next() - .map(ConstantBinary::Eq); + binaries.clear(); + if eq_set.len() == 1 { + let eq = eq_set.into_iter().next().map(ConstantBinary::Eq).unwrap(); - if let Some(eq) = eq_option { - Ok(vec![eq]) + binaries.push(eq); } else if !matches!( (&scope_min, &scope_max), (Bound::Unbounded, Bound::Unbounded) ) { - let scope_binary = ConstantBinary::Scope { - min: scope_min, - max: scope_max, - }; - - Ok(vec![scope_binary]) - } else { - Ok(vec![]) + // When there is something like `select * from t1 where c1 between 1 and null`, + // None will be returned + if matches!( + Self::bound_compared(&scope_min, &scope_max, true).map(Ordering::is_le), + Some(true) + ) { + binaries.push(ConstantBinary::Scope { + min: scope_min, + max: scope_max, + }); + } + } else if eq_set.is_empty() { + binaries.push(ConstantBinary::Scope { + min: Bound::Unbounded, + max: Bound::Unbounded, + }); } + + Ok(()) } // Tips: It only makes sense if the condition is or aggregation - fn or_scope_aggregation(binaries: &Vec) -> Vec { + fn or_scope_aggregation(binaries: &mut Vec) { if binaries.is_empty() { - return vec![]; + return; } let mut scopes = Vec::new(); - let mut eqs = Vec::new(); + let mut eqs = HashSet::new(); let mut scope_margin = None; - for binary in binaries { + let sort_op = |binary: &&mut ConstantBinary| match binary { + ConstantBinary::NotEq(_) => 2, + ConstantBinary::Eq(_) => 1, + _ => 3, + }; + for binary in binaries.iter_mut().sorted_by_key(sort_op) { if matches!(scope_margin, Some((Bound::Unbounded, Bound::Unbounded))) { break; } @@ -313,7 +340,12 @@ impl ConstantBinary { scopes.push((min, max)) } - ConstantBinary::Eq(val) => eqs.push(val), + ConstantBinary::Eq(val) => { + let _ = eqs.insert(val.clone()); + } + ConstantBinary::NotEq(val) => { + let _ = eqs.remove(val); + } _ => (), } } @@ -321,7 +353,18 @@ impl ConstantBinary { scope_margin, Some((Bound::Unbounded, Bound::Unbounded)) | None ) { - return vec![]; + binaries.clear(); + if eqs.is_empty() { + binaries.push(ConstantBinary::Scope { + min: Bound::Unbounded, + max: Bound::Unbounded, + }); + } else { + for val in eqs { + binaries.push(ConstantBinary::Eq(val)); + } + } + return; } let mut merge_scopes: Vec<(Bound, Bound)> = Vec::new(); @@ -368,14 +411,14 @@ impl ConstantBinary { } } } - merge_scopes + *binaries = merge_scopes .into_iter() .map(|(min, max)| ConstantBinary::Scope { min: min.clone(), max: max.clone(), }) - .chain(eqs.into_iter().map(|val| ConstantBinary::Eq(val.clone()))) - .collect_vec() + .chain(eqs.into_iter().map(ConstantBinary::Eq)) + .collect_vec(); } fn join_write(f: &mut Formatter, binaries: &[ConstantBinary], op: &str) -> fmt::Result { @@ -971,9 +1014,16 @@ impl ScalarExpression { } (ConstantBinary::And(mut binaries), binary) | (binary, ConstantBinary::And(mut binaries)) => { - binaries.push(binary); + if op == &BinaryOperator::Or { + Ok(Some(ConstantBinary::Or(vec![ + binary, + ConstantBinary::And(binaries), + ]))) + } else { + binaries.push(binary); - Ok(Some(ConstantBinary::And(binaries))) + Ok(Some(ConstantBinary::And(binaries))) + } } (ConstantBinary::Or(mut binaries), binary) | (binary, ConstantBinary::Or(mut binaries)) => { @@ -1299,7 +1349,7 @@ mod test { binary.scope_aggregation()?; - assert_eq!(binary, ConstantBinary::And(vec![ConstantBinary::Eq(val_0)])); + assert_eq!(binary, ConstantBinary::And(vec![])); Ok(()) } @@ -1407,10 +1457,7 @@ mod test { binary.scope_aggregation()?; - assert_eq!( - binary, - ConstantBinary::And(vec![ConstantBinary::Eq(val_0.clone())]) - ); + assert_eq!(binary, ConstantBinary::And(vec![])); Ok(()) } @@ -1482,7 +1529,13 @@ mod test { binary.scope_aggregation()?; - assert_eq!(binary, ConstantBinary::Or(vec![])); + assert_eq!( + binary, + ConstantBinary::Or(vec![ConstantBinary::Scope { + min: Bound::Unbounded, + max: Bound::Unbounded + }]) + ); Ok(()) } diff --git a/src/optimizer/core/memo.rs b/src/optimizer/core/memo.rs index 3dd7ab1b..2e2a729f 100644 --- a/src/optimizer/core/memo.rs +++ b/src/optimizer/core/memo.rs @@ -83,6 +83,7 @@ mod tests { use crate::binder::{Binder, BinderContext}; use crate::db::DataBaseBuilder; use crate::errors::DatabaseError; + use crate::expression::simplify::ConstantBinary; use crate::optimizer::core::memo::Memo; use crate::optimizer::heuristic::batch::HepBatchStrategy; use crate::optimizer::heuristic::graph::HepGraph; @@ -92,7 +93,12 @@ mod tests { use crate::planner::operator::PhysicalOption; use crate::storage::kip::KipTransaction; use crate::storage::{Storage, Transaction}; + use crate::types::index::{IndexInfo, IndexMeta}; + use crate::types::value::DataValue; + use crate::types::LogicalType; use petgraph::stable_graph::NodeIndex; + use std::ops::Bound; + use std::sync::Arc; use tempfile::TempDir; #[tokio::test] @@ -152,11 +158,28 @@ mod tests { assert_eq!(exprs.exprs.len(), 2); assert_eq!(exprs.exprs[0].cost, Some(1000)); assert_eq!(exprs.exprs[0].op, PhysicalOption::SeqScan); - assert!(exprs.exprs[1].cost.unwrap() >= 1920); + assert!(exprs.exprs[1].cost.unwrap() >= 960); assert!(matches!(exprs.exprs[1].op, PhysicalOption::IndexScan(_))); assert_eq!( best_plan.as_ref().unwrap().childrens[0].childrens[0].childrens[0].physical_option, - Some(PhysicalOption::SeqScan) + Some(PhysicalOption::IndexScan(IndexInfo { + meta: Arc::new(IndexMeta { + id: 0, + column_ids: vec![0], + table_name: Arc::new("t1".to_string()), + pk_ty: LogicalType::Integer, + name: "pk_c1".to_string(), + is_unique: false, + is_primary: true, + }), + ranges: Some(vec![ + ConstantBinary::Eq(Arc::new(DataValue::Int32(Some(2)))), + ConstantBinary::Scope { + min: Bound::Excluded(Arc::new(DataValue::Int32(Some(40)))), + max: Bound::Unbounded, + } + ]), + })) ); Ok(()) diff --git a/src/optimizer/heuristic/graph.rs b/src/optimizer/heuristic/graph.rs index 17945eec..32f13102 100644 --- a/src/optimizer/heuristic/graph.rs +++ b/src/optimizer/heuristic/graph.rs @@ -218,6 +218,7 @@ mod tests { use crate::errors::DatabaseError; use crate::optimizer::heuristic::graph::{HepGraph, HepNodeId}; use crate::planner::operator::Operator; + use crate::planner::LogicalPlan; use petgraph::stable_graph::{EdgeIndex, NodeIndex}; #[tokio::test] @@ -350,7 +351,16 @@ mod tests { #[tokio::test] async fn test_graph_to_plan() -> Result<(), DatabaseError> { - let plan = select_sql_run("select * from t1 left join t2 on c1 = c3").await?; + fn clear_output_schema_buf(plan: &mut LogicalPlan) { + plan._output_schema_ref = None; + + for child in plan.childrens.iter_mut() { + clear_output_schema_buf(child); + } + } + let mut plan = select_sql_run("select * from t1 left join t2 on c1 = c3").await?; + clear_output_schema_buf(&mut plan); + let graph = HepGraph::new(plan.clone()); let plan_for_graph = graph.into_plan(None).unwrap(); diff --git a/src/optimizer/rule/implementation/dql/scan.rs b/src/optimizer/rule/implementation/dql/scan.rs index 7409cb0b..ce624679 100644 --- a/src/optimizer/rule/implementation/dql/scan.rs +++ b/src/optimizer/rule/implementation/dql/scan.rs @@ -71,23 +71,25 @@ impl ImplementationRule for IndexScanImplementation { if let Operator::Scan(scan_op) = op { let column_metas = loader.load(scan_op.table_name.clone())?; for index_info in scan_op.index_infos.iter() { - if index_info.binaries.is_none() { + if index_info.ranges.is_none() { continue; } let mut cost = None; - if let Some(binaries) = &index_info.binaries { + if let Some(binaries) = &index_info.ranges { // FIXME: Only UniqueIndex if let Some(column_meta) = find_column_meta(column_metas, &index_info.meta.column_ids[0]) { - // need to return table query(non-covering index) - cost = Some(column_meta.collect_count(binaries) * 2); + let mut row_count = column_meta.collect_count(binaries); + + if !index_info.meta.is_primary { + // need to return table query(non-covering index) + row_count *= 2; + } + cost = Some(row_count); } } - if matches!(cost, Some(0)) { - continue; - } group_expr.append_expr(Expression { op: PhysicalOption::IndexScan(index_info.clone()), diff --git a/src/optimizer/rule/normalization/pushdown_predicates.rs b/src/optimizer/rule/normalization/pushdown_predicates.rs index c67a0608..2e9420e6 100644 --- a/src/optimizer/rule/normalization/pushdown_predicates.rs +++ b/src/optimizer/rule/normalization/pushdown_predicates.rs @@ -218,19 +218,16 @@ impl NormalizationRule for PushPredicateIntoScan { if let Some(child_id) = graph.eldest_child_at(node_id) { if let Operator::Scan(child_op) = graph.operator_mut(child_id) { //FIXME: now only support `unique` and `primary key` - for IndexInfo { meta, binaries } in &mut child_op.index_infos { + for IndexInfo { meta, ranges } in &mut child_op.index_infos { let mut option = op .predicate .convert_binary(meta.table_name.as_str(), &meta.column_ids[0])?; if let Some(mut binary) = option.take() { binary.scope_aggregation()?; - let rearrange_binaries = binary.rearrange()?; + let rearrange_ranges = binary.rearrange()?; - if rearrange_binaries.is_empty() { - continue; - } - let _ = binaries.replace(rearrange_binaries); + let _ = ranges.replace(rearrange_ranges); return Ok(()); } @@ -283,7 +280,7 @@ mod tests { max: Bound::Unbounded, }]; - assert_eq!(op.index_infos[1].binaries, Some(mock_binaries)); + assert_eq!(op.index_infos[1].ranges, Some(mock_binaries)); } else { unreachable!("Should be a filter operator") } diff --git a/src/planner/operator/scan.rs b/src/planner/operator/scan.rs index 4c2cc587..d731b013 100644 --- a/src/planner/operator/scan.rs +++ b/src/planner/operator/scan.rs @@ -41,7 +41,7 @@ impl ScanOperator { .iter() .map(|meta| IndexInfo { meta: meta.clone(), - binaries: None, + ranges: None, }) .collect_vec(); diff --git a/src/storage/kip.rs b/src/storage/kip.rs index b7b0c5c8..6ddcc0c0 100644 --- a/src/storage/kip.rs +++ b/src/storage/kip.rs @@ -105,7 +105,7 @@ impl Transaction for KipTransaction { (offset_option, limit_option): Bounds, columns: Vec<(usize, ColumnRef)>, index_meta: IndexMetaRef, - binaries: Vec, + ranges: Vec, ) -> Result, DatabaseError> { assert!(columns.is_sorted_by_key(|(i, _)| i)); assert!(columns.iter().map(|(i, _)| i).all_unique()); @@ -129,7 +129,7 @@ impl Transaction for KipTransaction { index_meta, table, index_values: VecDeque::new(), - binaries: VecDeque::from(binaries), + ranges: VecDeque::from(ranges), tx: &self.tx, scope_iter: None, projections, @@ -633,7 +633,7 @@ mod test { is_primary: true, }), table: &table, - binaries: VecDeque::from(vec![ + ranges: VecDeque::from(vec![ ConstantBinary::Eq(Arc::new(DataValue::Int32(Some(0)))), ConstantBinary::Scope { min: Bound::Included(Arc::new(DataValue::Int32(Some(2)))), diff --git a/src/storage/mod.rs b/src/storage/mod.rs index d7711252..5c311429 100644 --- a/src/storage/mod.rs +++ b/src/storage/mod.rs @@ -46,7 +46,7 @@ pub trait Transaction: Sync + Send + 'static { bounds: Bounds, columns: Vec<(usize, ColumnRef)>, index_meta: IndexMetaRef, - binaries: Vec, + ranges: Vec, ) -> Result, DatabaseError>; fn add_index( @@ -117,7 +117,7 @@ pub struct IndexIter<'a> { // for buffering data index_values: VecDeque, - binaries: VecDeque, + ranges: VecDeque, scope_iter: Option>, } @@ -156,7 +156,7 @@ impl IndexIter<'_> { } fn is_empty(&self) -> bool { - self.scope_iter.is_none() && self.index_values.is_empty() && self.binaries.is_empty() + self.scope_iter.is_none() && self.index_values.is_empty() && self.ranges.is_empty() } } @@ -165,7 +165,7 @@ impl Iter for IndexIter<'_> { // 1. check limit if matches!(self.limit, Some(0)) || self.is_empty() { self.scope_iter = None; - self.binaries.clear(); + self.ranges.clear(); return Ok(None); } @@ -222,7 +222,7 @@ impl Iter for IndexIter<'_> { } // 4. When `scope_iter` and `index_values` do not have a value, use the next expression to iterate - if let Some(binary) = self.binaries.pop_front() { + if let Some(binary) = self.ranges.pop_front() { match binary { ConstantBinary::Scope { min, max } => { let table_name = &self.table.name; diff --git a/src/types/index.rs b/src/types/index.rs index 66fce027..ef61d89e 100644 --- a/src/types/index.rs +++ b/src/types/index.rs @@ -14,7 +14,7 @@ pub type IndexMetaRef = Arc; #[derive(Debug, Clone, Eq, PartialEq, Hash)] pub struct IndexInfo { pub(crate) meta: IndexMetaRef, - pub(crate) binaries: Option>, + pub(crate) ranges: Option>, } #[derive(Debug, Clone, Serialize, Deserialize, Eq, PartialEq, Hash)] @@ -44,14 +44,18 @@ impl fmt::Display for IndexInfo { write!(f, "{}", self.meta)?; write!(f, " => ")?; - if let Some(binaries) = &self.binaries { + if let Some(binaries) = &self.ranges { + if binaries.is_empty() { + write!(f, "DUMMY")?; + return Ok(()); + } let binaries = binaries .iter() .map(|binary| format!("{}", binary)) .join(", "); write!(f, "{}", binaries)?; } else { - write!(f, "NONE")?; + write!(f, "EMPTY")?; } Ok(()) diff --git a/tests/slt/crdb/sqlite.slt b/tests/slt/crdb/sqlite.slt index 87f087ba..cc5a7337 100644 --- a/tests/slt/crdb/sqlite.slt +++ b/tests/slt/crdb/sqlite.slt @@ -30,7 +30,7 @@ CREATE TABLE tab64784(pk INTEGER primary key, col0 INTEGER, col1 FLOAT, col2 VAR statement ok INSERT INTO tab64784 VALUES(0,212,202.62,'nshdy',212,208.79,'wsxfc'),(1,213,203.64,'xwfuo',213,209.26,'lyswz'),(2,214,204.82,'jnued',216,210.48,'qczzf'),(3,215,205.40,'jtijf',217,211.96,'dpugl'),(4,216,206.3,'dpdzk',219,212.43,'xfirg'),(5,218,207.43,'qpwyw',220,213.50,'fmgky'),(6,219,208.3,'uooxb',221,215.30,'xpmdy'),(7,220,209.54,'ndtbb',225,218.8,'ivqyw'),(8,221,210.65,'zjpts',226,219.82,'sezsm'),(9,222,211.57,'slaxq',227,220.91,'bdqyb') -# TODO: Support `In` subquery +# TODO: Support `NestLoopJoin` # query II # SELECT pk, col0 FROM tab64784 WHERE (col0 IN (SELECT col3 FROM tab64784 WHERE col3 IS NULL OR (col1 < 22.54) OR col4 > 85.74) OR ((col4 IS NULL)) AND col3 < 8 OR (col4 > 82.93 AND (col0 <= 61) AND col0 > 94 AND col0 > 15)) ORDER BY 2 DESC # ---- diff --git a/tests/slt/sql_2016/E061_09.slt b/tests/slt/sql_2016/E061_09.slt index c317ea3c..b02d81ff 100644 --- a/tests/slt/sql_2016/E061_09.slt +++ b/tests/slt/sql_2016/E061_09.slt @@ -1,22 +1,24 @@ # E061-09: Subqueries in comparison predicate -statement ok -CREATE TABLE TABLE_E061_09_01_01 ( ID INT PRIMARY KEY, A INT ); +# TODO: NestLoopJoin -query I -SELECT A FROM TABLE_E061_09_01_01 WHERE A < ( SELECT 1 ) +# statement ok +# CREATE TABLE TABLE_E061_09_01_01 ( ID INT PRIMARY KEY, A INT ); -statement ok -CREATE TABLE TABLE_E061_09_01_02 ( ID INT PRIMARY KEY, A INT ); +# query I +# SELECT A FROM TABLE_E061_09_01_01 WHERE A < ( SELECT 1 ) -query I -SELECT A FROM TABLE_E061_09_01_02 WHERE A <= ( SELECT 1 ) +# statement ok +# CREATE TABLE TABLE_E061_09_01_02 ( ID INT PRIMARY KEY, A INT ); -statement ok -CREATE TABLE TABLE_E061_09_01_03 ( ID INT PRIMARY KEY, A INT ); +# query I +# SELECT A FROM TABLE_E061_09_01_02 WHERE A <= ( SELECT 1 ) -query I -SELECT A FROM TABLE_E061_09_01_03 WHERE A <> ( SELECT 1 ) +# statement ok +# CREATE TABLE TABLE_E061_09_01_03 ( ID INT PRIMARY KEY, A INT ); + +# query I +# SELECT A FROM TABLE_E061_09_01_03 WHERE A <> ( SELECT 1 ) statement ok CREATE TABLE TABLE_E061_09_01_04 ( ID INT PRIMARY KEY, A INT ); @@ -24,14 +26,14 @@ CREATE TABLE TABLE_E061_09_01_04 ( ID INT PRIMARY KEY, A INT ); query I SELECT A FROM TABLE_E061_09_01_04 WHERE A = ( SELECT 1 ) -statement ok -CREATE TABLE TABLE_E061_09_01_05 ( ID INT PRIMARY KEY, A INT ); +# statement ok +# CREATE TABLE TABLE_E061_09_01_05 ( ID INT PRIMARY KEY, A INT ); -query I -SELECT A FROM TABLE_E061_09_01_05 WHERE A > ( SELECT 1 ) +# query I +# SELECT A FROM TABLE_E061_09_01_05 WHERE A > ( SELECT 1 ) -statement ok -CREATE TABLE TABLE_E061_09_01_06 ( ID INT PRIMARY KEY, A INT ); +# statement ok +# CREATE TABLE TABLE_E061_09_01_06 ( ID INT PRIMARY KEY, A INT ); -query I -SELECT A FROM TABLE_E061_09_01_06 WHERE A >= ( SELECT 1 ) +# query I +# SELECT A FROM TABLE_E061_09_01_06 WHERE A >= ( SELECT 1 ) diff --git a/tests/slt/sql_2016/F041_08.slt b/tests/slt/sql_2016/F041_08.slt index 50e31c01..d54dce2a 100644 --- a/tests/slt/sql_2016/F041_08.slt +++ b/tests/slt/sql_2016/F041_08.slt @@ -1,31 +1,33 @@ # F041-08: All comparison operators are supported (rather than just =) -statement ok -CREATE TABLE TABLE_F041_08_01_011 ( ID INT PRIMARY KEY, A INTEGER ); +# TODO: NestLoopJoin -statement ok -CREATE TABLE TABLE_F041_08_01_012 ( ID INT PRIMARY KEY, B INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_011 ( ID INT PRIMARY KEY, A INTEGER ); -query II -SELECT TABLE_F041_08_01_011.A, TABLE_F041_08_01_012.B FROM TABLE_F041_08_01_011 JOIN TABLE_F041_08_01_012 ON TABLE_F041_08_01_011.A < TABLE_F041_08_01_012.B +# statement ok +# CREATE TABLE TABLE_F041_08_01_012 ( ID INT PRIMARY KEY, B INTEGER ); -statement ok -CREATE TABLE TABLE_F041_08_01_021 ( ID INT PRIMARY KEY, A INTEGER ); +# query II +# SELECT TABLE_F041_08_01_011.A, TABLE_F041_08_01_012.B FROM TABLE_F041_08_01_011 JOIN TABLE_F041_08_01_012 ON TABLE_F041_08_01_011.A < TABLE_F041_08_01_012.B -statement ok -CREATE TABLE TABLE_F041_08_01_022 ( ID INT PRIMARY KEY, B INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_021 ( ID INT PRIMARY KEY, A INTEGER ); -query II -SELECT TABLE_F041_08_01_021.A, TABLE_F041_08_01_022.B FROM TABLE_F041_08_01_021 JOIN TABLE_F041_08_01_022 ON TABLE_F041_08_01_021.A <= TABLE_F041_08_01_022.B +# statement ok +# CREATE TABLE TABLE_F041_08_01_022 ( ID INT PRIMARY KEY, B INTEGER ); -statement ok -CREATE TABLE TABLE_F041_08_01_031 ( ID INT PRIMARY KEY, A INTEGER ); +# query II +# SELECT TABLE_F041_08_01_021.A, TABLE_F041_08_01_022.B FROM TABLE_F041_08_01_021 JOIN TABLE_F041_08_01_022 ON TABLE_F041_08_01_021.A <= TABLE_F041_08_01_022.B -statement ok -CREATE TABLE TABLE_F041_08_01_032 ( ID INT PRIMARY KEY, B INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_031 ( ID INT PRIMARY KEY, A INTEGER ); -query II -SELECT TABLE_F041_08_01_031.A, TABLE_F041_08_01_032.B FROM TABLE_F041_08_01_031 JOIN TABLE_F041_08_01_032 ON TABLE_F041_08_01_031.A <> TABLE_F041_08_01_032.B +# statement ok +# CREATE TABLE TABLE_F041_08_01_032 ( ID INT PRIMARY KEY, B INTEGER ); + +# query II +# SELECT TABLE_F041_08_01_031.A, TABLE_F041_08_01_032.B FROM TABLE_F041_08_01_031 JOIN TABLE_F041_08_01_032 ON TABLE_F041_08_01_031.A <> TABLE_F041_08_01_032.B statement ok CREATE TABLE TABLE_F041_08_01_041 ( ID INT PRIMARY KEY, A INTEGER ); @@ -36,20 +38,20 @@ CREATE TABLE TABLE_F041_08_01_042 ( ID INT PRIMARY KEY, B INTEGER ); query II SELECT TABLE_F041_08_01_041.A, TABLE_F041_08_01_042.B FROM TABLE_F041_08_01_041 JOIN TABLE_F041_08_01_042 ON TABLE_F041_08_01_041.A = TABLE_F041_08_01_042.B -statement ok -CREATE TABLE TABLE_F041_08_01_051 ( ID INT PRIMARY KEY, A INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_051 ( ID INT PRIMARY KEY, A INTEGER ); -statement ok -CREATE TABLE TABLE_F041_08_01_052 ( ID INT PRIMARY KEY, B INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_052 ( ID INT PRIMARY KEY, B INTEGER ); -query II -SELECT TABLE_F041_08_01_051.A, TABLE_F041_08_01_052.B FROM TABLE_F041_08_01_051 JOIN TABLE_F041_08_01_052 ON TABLE_F041_08_01_051.A > TABLE_F041_08_01_052.B +# query II +# SELECT TABLE_F041_08_01_051.A, TABLE_F041_08_01_052.B FROM TABLE_F041_08_01_051 JOIN TABLE_F041_08_01_052 ON TABLE_F041_08_01_051.A > TABLE_F041_08_01_052.B -statement ok -CREATE TABLE TABLE_F041_08_01_061 ( ID INT PRIMARY KEY, A INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_061 ( ID INT PRIMARY KEY, A INTEGER ); -statement ok -CREATE TABLE TABLE_F041_08_01_062 ( ID INT PRIMARY KEY, B INTEGER ); +# statement ok +# CREATE TABLE TABLE_F041_08_01_062 ( ID INT PRIMARY KEY, B INTEGER ); -query II -SELECT TABLE_F041_08_01_061.A, TABLE_F041_08_01_062.B FROM TABLE_F041_08_01_061 JOIN TABLE_F041_08_01_062 ON TABLE_F041_08_01_061.A >= TABLE_F041_08_01_062.B +# query II +# SELECT TABLE_F041_08_01_061.A, TABLE_F041_08_01_062.B FROM TABLE_F041_08_01_061 JOIN TABLE_F041_08_01_062 ON TABLE_F041_08_01_061.A >= TABLE_F041_08_01_062.B diff --git a/tests/slt/subquery.slt b/tests/slt/subquery.slt index bc0ea9ac..c6f394d1 100644 --- a/tests/slt/subquery.slt +++ b/tests/slt/subquery.slt @@ -34,15 +34,16 @@ select x.a from (select -a as a from t1) as x; -1 -3 -query III -select * from t1 where a <= (select 4) and a > (select 1) ----- -1 3 4 +# TODO: NestLoopJoin +# query III +# select * from t1 where a <= (select 4) and a > (select 1) +# ---- +# 1 3 4 -query III -select * from t1 where a <= (select 4) and (-a + 1) < (select 1) - 1 ----- -1 3 4 +# query III +# select * from t1 where a <= (select 4) and (-a + 1) < (select 1) - 1 +# ---- +# 1 3 4 statement ok insert into t1 values (2, 3, 3), (3, 1, 4); diff --git a/tests/slt/where_by_index.slt b/tests/slt/where_by_index.slt index 8bc7ecd4..22d82f85 100644 --- a/tests/slt/where_by_index.slt +++ b/tests/slt/where_by_index.slt @@ -26,6 +26,28 @@ select * from t1 where id = 0; ---- 0 1 2 +query IIT +select * from t1 where id = 0 and id = 1; +---- + +query IIT +select * from t1 where id = 0 and id != 0; +---- + +query IIT +select * from t1 where id = 0 or id != 0 limit 10; +---- +0 1 2 +3 4 5 +6 7 8 +9 10 11 +12 13 14 +15 16 17 +18 19 20 +21 22 23 +24 25 26 +27 28 29 + query IIT select * from t1 where id = 0 and id != 0 and id = 3; ----