diff --git a/.gitignore b/.gitignore index c0fc418a..f03ee8a4 100644 --- a/.gitignore +++ b/.gitignore @@ -24,4 +24,5 @@ sqlite_bench kite_sql_tpcc copy.csv -tests/data/row_20000.csv \ No newline at end of file +tests/data/row_20000.csv +tests/data/distinct_rows.csv \ No newline at end of file diff --git a/AGENT.md b/AGENT.md index 0aab5eec..602d72a7 100644 --- a/AGENT.md +++ b/AGENT.md @@ -88,6 +88,9 @@ PRs that modify logic but leave obvious test gaps untouched may be rejected. If something can be written clearly using fewer concepts, do that. +- Prefer incremental fixes: solve the problem at hand with the smallest reasonable change + before introducing new abstractions or generalizations. + --- ### 3.2 Low Comments, High Signal diff --git a/Cargo.lock b/Cargo.lock index 67dddff2..062751a5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -50,6 +50,15 @@ dependencies = [ "memchr", ] +[[package]] +name = "aligned-vec" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc890384c8602f339876ded803c97ad529f3842aba97f6392b3dba0dd171769b" +dependencies = [ + "equator", +] + [[package]] name = "allocator-api2" version = "0.2.21" @@ -815,6 +824,26 @@ dependencies = [ "log", ] +[[package]] +name = "equator" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4711b213838dfee0117e3be6ac926007d7f433d7bbe33595975d4190cb07e6fc" +dependencies = [ + "equator-macro", +] + +[[package]] +name = "equator-macro" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44f23cf4b44bfce11a86ace86f8a73ffdec849c9fd00a386a53d278bd9e81fb3" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.104", +] + [[package]] name = "equivalent" version = "1.0.2" @@ -1882,10 +1911,11 @@ dependencies = [ [[package]] name = "pprof" -version = "0.13.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" +checksum = "38a01da47675efa7673b032bf8efd8214f1917d89685e07e395ab125ea42b187" dependencies = [ + "aligned-vec", "backtrace", "cfg-if", "criterion", @@ -1895,11 +1925,11 @@ dependencies = [ "log", "nix", "once_cell", - "parking_lot", "smallvec", + "spin", "symbolic-demangle", "tempfile", - "thiserror 1.0.69", + "thiserror 2.0.12", ] [[package]] @@ -2499,6 +2529,15 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "spin" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5fe4ccb98d9c292d56fec89a5e07da7fc4cf0dc11e156b41793132775d3e591" +dependencies = [ + "lock_api", +] + [[package]] name = "sqlite" version = "0.34.0" @@ -2861,6 +2900,8 @@ dependencies = [ "ordered-float", "rand 0.8.5", "rust_decimal", + "sqlite", + "sqlparser", "thiserror 1.0.69", ] diff --git a/Cargo.toml b/Cargo.toml index ad53d1a5..92d5d0e3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -71,7 +71,7 @@ tokio = { version = "1.36", features = ["full"], optional = true [target.'cfg(unix)'.dev-dependencies] -pprof = { version = "0.13", features = ["flamegraph", "criterion"] } +pprof = { version = "0.15", features = ["flamegraph", "criterion"] } [target.'cfg(not(target_arch = "wasm32"))'.dev-dependencies] criterion = { version = "0.5", features = ["html_reports"] } diff --git a/Makefile b/Makefile index 1cfedbd4..dc834df2 100644 --- a/Makefile +++ b/Makefile @@ -3,7 +3,7 @@ CARGO ?= cargo WASM_PACK ?= wasm-pack SQLLOGIC_PATH ?= tests/slt/**/*.slt -.PHONY: test test-wasm test-slt test-all wasm-build check tpcc cargo-check build wasm-examples native-examples fmt clippy +.PHONY: test test-wasm test-slt test-all wasm-build check tpcc tpcc-dual cargo-check build wasm-examples native-examples fmt clippy ## Run default Rust tests in the current environment (non-WASM). test: @@ -47,6 +47,10 @@ check: fmt clippy tpcc: $(CARGO) run -p tpcc --release +## Execute TPCC while mirroring every statement to an in-memory SQLite instance for validation. +tpcc-dual: + $(CARGO) run -p tpcc --release -- --backend dual --measure-time 60 + ## Run JavaScript-based Wasm example scripts. wasm-examples: node examples/wasm_hello_world.test.mjs diff --git a/README.md b/README.md index c327bdd3..692b37f4 100755 --- a/README.md +++ b/README.md @@ -82,7 +82,8 @@ for tuple in iter { - [transaction](examples/transaction.rs) ## TPC-C -run `cargo run -p tpcc --release` to run tpcc +Run `make tpcc` (or `cargo run -p tpcc --release`) to execute the benchmark against the default KiteSQL storage. +Run `make tpcc-dual` to mirror every TPCC statement to an in-memory SQLite database alongside KiteSQL and assert the two engines return identical results; this target runs for 60 seconds (`--measure-time 60`). Use `cargo run -p tpcc --release -- --backend dual --measure-time ` for a custom duration. - i9-13900HX - 32.0 GB @@ -92,13 +93,13 @@ run `cargo run -p tpcc --release` to run tpcc All cases have been fully optimized. ```shell <90th Percentile RT (MaxRT)> - New-Order : 0.002 (0.012) - Payment : 0.001 (0.002) -Order-Status : 0.002 (0.019) - Delivery : 0.001 (0.001) - Stock-Level : 0.002 (0.018) + New-Order : 0.002 (0.006) + Payment : 0.001 (0.019) +Order-Status : 0.001 (0.003) + Delivery : 0.022 (0.038) + Stock-Level : 0.002 (0.005) -37166 Tpmc +18432 Tpmc ``` #### 👉[check more](tpcc/README.md) diff --git a/src/db.rs b/src/db.rs index 0677d721..a1cfb9e7 100644 --- a/src/db.rs +++ b/src/db.rs @@ -203,9 +203,12 @@ fn default_optimizer_pipeline() -> HepOptimizerPipeline { vec![NormalizationRuleImpl::TopK], ) .after_batch( - "Eliminate Redundant Sort".to_string(), + "Eliminate Aggregate".to_string(), HepBatchStrategy::once_topdown(), - vec![NormalizationRuleImpl::EliminateRedundantSort], + vec![ + NormalizationRuleImpl::EliminateRedundantSort, + NormalizationRuleImpl::UseStreamDistinct, + ], ) .after_batch( "Expression Remapper".to_string(), diff --git a/src/execution/dql/aggregate/mod.rs b/src/execution/dql/aggregate/mod.rs index 72751731..4d2b2f2b 100644 --- a/src/execution/dql/aggregate/mod.rs +++ b/src/execution/dql/aggregate/mod.rs @@ -17,6 +17,7 @@ mod count; pub mod hash_agg; mod min_max; pub mod simple_agg; +pub mod stream_distinct; mod sum; use crate::errors::DatabaseError; diff --git a/src/execution/dql/aggregate/stream_distinct.rs b/src/execution/dql/aggregate/stream_distinct.rs new file mode 100644 index 00000000..d1e73200 --- /dev/null +++ b/src/execution/dql/aggregate/stream_distinct.rs @@ -0,0 +1,240 @@ +// Copyright 2024 KipData/KiteSQL +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::execution::{build_read, spawn_executor, Executor, ReadExecutor}; +use crate::expression::ScalarExpression; +use crate::planner::operator::aggregate::AggregateOperator; +use crate::planner::LogicalPlan; +use crate::storage::{StatisticsMetaCache, TableCache, Transaction, ViewCache}; +use crate::throw; +use crate::types::tuple::Tuple; +use crate::types::value::DataValue; +use itertools::Itertools; + +pub struct StreamDistinctExecutor { + groupby_exprs: Vec, + input: LogicalPlan, +} + +impl From<(AggregateOperator, LogicalPlan)> for StreamDistinctExecutor { + fn from((op, input): (AggregateOperator, LogicalPlan)) -> Self { + StreamDistinctExecutor { + groupby_exprs: op.groupby_exprs, + input, + } + } +} + +impl<'a, T: Transaction + 'a> ReadExecutor<'a, T> for StreamDistinctExecutor { + fn execute( + self, + cache: (&'a TableCache, &'a ViewCache, &'a StatisticsMetaCache), + transaction: *mut T, + ) -> Executor<'a> { + spawn_executor(move |co| async move { + let StreamDistinctExecutor { + groupby_exprs, + mut input, + } = self; + + let schema_ref = input.output_schema().clone(); + let mut executor = build_read(input, cache, transaction); + let mut last_keys: Option> = None; + + for result in executor.by_ref() { + let tuple = throw!(co, result); + let group_keys: Vec = throw!( + co, + groupby_exprs + .iter() + .map(|expr| expr.eval(Some((&tuple, &schema_ref)))) + .try_collect() + ); + + if last_keys.as_ref() != Some(&group_keys) { + last_keys = Some(group_keys.clone()); + co.yield_(Ok(Tuple::new(tuple.pk, group_keys))).await; + } + } + }) + } +} + +#[cfg(all(test, not(target_arch = "wasm32")))] +mod tests { + use crate::catalog::{ColumnCatalog, ColumnDesc, ColumnRef}; + use crate::errors::DatabaseError; + use crate::execution::dql::aggregate::stream_distinct::StreamDistinctExecutor; + use crate::execution::{try_collect, ReadExecutor}; + use crate::expression::ScalarExpression; + use crate::optimizer::heuristic::batch::HepBatchStrategy; + use crate::optimizer::heuristic::optimizer::HepOptimizerPipeline; + use crate::optimizer::rule::normalization::NormalizationRuleImpl; + use crate::planner::operator::aggregate::AggregateOperator; + use crate::planner::operator::values::ValuesOperator; + use crate::planner::operator::Operator; + use crate::planner::{Childrens, LogicalPlan}; + use crate::storage::rocksdb::{RocksStorage, RocksTransaction}; + use crate::storage::{StatisticsMetaCache, Storage, TableCache, ViewCache}; + use crate::types::value::DataValue; + use crate::types::LogicalType; + use crate::utils::lru::SharedLruCache; + use itertools::Itertools; + use std::hash::RandomState; + use std::sync::Arc; + use tempfile::TempDir; + + #[allow(clippy::type_complexity)] + fn build_test_storage() -> Result< + ( + Arc, + Arc, + Arc, + TempDir, + RocksStorage, + ), + DatabaseError, + > { + let meta_cache = Arc::new(SharedLruCache::new(4, 1, RandomState::new())?); + let view_cache = Arc::new(SharedLruCache::new(4, 1, RandomState::new())?); + let table_cache = Arc::new(SharedLruCache::new(4, 1, RandomState::new())?); + + let temp_dir = TempDir::new().expect("unable to create temporary working directory"); + let storage = RocksStorage::new(temp_dir.path())?; + + Ok((table_cache, view_cache, meta_cache, temp_dir, storage)) + } + + fn optimize_exprs(plan: LogicalPlan) -> Result { + HepOptimizerPipeline::builder() + .before_batch( + "Expression Remapper".to_string(), + HepBatchStrategy::once_topdown(), + vec![ + NormalizationRuleImpl::BindExpressionPosition, + NormalizationRuleImpl::EvaluatorBind, + ], + ) + .build() + .instantiate(plan) + .find_best::(None) + } + + #[test] + fn stream_distinct_single_column_sorted() -> Result<(), DatabaseError> { + let desc = ColumnDesc::new(LogicalType::Integer, None, false, None)?; + let schema_ref = Arc::new(vec![ColumnRef::from(ColumnCatalog::new( + "c1".to_string(), + true, + desc, + ))]); + + let input = LogicalPlan::new( + Operator::Values(ValuesOperator { + rows: vec![ + vec![DataValue::Int32(1)], + vec![DataValue::Int32(1)], + vec![DataValue::Int32(2)], + vec![DataValue::Int32(2)], + vec![DataValue::Int32(3)], + ], + schema_ref: schema_ref.clone(), + }), + Childrens::None, + ); + let agg = AggregateOperator { + groupby_exprs: vec![ScalarExpression::column_expr(schema_ref[0].clone())], + agg_calls: vec![], + is_distinct: true, + }; + let plan = LogicalPlan::new(Operator::Aggregate(agg), Childrens::Only(Box::new(input))); + let plan = optimize_exprs(plan)?; + let Operator::Aggregate(agg) = plan.operator else { + unreachable!() + }; + + let (table_cache, view_cache, meta_cache, _temp_dir, storage) = build_test_storage()?; + let mut transaction = storage.transaction()?; + let tuples = try_collect( + StreamDistinctExecutor::from((agg, plan.childrens.pop_only())) + .execute((&table_cache, &view_cache, &meta_cache), &mut transaction), + )?; + + let actual = tuples + .into_iter() + .flat_map(|tuple| tuple.values) + .flat_map(|value| value.i32()) + .collect_vec(); + assert_eq!(actual, vec![1, 2, 3]); + + Ok(()) + } + + #[test] + fn stream_distinct_multi_column_sorted() -> Result<(), DatabaseError> { + let desc = ColumnDesc::new(LogicalType::Integer, None, false, None)?; + let schema_ref = Arc::new(vec![ + ColumnRef::from(ColumnCatalog::new("c1".to_string(), true, desc.clone())), + ColumnRef::from(ColumnCatalog::new("c2".to_string(), true, desc)), + ]); + + let input = LogicalPlan::new( + Operator::Values(ValuesOperator { + rows: vec![ + vec![DataValue::Int32(1), DataValue::Int32(1)], + vec![DataValue::Int32(1), DataValue::Int32(1)], + vec![DataValue::Int32(1), DataValue::Int32(2)], + vec![DataValue::Int32(2), DataValue::Int32(1)], + vec![DataValue::Int32(2), DataValue::Int32(1)], + ], + schema_ref: schema_ref.clone(), + }), + Childrens::None, + ); + let agg = AggregateOperator { + groupby_exprs: vec![ + ScalarExpression::column_expr(schema_ref[0].clone()), + ScalarExpression::column_expr(schema_ref[1].clone()), + ], + agg_calls: vec![], + is_distinct: true, + }; + let plan = LogicalPlan::new(Operator::Aggregate(agg), Childrens::Only(Box::new(input))); + let plan = optimize_exprs(plan)?; + let Operator::Aggregate(agg) = plan.operator else { + unreachable!() + }; + + let (table_cache, view_cache, meta_cache, _temp_dir, storage) = build_test_storage()?; + let mut transaction = storage.transaction()?; + let tuples = try_collect( + StreamDistinctExecutor::from((agg, plan.childrens.pop_only())) + .execute((&table_cache, &view_cache, &meta_cache), &mut transaction), + )?; + + let actual = tuples + .into_iter() + .map(|tuple| { + tuple + .values + .into_iter() + .flat_map(|value| value.i32()) + .collect_vec() + }) + .collect_vec(); + assert_eq!(actual, vec![vec![1, 1], vec![1, 2], vec![2, 1]]); + + Ok(()) + } +} diff --git a/src/execution/mod.rs b/src/execution/mod.rs index 76027fed..14f0a3dc 100644 --- a/src/execution/mod.rs +++ b/src/execution/mod.rs @@ -36,6 +36,7 @@ use crate::execution::dml::insert::Insert; use crate::execution::dml::update::Update; use crate::execution::dql::aggregate::hash_agg::HashAggExecutor; use crate::execution::dql::aggregate::simple_agg::SimpleAggExecutor; +use crate::execution::dql::aggregate::stream_distinct::StreamDistinctExecutor; use crate::execution::dql::describe::Describe; use crate::execution::dql::dummy::Dummy; use crate::execution::dql::except::Except; @@ -96,6 +97,7 @@ pub fn build_read<'a, T: Transaction + 'a>( let LogicalPlan { operator, childrens, + physical_option, .. } = plan; @@ -106,6 +108,17 @@ pub fn build_read<'a, T: Transaction + 'a>( if op.groupby_exprs.is_empty() { SimpleAggExecutor::from((op, input)).execute(cache, transaction) + } else if op.is_distinct + && op.agg_calls.is_empty() + && matches!( + physical_option, + Some(PhysicalOption { + plan: PlanImpl::StreamDistinct, + .. + }) + ) + { + StreamDistinctExecutor::from((op, input)).execute(cache, transaction) } else { HashAggExecutor::from((op, input)).execute(cache, transaction) } @@ -122,7 +135,7 @@ pub fn build_read<'a, T: Transaction + 'a>( JoinCondition::On { on, .. } if !on.is_empty() && matches!( - plan.physical_option, + physical_option, Some(PhysicalOption { plan: PlanImpl::HashJoin, .. @@ -143,23 +156,30 @@ pub fn build_read<'a, T: Transaction + 'a>( } Operator::TableScan(op) => { if let Some(PhysicalOption { - plan: - PlanImpl::IndexScan(IndexInfo { + plan: PlanImpl::IndexScan(index_info), + .. + }) = physical_option + { + let IndexInfo { + meta, + range, + covered_deserializers, + cover_mapping, + .. + } = *index_info; + if let Some(range) = range { + return IndexScan::from(( + op, meta, - range: Some(range), + range, covered_deserializers, cover_mapping, - sort_option: _, - sort_elimination_hint: _, - }), - .. - }) = plan.physical_option - { - IndexScan::from((op, meta, range, covered_deserializers, cover_mapping)) - .execute(cache, transaction) - } else { - SeqScan::from(op).execute(cache, transaction) + )) + .execute(cache, transaction); + } } + + SeqScan::from(op).execute(cache, transaction) } Operator::FunctionScan(op) => FunctionScan::from(op).execute(cache, transaction), Operator::Sort(op) => { diff --git a/src/expression/simplify.rs b/src/expression/simplify.rs index 1f28cadf..f63cc604 100644 --- a/src/expression/simplify.rs +++ b/src/expression/simplify.rs @@ -97,6 +97,14 @@ impl VisitorMut<'_> for ConstantCalculator { let _ = mem::replace(expr, ScalarExpression::Constant(value)); } } + ScalarExpression::TypeCast { expr: arg_expr, ty } => { + self.visit(arg_expr)?; + + if let ScalarExpression::Constant(value) = arg_expr.as_mut() { + let casted = mem::replace(value, DataValue::Null).cast(ty)?; + let _ = mem::replace(expr, ScalarExpression::Constant(casted)); + } + } _ => walk_mut_expr(self, expr)?, } diff --git a/src/optimizer/core/memo.rs b/src/optimizer/core/memo.rs index beb2d9c7..dd99392f 100644 --- a/src/optimizer/core/memo.rs +++ b/src/optimizer/core/memo.rs @@ -275,7 +275,7 @@ mod tests { .pop_only() .physical_option, Some(PhysicalOption::new( - PlanImpl::IndexScan(IndexInfo { + PlanImpl::IndexScan(Box::new(IndexInfo { meta: Arc::new(IndexMeta { id: 0, column_ids: vec![c1_column.id().unwrap()], @@ -299,7 +299,8 @@ mod tests { covered_deserializers: None, cover_mapping: None, sort_elimination_hint: None, - }), + stream_distinct_hint: None, + })), SortOption::OrderBy { fields: sort_fields, ignore_prefix_len: 0, diff --git a/src/optimizer/heuristic/optimizer.rs b/src/optimizer/heuristic/optimizer.rs index 17f1f5a5..3a2bb222 100644 --- a/src/optimizer/heuristic/optimizer.rs +++ b/src/optimizer/heuristic/optimizer.rs @@ -20,8 +20,10 @@ use crate::optimizer::core::statistics_meta::StatisticMetaLoader; use crate::optimizer::heuristic::batch::{HepBatch, HepBatchStrategy}; use crate::optimizer::heuristic::matcher::PlanMatcher; use crate::optimizer::rule::implementation::ImplementationRuleImpl; -use crate::optimizer::rule::normalization::annotate_sort_preserving_indexes; use crate::optimizer::rule::normalization::NormalizationRuleImpl; +use crate::optimizer::rule::normalization::{ + annotate_sort_preserving_indexes, annotate_stream_distinct_indexes, +}; use crate::planner::{Childrens, LogicalPlan}; use crate::storage::Transaction; use std::ops::Not; @@ -54,6 +56,7 @@ impl<'a> HepOptimizer<'a> { ) -> Result { Self::apply_batches(&mut self.plan, self.before_batches)?; annotate_sort_preserving_indexes(&mut self.plan); + annotate_stream_distinct_indexes(&mut self.plan); if let Some(loader) = loader { if self.implementations.is_empty().not() { diff --git a/src/optimizer/rule/implementation/dql/table_scan.rs b/src/optimizer/rule/implementation/dql/table_scan.rs index a3bcf2dd..22574216 100644 --- a/src/optimizer/rule/implementation/dql/table_scan.rs +++ b/src/optimizer/rule/implementation/dql/table_scan.rs @@ -101,17 +101,35 @@ impl ImplementationRule for IndexScanImplementation { } } - if let (Some(covered), Some(row_count)) = (index_info.sort_elimination_hint, cost) { - let rows = row_count.max(1) as f64; - let raw_bonus = rows * rows.log2(); - // TODO: replace this heuristic with accurate row-count driven sort cost once available. - let bonus = (raw_bonus as usize) / covered.max(1); - cost = Some(row_count.saturating_sub(bonus)); + if let Some(row_count) = cost { + // bonus = 0.5 * (rows * log2(rows)) / log2(hint_sum), capped to rows/2 + // c = rows - bonus, min rows/2 + // c + // rows |\ + // | \ + // | \ + // | \______ + // rows/2 |__________\______ + // | + // | + // └────────────────────── hint_sum + // 2 4 8 16 32 + let hint_sum = index_info.sort_elimination_hint.unwrap_or(0) + + index_info.stream_distinct_hint.unwrap_or(0); + if hint_sum > 0 { + let rows = row_count.max(1) as f64; + let raw_bonus = rows * rows.log2(); + let hint_weight = (hint_sum as f64).log2().max(1.0); + // TODO: replace this heuristic with accurate row-count driven sort cost once available. + let bonus = (raw_bonus / hint_weight * 0.5) as usize; + let min_cost = row_count.div_ceil(2); + cost = Some(row_count.saturating_sub(bonus).max(min_cost)); + } } group_expr.append_expr(Expression { op: PhysicalOption::new( - PlanImpl::IndexScan(index_info.clone()), + PlanImpl::IndexScan(Box::new(index_info.clone())), index_info.sort_option.clone(), ), cost, diff --git a/src/optimizer/rule/normalization/sort_elimination.rs b/src/optimizer/rule/normalization/agg_elimination.rs similarity index 62% rename from src/optimizer/rule/normalization/sort_elimination.rs rename to src/optimizer/rule/normalization/agg_elimination.rs index bf454ae1..c122e375 100644 --- a/src/optimizer/rule/normalization/sort_elimination.rs +++ b/src/optimizer/rule/normalization/agg_elimination.rs @@ -14,6 +14,7 @@ use crate::catalog::ColumnRef; use crate::errors::DatabaseError; +use crate::expression::ScalarExpression; use crate::optimizer::core::pattern::{Pattern, PatternChildrenPredicate}; use crate::optimizer::core::rule::{MatchPattern, NormalizationRule}; use crate::optimizer::plan_utils::{only_child_mut, replace_with_only_child}; @@ -76,6 +77,39 @@ pub fn annotate_sort_preserving_indexes(plan: &mut LogicalPlan) { } fn mark_sort_preserving_indexes(plan: &mut LogicalPlan, required: &[SortField]) { + mark_order_hint(plan, required, OrderHintKind::SortElimination); +} + +pub fn annotate_stream_distinct_indexes(plan: &mut LogicalPlan) { + fn visit(plan: &mut LogicalPlan) { + if let Operator::Aggregate(op) = &plan.operator { + if op.is_distinct && op.agg_calls.is_empty() && !op.groupby_exprs.is_empty() { + if let Childrens::Only(child) = plan.childrens.as_mut() { + let required = distinct_sort_fields(&op.groupby_exprs); + mark_order_hint(child, &required, OrderHintKind::StreamDistinct); + } + } + } + + match plan.childrens.as_mut() { + Childrens::Only(child) => visit(child), + Childrens::Twins { left, right } => { + visit(left); + visit(right); + } + Childrens::None => {} + } + } + visit(plan); +} + +#[derive(Copy, Clone)] +enum OrderHintKind { + SortElimination, + StreamDistinct, +} + +fn mark_order_hint(plan: &mut LogicalPlan, required: &[SortField], hint: OrderHintKind) { if required.is_empty() { return; } @@ -87,7 +121,7 @@ fn mark_sort_preserving_indexes(plan: &mut LogicalPlan, required: &[SortField]) | Operator::TopK(_) | Operator::Sort(_) => { if let Childrens::Only(child) = plan.childrens.as_mut() { - mark_sort_preserving_indexes(child, required); + mark_order_hint(child, required, hint); } } Operator::TableScan(scan_op) => { @@ -104,11 +138,22 @@ fn mark_sort_preserving_indexes(plan: &mut LogicalPlan, required: &[SortField]) for index_info in scan_op.index_infos.iter_mut() { if covers(required, &index_info.sort_option) { let covered = required.len(); - index_info.sort_elimination_hint = Some( - index_info - .sort_elimination_hint - .map_or(covered, |old| old.max(covered)), - ); + match hint { + OrderHintKind::SortElimination => { + index_info.sort_elimination_hint = Some( + index_info + .sort_elimination_hint + .map_or(covered, |old| old.max(covered)), + ); + } + OrderHintKind::StreamDistinct => { + index_info.stream_distinct_hint = Some( + index_info + .stream_distinct_hint + .map_or(covered, |old| old.max(covered)), + ); + } + } } } } @@ -116,6 +161,98 @@ fn mark_sort_preserving_indexes(plan: &mut LogicalPlan, required: &[SortField]) } } +fn distinct_sort_fields(groupby_exprs: &[ScalarExpression]) -> Vec { + groupby_exprs + .iter() + .cloned() + .map(|expr| SortField::new(expr, true, true)) + .collect() +} + +static STREAM_DISTINCT_PATTERN: LazyLock = LazyLock::new(|| Pattern { + predicate: |op| match op { + Operator::Aggregate(op) => { + op.is_distinct && op.agg_calls.is_empty() && !op.groupby_exprs.is_empty() + } + _ => false, + }, + children: PatternChildrenPredicate::None, +}); + +pub struct UseStreamDistinct; + +impl MatchPattern for UseStreamDistinct { + fn pattern(&self) -> &Pattern { + &STREAM_DISTINCT_PATTERN + } +} + +impl NormalizationRule for UseStreamDistinct { + fn apply(&self, plan: &mut LogicalPlan) -> Result { + let Operator::Aggregate(op) = &plan.operator else { + return Ok(false); + }; + if !op.is_distinct || !op.agg_calls.is_empty() || op.groupby_exprs.is_empty() { + return Ok(false); + } + if !matches!( + &plan.physical_option, + Some(PhysicalOption { + plan: PlanImpl::HashAggregate, + .. + }) + ) { + return Ok(false); + } + + let required = distinct_sort_fields(&op.groupby_exprs); + let child = match only_child_mut(plan) { + Some(child) => child, + None => return Ok(false), + }; + if !ensure_stream_distinct_order(child, &required) { + return Ok(false); + } + + plan.physical_option = Some(PhysicalOption::new( + PlanImpl::StreamDistinct, + SortOption::Follow, + )); + Ok(true) + } +} + +fn ensure_stream_distinct_order(plan: &mut LogicalPlan, required: &[SortField]) -> bool { + if let Some(PhysicalOption { + plan: PlanImpl::IndexScan(index_info), + .. + }) = plan.physical_option.as_ref() + { + if covers(required, &index_info.sort_option) { + return true; + } + } + + if let Some(physical_option) = plan.physical_option.as_ref() { + match physical_option.sort_option() { + SortOption::OrderBy { .. } if covers(required, physical_option.sort_option()) => { + return true + } + SortOption::OrderBy { .. } => {} + SortOption::Follow => { + if let Childrens::Only(child) = plan.childrens.as_mut() { + if ensure_stream_distinct_order(child, required) { + return true; + } + } + } + SortOption::None => {} + } + } + + false +} + fn ensure_index_order(plan: &mut LogicalPlan, required: &[SortField]) -> bool { if let Some(PhysicalOption { plan: PlanImpl::IndexScan(index_info), @@ -175,12 +312,13 @@ fn covers(required: &[SortField], provided: &SortOption) -> bool { #[cfg(all(test, not(target_arch = "wasm32")))] mod tests { - use super::EliminateRedundantSort; + use super::{EliminateRedundantSort, UseStreamDistinct}; use crate::catalog::{ColumnCatalog, ColumnRef, TableName}; use crate::errors::DatabaseError; use crate::expression::range_detacher::Range; use crate::expression::ScalarExpression; use crate::optimizer::core::rule::NormalizationRule; + use crate::planner::operator::aggregate::AggregateOperator; use crate::planner::operator::filter::FilterOperator; use crate::planner::operator::sort::{SortField, SortOperator}; use crate::planner::operator::table_scan::TableScanOperator; @@ -208,7 +346,7 @@ mod tests { let mut leaf = LogicalPlan::new(Operator::Dummy, Childrens::None); leaf.physical_option = Some(PhysicalOption::new( - PlanImpl::IndexScan(index_info), + PlanImpl::IndexScan(Box::new(index_info)), index_sort_option, )); @@ -259,10 +397,69 @@ mod tests { covered_deserializers: None, cover_mapping: None, sort_elimination_hint: None, + stream_distinct_hint: None, }; (index_info, sort_option) } + fn build_distinct_scan_plan() -> (LogicalPlan, SortOption) { + let table_name: TableName = Arc::from("t1"); + let c1 = ColumnRef::from(ColumnCatalog::new_dummy("c1".to_string())); + let c1_id = Ulid::new(); + let mut columns = BTreeMap::new(); + columns.insert(0, c1.clone()); + + let sort_fields = vec![SortField::new( + ScalarExpression::column_expr(c1.clone()), + true, + true, + )]; + let sort_option = SortOption::OrderBy { + fields: sort_fields.clone(), + ignore_prefix_len: 0, + }; + let index_info = IndexInfo { + meta: Arc::new(IndexMeta { + id: 1, + column_ids: vec![c1_id], + table_name: table_name.clone(), + pk_ty: LogicalType::Integer, + value_ty: LogicalType::Integer, + name: "idx".to_string(), + ty: IndexType::PrimaryKey { is_multiple: false }, + }), + sort_option: sort_option.clone(), + range: None, + covered_deserializers: None, + cover_mapping: None, + sort_elimination_hint: None, + stream_distinct_hint: None, + }; + + let scan = LogicalPlan::new( + Operator::TableScan(TableScanOperator { + table_name, + primary_keys: vec![c1_id], + columns, + limit: (None, None), + index_infos: vec![index_info], + with_pk: false, + }), + Childrens::None, + ); + + let plan = LogicalPlan::new( + Operator::Aggregate(AggregateOperator { + groupby_exprs: vec![ScalarExpression::column_expr(c1)], + agg_calls: vec![], + is_distinct: true, + }), + Childrens::Only(Box::new(scan)), + ); + + (plan, sort_option) + } + #[test] fn remove_sort_when_index_matches_order() -> Result<(), DatabaseError> { let sort_field = make_sort_field("c1"); @@ -331,6 +528,50 @@ mod tests { Ok(()) } + #[test] + fn annotate_sets_stream_distinct_hint_on_table_scan() -> Result<(), DatabaseError> { + let (mut plan, _) = build_distinct_scan_plan(); + + super::annotate_stream_distinct_indexes(&mut plan); + let child = plan.childrens.pop_only(); + let Operator::TableScan(scan_op) = child.operator else { + unreachable!() + }; + + assert_eq!(scan_op.index_infos.len(), 1); + assert_eq!(scan_op.index_infos[0].stream_distinct_hint, Some(1)); + Ok(()) + } + + #[test] + fn use_stream_distinct_when_order_satisfied() -> Result<(), DatabaseError> { + let (mut plan, sort_option) = build_distinct_scan_plan(); + if let Childrens::Only(child) = plan.childrens.as_mut() { + if let Operator::TableScan(scan_op) = &child.operator { + let index_info = scan_op.index_infos[0].clone(); + child.physical_option = Some(PhysicalOption::new( + PlanImpl::IndexScan(Box::new(index_info)), + sort_option.clone(), + )); + } + } + plan.physical_option = Some(PhysicalOption::new( + PlanImpl::HashAggregate, + SortOption::None, + )); + + let rule = UseStreamDistinct; + assert!(rule.apply(&mut plan)?); + assert!(matches!( + plan.physical_option, + Some(PhysicalOption { + plan: PlanImpl::StreamDistinct, + .. + }) + )); + Ok(()) + } + #[test] fn keep_sort_when_order_not_covered() -> Result<(), DatabaseError> { let c1 = make_sort_field("c1"); @@ -371,7 +612,7 @@ mod tests { if let Operator::TableScan(scan_op) = &scan_plan.operator { let index_info = scan_op.index_infos[0].clone(); scan_plan.physical_option = Some(PhysicalOption::new( - PlanImpl::IndexScan(index_info.clone()), + PlanImpl::IndexScan(Box::new(index_info.clone())), index_info.sort_option.clone(), )); } diff --git a/src/optimizer/rule/normalization/mod.rs b/src/optimizer/rule/normalization/mod.rs index 4b2129ed..348cc4d8 100644 --- a/src/optimizer/rule/normalization/mod.rs +++ b/src/optimizer/rule/normalization/mod.rs @@ -24,6 +24,9 @@ use crate::optimizer::rule::normalization::compilation_in_advance::{ BindExpressionPosition, EvaluatorBind, }; +use crate::optimizer::rule::normalization::agg_elimination::{ + EliminateRedundantSort, UseStreamDistinct, +}; use crate::optimizer::rule::normalization::pushdown_limit::{ LimitProjectTranspose, PushLimitIntoScan, PushLimitThroughJoin, }; @@ -32,18 +35,17 @@ use crate::optimizer::rule::normalization::pushdown_predicates::{ }; use crate::optimizer::rule::normalization::simplification::ConstantCalculation; use crate::optimizer::rule::normalization::simplification::SimplifyFilter; -use crate::optimizer::rule::normalization::sort_elimination::EliminateRedundantSort; use crate::optimizer::rule::normalization::top_k::TopK; use crate::planner::LogicalPlan; +mod agg_elimination; mod column_pruning; mod combine_operators; mod compilation_in_advance; mod pushdown_limit; mod pushdown_predicates; mod simplification; -mod sort_elimination; mod top_k; -pub use sort_elimination::annotate_sort_preserving_indexes; +pub use agg_elimination::{annotate_sort_preserving_indexes, annotate_stream_distinct_indexes}; #[derive(Debug, Copy, Clone)] pub enum NormalizationRuleImpl { @@ -69,6 +71,7 @@ pub enum NormalizationRuleImpl { EvaluatorBind, TopK, EliminateRedundantSort, + UseStreamDistinct, } impl MatchPattern for NormalizationRuleImpl { @@ -90,6 +93,7 @@ impl MatchPattern for NormalizationRuleImpl { NormalizationRuleImpl::EvaluatorBind => EvaluatorBind.pattern(), NormalizationRuleImpl::TopK => TopK.pattern(), NormalizationRuleImpl::EliminateRedundantSort => EliminateRedundantSort.pattern(), + NormalizationRuleImpl::UseStreamDistinct => UseStreamDistinct.pattern(), } } } @@ -115,6 +119,7 @@ impl NormalizationRule for NormalizationRuleImpl { NormalizationRuleImpl::EvaluatorBind => EvaluatorBind.apply(plan), NormalizationRuleImpl::TopK => TopK.apply(plan), NormalizationRuleImpl::EliminateRedundantSort => EliminateRedundantSort.apply(plan), + NormalizationRuleImpl::UseStreamDistinct => UseStreamDistinct.apply(plan), } } } diff --git a/src/optimizer/rule/normalization/pushdown_predicates.rs b/src/optimizer/rule/normalization/pushdown_predicates.rs index 71842859..94a12d34 100644 --- a/src/optimizer/rule/normalization/pushdown_predicates.rs +++ b/src/optimizer/rule/normalization/pushdown_predicates.rs @@ -258,6 +258,7 @@ impl NormalizationRule for PushPredicateIntoScan { cover_mapping, sort_option, sort_elimination_hint: _, + stream_distinct_hint: _, } in &mut scan_op.index_infos { if range.is_some() { @@ -664,6 +665,7 @@ mod tests { covered_deserializers: None, cover_mapping: None, sort_elimination_hint: None, + stream_distinct_hint: None, }, IndexInfo { meta: index_meta_aligned, @@ -675,6 +677,7 @@ mod tests { covered_deserializers: None, cover_mapping: None, sort_elimination_hint: None, + stream_distinct_hint: None, }, ], with_pk: false, diff --git a/src/optimizer/rule/normalization/simplification.rs b/src/optimizer/rule/normalization/simplification.rs index ffeb33de..ccd0560b 100644 --- a/src/optimizer/rule/normalization/simplification.rs +++ b/src/optimizer/rule/normalization/simplification.rs @@ -204,6 +204,45 @@ mod test { Ok(()) } + #[test] + fn test_constant_cast_elimination() -> Result<(), DatabaseError> { + let table_state = build_t1_table()?; + let plan = table_state + .plan("select cast(1 as int), cast(2 as int) + 1 from t1 where cast(3 as int) = 3")?; + + let best_plan = run_with_single_batch( + plan, + "test_constant_cast_elimination", + HepBatchStrategy::once_topdown(), + vec![NormalizationRuleImpl::ConstantCalculation], + )?; + + if let Operator::Project(project_op) = best_plan.operator { + assert_eq!( + project_op.exprs[0], + ScalarExpression::Constant(DataValue::Int32(1)) + ); + assert_eq!( + project_op.exprs[1], + ScalarExpression::Constant(DataValue::Int32(3)) + ); + } else { + unreachable!(); + } + + let filter_op = best_plan.childrens.pop_only(); + if let Operator::Filter(filter_op) = filter_op.operator { + assert_eq!( + filter_op.predicate, + ScalarExpression::Constant(DataValue::Boolean(true)) + ); + } else { + unreachable!(); + } + + Ok(()) + } + #[test] fn test_simplify_filter_single_column() -> Result<(), DatabaseError> { let table_state = build_t1_table()?; diff --git a/src/planner/operator/mod.rs b/src/planner/operator/mod.rs index 74b3437c..53cdc909 100644 --- a/src/planner/operator/mod.rs +++ b/src/planner/operator/mod.rs @@ -149,13 +149,14 @@ pub enum PlanImpl { Dummy, SimpleAggregate, HashAggregate, + StreamDistinct, Filter, HashJoin, NestLoopJoin, Project, SeqScan, FunctionScan, - IndexScan(IndexInfo), + IndexScan(Box), Sort, Limit, TopK, @@ -399,6 +400,7 @@ impl fmt::Display for PlanImpl { PlanImpl::Dummy => write!(f, "Dummy"), PlanImpl::SimpleAggregate => write!(f, "SimpleAggregate"), PlanImpl::HashAggregate => write!(f, "HashAggregate"), + PlanImpl::StreamDistinct => write!(f, "StreamDistinct"), PlanImpl::Filter => write!(f, "Filter"), PlanImpl::HashJoin => write!(f, "HashJoin"), PlanImpl::NestLoopJoin => write!(f, "NestLoopJoin"), diff --git a/src/planner/operator/table_scan.rs b/src/planner/operator/table_scan.rs index 2d1f8497..2918fc02 100644 --- a/src/planner/operator/table_scan.rs +++ b/src/planner/operator/table_scan.rs @@ -84,6 +84,7 @@ impl TableScanOperator { covered_deserializers: None, cover_mapping: None, sort_elimination_hint: None, + stream_distinct_hint: None, }); } diff --git a/src/storage/mod.rs b/src/storage/mod.rs index a35df93b..6507acc7 100644 --- a/src/storage/mod.rs +++ b/src/storage/mod.rs @@ -130,26 +130,30 @@ pub trait Transaction: Sized { columns.insert(*i, column.clone()); } } - let (inner, deserializers, cover_mapping) = - match (covered_deserializers, cover_mapping_indices) { - (Some(deserializers), mapping) => { - let tuple_len = match &index_meta.value_ty { - LogicalType::Tuple(tys) => tys.len(), - _ => 1, - }; - let cover_mapping = mapping.map(|slots| TupleMapping::new(slots, tuple_len)); + let is_primary_index = matches!(index_meta.ty, IndexType::PrimaryKey { .. }); + let (inner, deserializers, cover_mapping) = match ( + covered_deserializers, + cover_mapping_indices, + is_primary_index, + ) { + (Some(deserializers), mapping, false) => { + let tuple_len = match &index_meta.value_ty { + LogicalType::Tuple(tys) => tys.len(), + _ => 1, + }; + let cover_mapping = mapping.map(|slots| TupleMapping::new(slots, tuple_len)); - ( - IndexImplEnum::Covered(CoveredIndexImpl), - deserializers, - cover_mapping, - ) - } - (None, _) => { - let deserializers = Self::create_deserializers(&columns, table); - (IndexImplEnum::instance(index_meta.ty), deserializers, None) - } - }; + ( + IndexImplEnum::Covered(CoveredIndexImpl), + deserializers, + cover_mapping, + ) + } + _ => { + let deserializers = Self::create_deserializers(&columns, table); + (IndexImplEnum::instance(index_meta.ty), deserializers, None) + } + }; Ok(IndexIter { offset, @@ -1320,7 +1324,6 @@ impl Iter for IndexIter<'_, T> { let mut encode_max = bound_encode(max, true)?; check_bound(&mut encode_max, bound_max); - let iter = self.params.tx.range(encode_min, encode_max)?; self.state = IndexIterState::Range(iter); } diff --git a/src/storage/rocksdb.rs b/src/storage/rocksdb.rs index 17ca1547..d979888c 100644 --- a/src/storage/rocksdb.rs +++ b/src/storage/rocksdb.rs @@ -598,6 +598,37 @@ mod test { assert_eq!(tuples[0].pk, None); assert_eq!(tuples[0].values, vec![covered_value]); + // primary key index should ignore covered-deserializer hint and still return rows + let pk_index = table + .indexes + .iter() + .find(|index| index.name == "pk_index") + .unwrap() + .clone(); + let mut pk_columns = BTreeMap::new(); + pk_columns.insert(0, a_cover_column.clone()); + let pk_deserializers = vec![a_cover_column.datatype().serializable()]; + let mut iter = transaction.read_by_index( + kite_sql.state.table_cache(), + "t1".to_string().into(), + (None, None), + pk_columns, + pk_index, + vec![Range::Scope { + min: Bound::Unbounded, + max: Bound::Unbounded, + }], + false, + Some(pk_deserializers), + Some(vec![0]), + )?; + let mut row_count = 0; + while let Some(tuple) = iter.next_tuple()? { + assert_eq!(tuple.values.len(), 1); + row_count += 1; + } + assert_eq!(row_count, 3); + Ok(()) } } diff --git a/src/storage/table_codec.rs b/src/storage/table_codec.rs index a1ef5745..1ac359f6 100644 --- a/src/storage/table_codec.rs +++ b/src/storage/table_codec.rs @@ -103,6 +103,10 @@ impl TableCodec { | LogicalType::UBigint | LogicalType::Char(..) | LogicalType::Varchar(..) + | LogicalType::Date + | LogicalType::DateTime + | LogicalType::Time(..) + | LogicalType::TimeStamp(..) ) { return Err(DatabaseError::InvalidType); } diff --git a/src/types/index.rs b/src/types/index.rs index 78b81ded..b009aaeb 100644 --- a/src/types/index.rs +++ b/src/types/index.rs @@ -46,6 +46,7 @@ pub struct IndexInfo { pub(crate) covered_deserializers: Option>, pub(crate) cover_mapping: Option>, pub(crate) sort_elimination_hint: Option, + pub(crate) stream_distinct_hint: Option, } #[derive(Debug, Clone, Eq, PartialEq, Hash, ReferenceSerialization)] diff --git a/src/types/value.rs b/src/types/value.rs index a4a6bf06..0753ecea 100644 --- a/src/types/value.rs +++ b/src/types/value.rs @@ -422,6 +422,14 @@ impl DataValue { } } + pub fn new_utf8(string: String) -> Self { + DataValue::Utf8 { + ty: Utf8Type::Fixed(string.len() as u32), + value: string, + unit: CharLengthUnits::Characters, + } + } + pub fn date(&self) -> Option { if let DataValue::Date32(val) = self { NaiveDate::from_num_days_from_ce_opt(*val) diff --git a/tests/slt/stream_distinct.slt b/tests/slt/stream_distinct.slt new file mode 100644 index 00000000..54def2c8 --- /dev/null +++ b/tests/slt/stream_distinct.slt @@ -0,0 +1,45 @@ +statement ok +create table distinct_t(id int primary key, c1 int, c2 int); + +statement ok +copy distinct_t from 'tests/data/distinct_rows.csv' ( DELIMITER '|' ); + +statement ok +create index distinct_t_c1_index on distinct_t (c1); + +statement ok +analyze table distinct_t; + +# stream distinct +query I rowsort +select distinct c1 from distinct_t where c1 < 10 and c1 > 0; +---- +1 +2 +3 +4 +5 +6 +7 +8 +9 + +statement ok +drop index distinct_t.distinct_t_c1_index; + +# hash distinct +query I rowsort +select distinct c1 from distinct_t where c1 < 10 and c1 > 0; +---- +1 +2 +3 +4 +5 +6 +7 +8 +9 + +statement ok +drop table distinct_t; diff --git a/tests/slt/where_by_index.slt b/tests/slt/where_by_index.slt index 4b95c750..a960bd9b 100644 --- a/tests/slt/where_by_index.slt +++ b/tests/slt/where_by_index.slt @@ -19,7 +19,7 @@ create index p_index on t1 (c1, c2); statement ok analyze table t1; -query IIT +query IIT select * from t1 limit 10; ---- 0 1 2 @@ -198,6 +198,7 @@ select * from t1 where c2 > 0 and c2 < 10; 0 1 9 6 7 8 + # unique covered query I rowsort select c1 from t1 where c1 < 10; @@ -222,6 +223,19 @@ select c2 from t1 where c2 < 10 and c2 > 0; 8 9 +statement ok +insert into t1 values(100000002, 100000002, 8); + +# stream distinct +query I rowsort +select distinct c2 from t1 where c2 < 10 and c2 > 0; +---- +8 +9 + +statement ok +delete from t1 where id = 100000002; + statement ok drop index t1.c2_index; diff --git a/tests/sqllogictest/src/main.rs b/tests/sqllogictest/src/main.rs index 39528205..fcfce867 100644 --- a/tests/sqllogictest/src/main.rs +++ b/tests/sqllogictest/src/main.rs @@ -38,6 +38,7 @@ fn main() { println!("KiteSQL Test Start!\n"); init_20000_row_csv().expect("failed to init csv"); + init_distinct_rows_csv().expect("failed to init distinct csv"); let mut file_num = 0; let start = Instant::now(); @@ -87,3 +88,22 @@ fn init_20000_row_csv() -> io::Result<()> { Ok(()) } + +fn init_distinct_rows_csv() -> io::Result<()> { + let path = "tests/data/distinct_rows.csv"; + + if !Path::new(path).exists() { + let mut file = File::create(path)?; + let rows = 200_000usize; + let distinct = 20usize; + + for i in 0..rows { + let id = i as i32; + let c1 = (i % distinct) as i32; + let c2 = ((i * 7) % 1000) as i32; + writeln!(file, "{}|{}|{}", id, c1, c2)?; + } + } + + Ok(()) +} diff --git a/tpcc/Cargo.toml b/tpcc/Cargo.toml index a980ec78..5d219f84 100644 --- a/tpcc/Cargo.toml +++ b/tpcc/Cargo.toml @@ -11,4 +11,6 @@ indicatif = { version = "0.17" } ordered-float = { version = "4" } rand = { version = "0.8" } rust_decimal = { version = "1" } -thiserror = { version = "1" } \ No newline at end of file +thiserror = { version = "1" } +sqlite = { version = "0.34" } +sqlparser = { version = "0.34" } diff --git a/tpcc/README.md b/tpcc/README.md index 1de8d175..00348317 100644 --- a/tpcc/README.md +++ b/tpcc/README.md @@ -1,88 +1,102 @@ # TPCC on KiteSQL -run `cargo run -p tpcc --release` to run tpcc +Run `make tpcc` (or `cargo run -p tpcc --release`) to exercise the workload on KiteSQL's native storage. + +Run `make tpcc-dual` to execute the workload on KiteSQL while mirroring every statement to an in-memory SQLite database; the runner asserts that both engines return identical tuples, making it ideal for correctness validation. This target runs for 60 seconds (`--measure-time 60`). Use `cargo run -p tpcc --release -- --backend dual --measure-time ` for a custom duration. - i9-13900HX - 32.0 GB - KIOXIA-EXCERIA PLUS G3 SSD - Tips: Pass `--threads ` to run multiple worker threads (default: 8) ```shell -|New-Order| sc: 445996 lt: 0 fl: 4649 -|Payment| sc: 445972 lt: 0 fl: 0 -|Order-Status| sc: 44597 lt: 0 fl: 622 -|Delivery| sc: 44597 lt: 0 fl: 0 -|Stock-Level| sc: 44597 lt: 0 fl: 0 -in 720 sec. +Transaction Summary (elapsed 720.0s) ++--------------+---------+------+---------+-------+ +| Transaction | Success | Late | Failure | Total | ++--------------+---------+------+---------+-------+ +| New-Order | 221183 | 0 | 2284 | 223467 | +| Payment | 221160 | 0 | 7346 | 228506 | +| Order-Status | 22116 | 0 | 493 | 22609 | +| Delivery | 22117 | 0 | 0 | 22117 | +| Stock-Level | 22116 | 0 | 0 | 22116 | ++--------------+---------+------+---------+-------+ (all must be [OK]) [transaction percentage] - Payment: 43.0% (>=43.0%) [Ok] - Order-Status: 4.0% (>=4.0%) [Ok] - Delivery: 4.0% (>=4.0%) [Ok] - Stock-Level: 4.0% (>=4.0%) [Ok] -[response time (at least 90%% passed)] - New-Order: 100.0 [OK] - Payment: 100.0 [OK] - Order-Status: 100.0 [OK] - Delivery: 100.0 [OK] - Stock-Level: 100.0 [OK] - New-Order Total: 445996 - Payment Total: 445972 - Order-Status Total: 44597 - Delivery Total: 44597 - Stock-Level Total: 44597 + Payment: 43.5% (>=43.0%) [OK] + Order-Status: 4.3% (>=4.0%) [OK] + Delivery: 4.3% (>=4.0%) [OK] + Stock-Level: 4.3% (>=4.0%) [OK] +[response time (at least 90% passed)] + New-Order: 100.0% [OK] + Payment: 100.0% [OK] + Order-Status: 100.0% [OK] + Delivery: 100.0% [OK] + Stock-Level: 100.0% [OK] 1.New-Order -0.001, 282206 -0.002, 163359 -0.003, 95 -0.004, 6 +0.001, 153654 +0.002, 66775 +0.003, 17 +0.004, 2 +0.006, 1 2.Payment -0.001, 444948 -0.002, 260 +0.001, 220847 +0.002, 11 +0.003, 1 3.Order-Status -0.001, 36721 -0.002, 5254 -0.003, 1110 -0.004, 396 -0.005, 189 -0.006, 62 -0.007, 6 +0.001, 20887 +0.002, 946 +0.003, 123 4.Delivery -0.001, 43260 +0.009, 342 +0.010, 802 +0.011, 862 +0.012, 1017 +0.013, 1309 +0.014, 1514 +0.015, 1647 +0.016, 1788 +0.017, 1797 +0.018, 1792 +0.019, 1747 +0.020, 1533 +0.021, 1333 +0.022, 1179 +0.023, 719 +0.024, 358 +0.025, 153 +0.026, 13 +0.027, 3 +0.029, 2 +0.037, 1 +0.038, 1 +0.039, 1 5.Stock-Level -0.001, 22651 -0.002, 18136 -0.003, 3223 -0.004, 108 -0.005, 5 -0.006, 4 -0.007, 1 +0.001, 10061 +0.002, 7547 +0.003, 1844 +0.004, 25 +0.005, 1 <90th Percentile RT (MaxRT)> - New-Order : 0.002 (0.012) - Payment : 0.001 (0.002) -Order-Status : 0.002 (0.019) - Delivery : 0.001 (0.001) - Stock-Level : 0.002 (0.018) + New-Order : 0.002 (0.006) + Payment : 0.001 (0.019) +Order-Status : 0.001 (0.003) + Delivery : 0.022 (0.038) + Stock-Level : 0.002 (0.005) -37166 Tpmc +18432 Tpmc ``` -## Explain -run `cargo test -p tpcc explain_tpcc -- --ignored` to explain tpcc statements - -Tips: after TPCC loaded tables - ## Refer to -- https://github.com/AgilData/tpcc \ No newline at end of file +- https://github.com/AgilData/tpcc diff --git a/tpcc/src/backend/dual.rs b/tpcc/src/backend/dual.rs new file mode 100644 index 00000000..d308be3b --- /dev/null +++ b/tpcc/src/backend/dual.rs @@ -0,0 +1,292 @@ +// Copyright 2024 KipData/KiteSQL +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::kite::{KiteBackend, KiteTransaction, KiteTxnResult}; +use super::sqlite::{SqliteBackend, SqliteResult, SqliteTransaction}; +use super::{ + BackendControl, BackendTransaction, DbParam, PreparedStatement, QueryResult, SimpleExecutor, + StatementSpec, +}; +use crate::{TpccError, STOCK_LEVEL_DISTINCT_SQL, STOCK_LEVEL_DISTINCT_SQLITE}; +use kite_sql::types::tuple::Tuple; +use kite_sql::types::value::DataValue; +use std::borrow::Cow; +use std::collections::HashMap; + +pub struct DualBackend { + kite: KiteBackend, + sqlite: SqliteBackend, +} + +impl DualBackend { + pub fn new(path: &str) -> Result { + Ok(Self { + kite: KiteBackend::new(path)?, + sqlite: SqliteBackend::new_memory()?, + }) + } +} + +impl BackendControl for DualBackend { + type Transaction<'a> + = DualTransaction<'a> + where + Self: 'a; + + fn prepare_statements( + &self, + specs: &[Vec], + ) -> Result>, TpccError> { + self.kite.prepare_statements(specs) + } + + fn new_transaction(&self) -> Result, TpccError> { + Ok(DualTransaction { + kite: self.kite.new_transaction()?, + sqlite: self.sqlite.new_transaction()?, + }) + } +} + +impl SimpleExecutor for DualBackend { + fn execute_batch(&self, sql: &str) -> Result<(), TpccError> { + self.kite.execute_batch(sql)?; + if let Some(stmt) = normalize_sqlite_sql(sql) { + self.sqlite.execute_batch(&stmt)?; + } + Ok(()) + } +} + +pub struct DualTransaction<'a> { + kite: KiteTransaction<'a>, + sqlite: SqliteTransaction<'a>, +} + +impl<'a> BackendTransaction for DualTransaction<'a> { + fn execute<'b>( + &'b mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError> { + let spec = statement.spec().clone(); + let sql_lower = spec.sql.trim_start().to_ascii_lowercase(); + let kite_iter = self.kite.execute_raw(statement, params)?; + let sqlite_spec = sqlite_statement_spec(&spec); + let sqlite_stmt = PreparedStatement::Sqlite { spec: sqlite_spec }; + let sqlite_iter = self.sqlite.execute_raw(&sqlite_stmt, params)?; + + if sql_lower.starts_with("select") { + if spec.sql == STOCK_LEVEL_DISTINCT_SQL { + // DISTINCT without ORDER BY has undefined ordering; compare as sets. + let kite_rows = collect_all_rows(kite_iter)?; + let sqlite_rows = collect_all_rows(sqlite_iter)?; + compare_unordered_rows(&kite_rows, &sqlite_rows, statement.spec().sql)?; + return Ok(QueryResult::from_dual(DualQueryResult::CompareUnordered( + DualUnorderedResult::new(kite_rows), + ))); + } + Ok(QueryResult::from_dual(DualQueryResult::Compare( + DualResult::new(kite_iter, sqlite_iter, statement.spec().sql), + ))) + } else { + drain_sqlite_iter(sqlite_iter)?; + Ok(QueryResult::from_kite(kite_iter)) + } + } + + fn commit(self) -> Result<(), TpccError> { + self.sqlite.commit()?; + self.kite.commit() + } +} + +pub(crate) enum DualQueryResult<'a> { + Compare(DualResult<'a>), + CompareUnordered(DualUnorderedResult), +} + +impl<'a> Iterator for DualQueryResult<'a> { + type Item = Result; + + fn next(&mut self) -> Option { + match self { + DualQueryResult::Compare(result) => result.next(), + DualQueryResult::CompareUnordered(result) => result.next(), + } + } +} + +pub(crate) struct DualResult<'a> { + kite: KiteTxnResult<'a>, + sqlite: SqliteResult<'a>, + sql: &'static str, +} + +impl<'a> DualResult<'a> { + fn new(kite: KiteTxnResult<'a>, sqlite: SqliteResult<'a>, sql: &'static str) -> Self { + Self { kite, sqlite, sql } + } +} + +impl Iterator for DualResult<'_> { + type Item = Result; + + fn next(&mut self) -> Option { + match self.kite.next() { + Some(kite_row) => { + let sqlite_row = match self.sqlite.next() { + Some(row) => row, + None => { + return Some(Err(TpccError::BackendMismatch(format!( + "SQLite returned fewer rows for SQL: {}", + self.sql + )))) + } + }; + match (kite_row, sqlite_row) { + (Ok(kite_tuple), Ok(sqlite_tuple)) => { + if kite_tuple.values != sqlite_tuple.values { + println!("[Dual] mismatch SQL: {}", self.sql); + println!(" KiteSQL row: {:?}", kite_tuple.values); + println!(" SQLite row: {:?}", sqlite_tuple.values); + return Some(Err(TpccError::BackendMismatch(format!( + "Result mismatch for SQL: {}", + self.sql + )))); + } + Some(Ok(kite_tuple)) + } + (Err(err), _) => Some(Err(err)), + (_, Err(err)) => Some(Err(err)), + } + } + None => { + if let Some(extra) = self.sqlite.next() { + let err = extra.err().unwrap_or_else(|| { + TpccError::BackendMismatch(format!( + "SQLite returned extra rows for SQL: {}", + self.sql + )) + }); + return Some(Err(err)); + } + None + } + } + } +} + +pub(crate) struct DualUnorderedResult { + rows: std::vec::IntoIter, +} + +impl DualUnorderedResult { + fn new(rows: Vec) -> Self { + Self { + rows: rows.into_iter(), + } + } +} + +impl Iterator for DualUnorderedResult { + type Item = Result; + + fn next(&mut self) -> Option { + self.rows.next().map(Ok) + } +} + +fn normalize_sqlite_sql(sql: &str) -> Option> { + let trimmed = sql.trim(); + let lower = trimmed.to_ascii_lowercase(); + if lower.starts_with("analyze table ") { + None + } else { + Some(Cow::Borrowed(sql)) + } +} + +fn drain_sqlite_iter(mut iter: SqliteResult<'_>) -> Result<(), TpccError> { + while let Some(row) = iter.next() { + row?; + } + Ok(()) +} + +fn collect_all_rows(mut iter: I) -> Result, TpccError> +where + I: Iterator>, +{ + let mut rows = Vec::new(); + while let Some(row) = iter.next() { + rows.push(row?); + } + Ok(rows) +} + +fn compare_unordered_rows( + kite_rows: &[Tuple], + sqlite_rows: &[Tuple], + sql: &'static str, +) -> Result<(), TpccError> { + if kite_rows.len() != sqlite_rows.len() { + return Err(TpccError::BackendMismatch(format!( + "SQLite returned different row count for SQL: {}", + sql + ))); + } + + let mut counts: HashMap, usize> = HashMap::new(); + for row in kite_rows { + *counts.entry(row.values.clone()).or_insert(0) += 1; + } + for row in sqlite_rows { + match counts.get_mut(&row.values) { + Some(count) => { + if *count == 1 { + counts.remove(&row.values); + } else { + *count -= 1; + } + } + None => { + return Err(TpccError::BackendMismatch(format!( + "SQLite returned different distinct set for SQL: {}", + sql + ))); + } + } + } + + if counts.is_empty() { + Ok(()) + } else { + Err(TpccError::BackendMismatch(format!( + "SQLite returned different distinct set for SQL: {}", + sql + ))) + } +} + +fn sqlite_statement_spec(spec: &StatementSpec) -> StatementSpec { + if spec.sql == STOCK_LEVEL_DISTINCT_SQL { + StatementSpec { + sql: STOCK_LEVEL_DISTINCT_SQLITE, + result_types: spec.result_types, + } + } else { + spec.clone() + } +} diff --git a/tpcc/src/backend/kite.rs b/tpcc/src/backend/kite.rs new file mode 100644 index 00000000..eaba24d0 --- /dev/null +++ b/tpcc/src/backend/kite.rs @@ -0,0 +1,127 @@ +// Copyright 2024 KipData/KiteSQL +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::{ + BackendControl, BackendTransaction, DbParam, PreparedStatement, QueryResult, SimpleExecutor, + StatementSpec, +}; +use crate::TpccError; +use kite_sql::db::{DBTransaction, DataBaseBuilder, Database, ResultIter, TransactionIter}; +use kite_sql::storage::rocksdb::RocksStorage; +use kite_sql::types::tuple::Tuple; + +pub struct KiteBackend { + database: Database, +} + +impl KiteBackend { + pub fn new(path: &str) -> Result { + Ok(Self { + database: DataBaseBuilder::path(path).build()?, + }) + } + + fn prepare_spec_groups( + &self, + specs: &[Vec], + ) -> Result>, TpccError> { + let mut groups = Vec::with_capacity(specs.len()); + for group in specs { + let mut prepared = Vec::with_capacity(group.len()); + for spec in group { + let statement = self.database.prepare(spec.sql)?; + prepared.push(PreparedStatement::Kite { + statement, + spec: spec.clone(), + }); + } + groups.push(prepared); + } + Ok(groups) + } + + fn start_transaction(&self) -> Result, TpccError> { + Ok(KiteTransaction { + inner: self.database.new_transaction()?, + }) + } +} + +impl BackendControl for KiteBackend { + type Transaction<'a> + = KiteTransaction<'a> + where + Self: 'a; + + fn prepare_statements( + &self, + specs: &[Vec], + ) -> Result>, TpccError> { + self.prepare_spec_groups(specs) + } + + fn new_transaction(&self) -> Result, TpccError> { + self.start_transaction() + } +} + +impl SimpleExecutor for KiteBackend { + fn execute_batch(&self, sql: &str) -> Result<(), TpccError> { + self.database.run(sql)?.done()?; + Ok(()) + } +} + +pub struct KiteTransaction<'a> { + inner: DBTransaction<'a, RocksStorage>, +} + +impl<'a> KiteTransaction<'a> { + pub(crate) fn execute_raw<'b>( + &'b mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError> { + let PreparedStatement::Kite { statement, .. } = statement else { + return Err(TpccError::InvalidBackend); + }; + Ok(KiteTxnResult(self.inner.execute(statement, params)?)) + } +} + +impl<'a> BackendTransaction for KiteTransaction<'a> { + fn execute<'b>( + &'b mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError> { + let iter = self.execute_raw(statement, params)?; + Ok(QueryResult::from_kite(iter)) + } + + fn commit(self) -> Result<(), TpccError> { + self.inner.commit()?; + Ok(()) + } +} + +pub struct KiteTxnResult<'a>(TransactionIter<'a>); + +impl Iterator for KiteTxnResult<'_> { + type Item = Result; + + fn next(&mut self) -> Option { + self.0.next().map(|item| item.map_err(TpccError::from)) + } +} diff --git a/tpcc/src/backend/mod.rs b/tpcc/src/backend/mod.rs new file mode 100644 index 00000000..adb74e6b --- /dev/null +++ b/tpcc/src/backend/mod.rs @@ -0,0 +1,154 @@ +// Copyright 2024 KipData/KiteSQL +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod dual; +pub mod kite; +pub mod sqlite; + +use self::dual::DualQueryResult; +use self::kite::KiteTxnResult; +use self::sqlite::SqliteResult; +use crate::TpccError; +use kite_sql::db::Statement; +use kite_sql::types::tuple::Tuple; +use kite_sql::types::value::DataValue; + +pub type DbParam = (&'static str, DataValue); + +pub trait SimpleExecutor { + fn execute_batch(&self, sql: &str) -> Result<(), TpccError>; +} + +pub trait BackendControl: SimpleExecutor { + type Transaction<'a>: BackendTransaction + 'a + where + Self: 'a; + + fn prepare_statements( + &self, + specs: &[Vec], + ) -> Result>, TpccError>; + + fn new_transaction(&self) -> Result, TpccError>; +} + +pub struct QueryResult<'a>(QueryResultKind<'a>); + +enum QueryResultKind<'a> { + Kite(KiteTxnResult<'a>), + Sqlite(SqliteResult<'a>), + Dual(DualQueryResult<'a>), +} + +impl<'a> QueryResult<'a> { + pub(crate) fn from_kite(iter: KiteTxnResult<'a>) -> Self { + Self(QueryResultKind::Kite(iter)) + } + + pub(crate) fn from_sqlite(iter: SqliteResult<'a>) -> Self { + Self(QueryResultKind::Sqlite(iter)) + } + + pub(crate) fn from_dual(iter: DualQueryResult<'a>) -> Self { + Self(QueryResultKind::Dual(iter)) + } +} + +impl<'a> Iterator for QueryResult<'a> { + type Item = Result; + + fn next(&mut self) -> Option { + match &mut self.0 { + QueryResultKind::Kite(iter) => iter.next(), + QueryResultKind::Sqlite(iter) => iter.next(), + QueryResultKind::Dual(iter) => iter.next(), + } + } +} + +pub trait BackendTransaction { + fn execute<'a>( + &'a mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError>; + + fn commit(self) -> Result<(), TpccError>; + + fn execute_drain( + &mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result<(), TpccError> { + let mut iter = self.execute(statement, params)?; + while let Some(row) = iter.next() { + row?; + } + Ok(()) + } +} + +pub trait TransactionExt: BackendTransaction { + fn query_one( + &mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result { + let mut iter = self.execute(statement, params)?; + match iter.next() { + Some(row) => row, + None => Err(TpccError::EmptyTuples), + } + } +} + +impl TransactionExt for T {} + +#[derive(Clone, Copy)] +pub enum ColumnType { + Int8, + Int16, + Int32, + Int64, + Decimal, + Utf8, + DateTime, + NullableDateTime, +} + +#[derive(Clone)] +pub struct StatementSpec { + pub sql: &'static str, + pub result_types: &'static [ColumnType], +} + +#[derive(Clone)] +pub enum PreparedStatement { + Kite { + statement: Statement, + spec: StatementSpec, + }, + Sqlite { + spec: StatementSpec, + }, +} + +impl PreparedStatement { + pub fn spec(&self) -> &StatementSpec { + match self { + PreparedStatement::Kite { spec, .. } => spec, + PreparedStatement::Sqlite { spec } => spec, + } + } +} diff --git a/tpcc/src/backend/sqlite.rs b/tpcc/src/backend/sqlite.rs new file mode 100644 index 00000000..f3239ea4 --- /dev/null +++ b/tpcc/src/backend/sqlite.rs @@ -0,0 +1,257 @@ +// Copyright 2024 KipData/KiteSQL +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::{ + BackendControl, BackendTransaction, ColumnType, DbParam, PreparedStatement, QueryResult, + SimpleExecutor, StatementSpec, +}; +use crate::TpccError; +use chrono::{NaiveDateTime, TimeZone, Utc}; +use kite_sql::types::tuple::Tuple; +use kite_sql::types::value::{DataValue, Utf8Type}; +use rust_decimal::Decimal; +use sqlite::{Connection, CursorWithOwnership, Row, Statement as SqliteStatement, Value}; +use sqlparser::ast::CharLengthUnits; + +pub struct SqliteBackend { + connection: Connection, +} + +impl SqliteBackend { + #[allow(dead_code)] + pub fn new(path: &str) -> Result { + Ok(Self { + connection: Connection::open(path)?, + }) + } + + pub fn new_memory() -> Result { + Ok(Self { + connection: Connection::open(":memory:")?, + }) + } + + fn prepare_spec_groups( + &self, + specs: &[Vec], + ) -> Result>, TpccError> { + Ok(specs + .iter() + .map(|group| { + group + .iter() + .cloned() + .map(|spec| PreparedStatement::Sqlite { spec }) + .collect() + }) + .collect()) + } + + fn start_transaction(&self) -> Result, TpccError> { + self.connection.execute("BEGIN IMMEDIATE")?; + Ok(SqliteTransaction { + connection: &self.connection, + finished: false, + }) + } +} + +impl BackendControl for SqliteBackend { + type Transaction<'a> + = SqliteTransaction<'a> + where + Self: 'a; + + fn prepare_statements( + &self, + specs: &[Vec], + ) -> Result>, TpccError> { + self.prepare_spec_groups(specs) + } + + fn new_transaction(&self) -> Result, TpccError> { + self.start_transaction() + } +} + +impl SimpleExecutor for SqliteBackend { + fn execute_batch(&self, sql: &str) -> Result<(), TpccError> { + self.connection.execute(sql)?; + Ok(()) + } +} + +pub struct SqliteTransaction<'a> { + connection: &'a Connection, + finished: bool, +} + +impl<'a> SqliteTransaction<'a> { + pub(crate) fn execute_raw<'b>( + &'b mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError> { + let PreparedStatement::Sqlite { spec } = statement else { + return Err(TpccError::InvalidBackend); + }; + let mut stmt = self.connection.prepare(spec.sql)?; + bind_params(&mut stmt, params)?; + SqliteResult::new(stmt.into_iter(), spec.result_types) + } +} + +impl Drop for SqliteTransaction<'_> { + fn drop(&mut self) { + if !self.finished { + let _ = self.connection.execute("ROLLBACK"); + } + } +} + +impl<'a> BackendTransaction for SqliteTransaction<'a> { + fn execute<'b>( + &'b mut self, + statement: &PreparedStatement, + params: &[DbParam], + ) -> Result, TpccError> { + let iter = self.execute_raw(statement, params)?; + Ok(QueryResult::from_sqlite(iter)) + } + + fn commit(mut self) -> Result<(), TpccError> { + self.connection.execute("COMMIT")?; + self.finished = true; + Ok(()) + } +} + +fn bind_params(statement: &mut SqliteStatement<'_>, params: &[DbParam]) -> Result<(), TpccError> { + for (key, value) in params { + let sqlite_value = convert_value(value)?; + if let Some(index) = key.strip_prefix('?') { + let idx: usize = index.parse().map_err(|_| TpccError::InvalidParameter)?; + statement.bind((idx, sqlite_value.clone()))?; + } else { + statement.bind((key.as_ref(), sqlite_value.clone()))?; + } + } + Ok(()) +} + +fn convert_value(value: &DataValue) -> Result { + Ok(match value { + DataValue::Null => Value::Null, + DataValue::Boolean(v) => Value::Integer(*v as i64), + DataValue::Float32(v) => Value::Float(v.0 as f64), + DataValue::Float64(v) => Value::Float(v.0), + DataValue::Int8(v) => Value::Integer(*v as i64), + DataValue::Int16(v) => Value::Integer(*v as i64), + DataValue::Int32(v) => Value::Integer(*v as i64), + DataValue::Int64(v) => Value::Integer(*v), + DataValue::UInt8(v) => Value::Integer(*v as i64), + DataValue::UInt16(v) => Value::Integer(*v as i64), + DataValue::UInt32(v) => Value::Integer(*v as i64), + DataValue::UInt64(v) => Value::Integer(*v as i64), + DataValue::Utf8 { value, .. } => Value::String(value.clone()), + DataValue::Date32(v) => Value::Integer(*v as i64), + DataValue::Date64(v) => Value::String( + Utc.timestamp_opt(*v, 0) + .single() + .ok_or(TpccError::InvalidDateTime)? + .format("%Y-%m-%d %H:%M:%S") + .to_string(), + ), + DataValue::Time32(_, _) | DataValue::Time64(_, _, _) => Value::Null, + DataValue::Decimal(v) => Value::String(v.to_string()), + DataValue::Tuple(_, _) => Value::Null, + }) +} + +pub struct SqliteResult<'a> { + cursor: CursorWithOwnership<'a>, + column_types: &'static [ColumnType], +} + +impl<'a> SqliteResult<'a> { + fn new( + cursor: CursorWithOwnership<'a>, + column_types: &'static [ColumnType], + ) -> Result { + Ok(Self { + cursor, + column_types, + }) + } +} + +impl Iterator for SqliteResult<'_> { + type Item = Result; + + fn next(&mut self) -> Option { + let row = match self.cursor.next()? { + Ok(row) => row, + Err(err) => return Some(Err(TpccError::Sqlite(err))), + }; + + Some(convert_row(row, self.column_types)) + } +} + +fn convert_row(row: Row, types: &[ColumnType]) -> Result { + let mut values = Vec::with_capacity(types.len()); + for (idx, column_type) in types.iter().enumerate() { + let value = match column_type { + ColumnType::Int8 => DataValue::Int8(row.try_read::(idx)? as i8), + ColumnType::Int16 => DataValue::Int16(row.try_read::(idx)? as i16), + ColumnType::Int32 => DataValue::Int32(row.try_read::(idx)? as i32), + ColumnType::Int64 => DataValue::Int64(row.try_read::(idx)?), + ColumnType::Decimal => DataValue::Decimal(read_decimal(&row, idx)?), + ColumnType::Utf8 => DataValue::Utf8 { + value: row.try_read::<&str, _>(idx)?.to_string(), + ty: Utf8Type::Variable(None), + unit: CharLengthUnits::Characters, + }, + ColumnType::DateTime => { + let text: &str = row.try_read(idx)?; + parse_datetime(text)? + } + ColumnType::NullableDateTime => { + let text: Option<&str> = row.try_read(idx)?; + match text { + Some(value) => parse_datetime(value)?, + None => DataValue::Null, + } + } + }; + values.push(value); + } + Ok(Tuple::new(None, values)) +} + +fn parse_datetime(text: &str) -> Result { + let dt = NaiveDateTime::parse_from_str(text, "%Y-%m-%d %H:%M:%S")?; + Ok(DataValue::from(&dt)) +} + +fn read_decimal(row: &Row, idx: usize) -> Result { + if let Ok(text) = row.try_read::<&str, _>(idx) { + return Ok(Decimal::from_str_exact(text)?); + } + if let Ok(value) = row.try_read::(idx) { + return Ok(Decimal::from_str_exact(&value.to_string())?); + } + let value: i64 = row.try_read(idx)?; + Ok(Decimal::from_str_exact(&value.to_string())?) +} diff --git a/tpcc/src/delivery.rs b/tpcc/src/delivery.rs index a5f33cec..a752168c 100644 --- a/tpcc/src/delivery.rs +++ b/tpcc/src/delivery.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::{BackendTransaction, PreparedStatement, TransactionExt}; use crate::load::DIST_PER_WARE; use crate::{TpccArgs, TpccError, TpccTest, TpccTransaction}; use chrono::Utc; -use kite_sql::db::{DBTransaction, ResultIter, Statement}; -use kite_sql::storage::Storage; use kite_sql::types::value::DataValue; use rand::prelude::ThreadRng; use rand::Rng; @@ -36,58 +35,51 @@ impl DeliveryArgs { pub(crate) struct Delivery; pub(crate) struct DeliveryTest; -impl TpccTransaction for Delivery { +impl TpccTransaction for Delivery { type Args = DeliveryArgs; fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let now = Utc::now().naive_utc(); for d_id in 1..DIST_PER_WARE + 1 { // "SELECT COALESCE(MIN(no_o_id),0) FROM new_orders WHERE no_d_id = ? AND no_w_id = ?" - let tuple = tx - .execute( - &statements[0], - &[ - ("?1", DataValue::Int8(d_id as i8)), - ("?2", DataValue::Int16(args.w_id as i16)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[0], + &[ + ("?1", DataValue::Int8(d_id as i8)), + ("?2", DataValue::Int16(args.w_id as i16)), + ], + )?; let no_o_id = tuple.values[0].i32().unwrap(); if no_o_id == 0 { continue; } // "DELETE FROM new_orders WHERE no_o_id = ? AND no_d_id = ? AND no_w_id = ?" - tx.execute( + tx.execute_drain( &statements[1], &[ ("?1", DataValue::Int32(no_o_id)), ("?2", DataValue::Int8(d_id as i8)), ("?3", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; // "SELECT o_c_id FROM orders WHERE o_id = ? AND o_d_id = ? AND o_w_id = ?" - let tuple = tx - .execute( - &statements[2], - &[ - ("?1", DataValue::Int32(no_o_id)), - ("?2", DataValue::Int8(d_id as i8)), - ("?3", DataValue::Int16(args.w_id as i16)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[2], + &[ + ("?1", DataValue::Int32(no_o_id)), + ("?2", DataValue::Int8(d_id as i8)), + ("?3", DataValue::Int16(args.w_id as i16)), + ], + )?; let c_id = tuple.values[0].i32().unwrap(); // "UPDATE orders SET o_carrier_id = ? WHERE o_id = ? AND o_d_id = ? AND o_w_id = ?" - tx.execute( + tx.execute_drain( &statements[3], &[ ("?1", DataValue::Int8(args.o_carrier_id as i8)), @@ -95,10 +87,9 @@ impl TpccTransaction for Delivery { ("?3", DataValue::Int8(d_id as i8)), ("?4", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; // "UPDATE order_line SET ol_delivery_d = ? WHERE ol_o_id = ? AND ol_d_id = ? AND ol_w_id = ?" - tx.execute( + tx.execute_drain( &statements[4], &[ ("?1", DataValue::from(&now)), @@ -106,23 +97,19 @@ impl TpccTransaction for Delivery { ("?3", DataValue::Int8(d_id as i8)), ("?4", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; // "SELECT SUM(ol_amount) FROM order_line WHERE ol_o_id = ? AND ol_d_id = ? AND ol_w_id = ?" - let tuple = tx - .execute( - &statements[5], - &[ - ("?1", DataValue::Int32(no_o_id)), - ("?2", DataValue::Int8(d_id as i8)), - ("?3", DataValue::Int16(args.w_id as i16)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[5], + &[ + ("?1", DataValue::Int32(no_o_id)), + ("?2", DataValue::Int8(d_id as i8)), + ("?3", DataValue::Int16(args.w_id as i16)), + ], + )?; let ol_total = tuple.values[0].decimal().unwrap(); // "UPDATE customer SET c_balance = c_balance + ? , c_delivery_cnt = c_delivery_cnt + 1 WHERE c_id = ? AND c_d_id = ? AND c_w_id = ?" - tx.execute( + tx.execute_drain( &statements[6], &[ ("?1", DataValue::Decimal(ol_total)), @@ -130,15 +117,14 @@ impl TpccTransaction for Delivery { ("?3", DataValue::Int8(d_id as i8)), ("?4", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; } Ok(()) } } -impl TpccTest for DeliveryTest { +impl TpccTest for DeliveryTest { fn name(&self) -> &'static str { "Delivery" } @@ -146,10 +132,10 @@ impl TpccTest for DeliveryTest { fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, _: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let w_id = rng.gen_range(0..num_ware) + 1; let o_carrier_id = rng.gen_range(1..10); diff --git a/tpcc/src/load.rs b/tpcc/src/load.rs index b76aef32..2048213d 100644 --- a/tpcc/src/load.rs +++ b/tpcc/src/load.rs @@ -12,15 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::SimpleExecutor; use crate::TpccError; use chrono::Utc; use indicatif::{ProgressBar, ProgressStyle}; -use kite_sql::db::{Database, ResultIter}; -use kite_sql::storage::Storage; +use kite_sql::errors::DatabaseError; use rand::rngs::ThreadRng; use rand::Rng; use rust_decimal::Decimal; -use std::marker::PhantomData; use std::ops::Add; // https://github.com/AgilData/tpcc/blob/master/src/main/java/com/codefutures/tpcc/Load.java @@ -31,6 +30,24 @@ pub(crate) const ORD_PER_DIST: usize = 3000; pub(crate) static MAX_NUM_ITEMS: usize = 15; +fn finish_progress(pb: &ProgressBar, label: Option<&str>) { + let elapsed = pb.elapsed(); + pb.finish_and_clear(); + if let Some(name) = label { + println!("[{name}] completed in {:.2}s", elapsed.as_secs_f32()); + } +} + +fn log_phase(task: &str, current: usize, total: usize, context: &str) { + if total == 0 { + println!("[{task}] {context}"); + } else if context.is_empty() { + println!("[{task} {}/{}]", current, total); + } else { + println!("[{task} {}/{}] {}", current, total, context); + } +} + fn generate_string(rng: &mut ThreadRng, min: usize, max: usize) -> String { let chars: Vec = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789" .chars() @@ -87,11 +104,9 @@ fn init_permutation(rng: &mut ThreadRng) -> [usize; CUST_PER_DIST] { nums } -pub struct Load { - phantom: PhantomData, -} +pub struct Load; -impl Load { +impl Load { /// table: item /// /// i_id int not null @@ -101,9 +116,9 @@ impl Load { /// i_data varchar(50) /// /// primary key (i_id) - pub fn load_items(rng: &mut ThreadRng, db: &Database) -> Result<(), TpccError> { - db.run("drop table if exists item;")?.done()?; - db.run( + pub fn load_items(rng: &mut ThreadRng, exec: &impl SimpleExecutor) -> Result<(), TpccError> { + exec.execute_batch("drop table if exists item;")?; + exec.execute_batch( "create table item ( i_id int not null, i_im_id int, @@ -111,8 +126,7 @@ impl Load { i_price decimal(5,2), i_data varchar(50), PRIMARY KEY(i_id) );", - )? - .done()?; + )?; let pb = ProgressBar::new(MAX_ITEMS as u64); pb.set_style( ProgressStyle::default_bar() @@ -138,15 +152,14 @@ impl Load { i_data = format!("{}original{}", prefix, remainder); } - db.run(format!( + exec.execute_batch(&format!( "insert into item values ({i_id}, {i_im_id}, '{i_name}', {i_price}, '{i_data}')" - ))? - .done()?; + ))?; pb.set_position(i_id as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, Some("Items loaded")); println!("[Analyze Table: item]"); - db.run("analyze table item")?.done()?; + exec.execute_batch("analyze table item")?; Ok(()) } @@ -165,11 +178,11 @@ impl Load { /// primary key (w_id) pub fn load_warehouses( rng: &mut ThreadRng, - db: &Database, + exec: &impl SimpleExecutor, num_ware: usize, ) -> Result<(), TpccError> { - db.run("drop table if exists warehouse;")?.done()?; - db.run( + exec.execute_batch("drop table if exists warehouse;")?; + exec.execute_batch( "create table warehouse ( w_id smallint not null, w_name varchar(10), @@ -181,10 +194,9 @@ impl Load { w_tax decimal(4,2), w_ytd decimal(12,2), PRIMARY KEY(w_id) );", - )? - .done()?; - db.run("drop table if exists stock;")?.done()?; - db.run( + )?; + exec.execute_batch("drop table if exists stock;")?; + exec.execute_batch( "create table stock ( s_i_id int not null, s_w_id smallint not null, @@ -204,12 +216,10 @@ impl Load { s_remote_cnt smallint, s_data varchar(50), PRIMARY KEY(s_w_id, s_i_id) );", - )? - .done()?; - db.run("CREATE INDEX fkey_stock_2 ON stock (s_i_id);")? - .done()?; - db.run("drop table if exists district;")?.done()?; - db.run( + )?; + exec.execute_batch("CREATE INDEX fkey_stock_2 ON stock (s_i_id);")?; + exec.execute_batch("drop table if exists district;")?; + exec.execute_batch( "create table district ( d_id tinyint not null, d_w_id smallint not null, @@ -223,8 +233,7 @@ impl Load { d_ytd decimal(12,2), d_next_o_id int, primary key (d_w_id, d_id) );", - )? - .done()?; + )?; let pb = ProgressBar::new(num_ware as u64); pb.set_style( ProgressStyle::default_bar() @@ -234,6 +243,7 @@ impl Load { .unwrap(), ); for w_id in 1..num_ware + 1 { + log_phase("Warehouses", w_id, num_ware, ""); let w_name = generate_string(rng, 6, 10); let w_street_1 = generate_string(rng, 10, 20); let w_street_2 = generate_string(rng, 10, 20); @@ -246,29 +256,34 @@ impl Load { .round_dp(2); let w_ytd = Decimal::from_f64_retain(3000000.00).unwrap().round_dp(2); - db.run(format!( + exec.execute_batch(&format!( "insert into warehouse values({}, '{}', '{}', '{}', '{}', '{}', '{}', {}, {})", w_id, w_name, w_street_1, w_street_2, w_city, w_state, w_zip, w_tax, w_ytd, - ))? - .done()?; - Self::stock(rng, db, w_id)?; - Self::district(rng, db, w_id)?; + ))?; + Self::stock(rng, exec, w_id)?; + Self::district(rng, exec, w_id)?; pb.set_position(w_id as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, Some("Warehouses loaded")); println!("[Analyze Table: stock]"); - db.run("analyze table stock")?.done()?; + exec.execute_batch("analyze table stock")?; + println!("[Analyze Table: district]"); + match exec.execute_batch("analyze table district") { + Ok(_) | Err(TpccError::Database(DatabaseError::TooManyBuckets(..))) => (), + err => return err, + }; + Ok(()) } pub fn load_custs( rng: &mut ThreadRng, - db: &Database, + exec: &impl SimpleExecutor, num_ware: usize, ) -> Result<(), TpccError> { - db.run("drop table if exists customer;")?.done()?; - db.run( + exec.execute_batch("drop table if exists customer;")?; + exec.execute_batch( "create table customer ( c_id int not null, c_d_id tinyint not null, @@ -292,12 +307,12 @@ impl Load { c_delivery_cnt smallint, c_data text, PRIMARY KEY(c_w_id, c_d_id, c_id) );", - )? - .done()?; - db.run("CREATE INDEX idx_customer ON customer (c_w_id,c_d_id,c_last,c_first);")? - .done()?; - db.run("drop table if exists history;")?.done()?; - db.run( + )?; + exec.execute_batch( + "CREATE INDEX idx_customer ON customer (c_w_id,c_d_id,c_last,c_first);", + )?; + exec.execute_batch("drop table if exists history;")?; + exec.execute_batch( "create table history ( h_c_id int, h_c_d_id tinyint, @@ -307,27 +322,32 @@ impl Load { h_date datetime, h_amount decimal(6,2), h_data varchar(24), - PRIMARY KEY(h_c_id, h_c_d_id, h_c_w_id, h_d_id, h_w_id) );", - )? - .done()?; + PRIMARY KEY(h_c_id, h_c_d_id, h_c_w_id, h_d_id, h_w_id, h_date) );", + )?; for w_id in 1..num_ware + 1 { for d_id in 1..DIST_PER_WARE + 1 { - Self::load_customers(rng, db, d_id, w_id)?; + log_phase( + "Customers", + d_id, + DIST_PER_WARE, + &format!("Warehouse {}/{}", w_id, num_ware), + ); + Self::load_customers(rng, exec, d_id, w_id)?; } } println!("[Analyze Table: customer]"); - db.run("analyze table customer")?.done()?; + exec.execute_batch("analyze table customer")?; Ok(()) } pub fn load_ord( rng: &mut ThreadRng, - db: &Database, + exec: &impl SimpleExecutor, num_ware: usize, ) -> Result<(), TpccError> { - db.run("drop table if exists orders;")?.done()?; - db.run( + exec.execute_batch("drop table if exists orders;")?; + exec.execute_batch( "create table orders ( o_id int not null, o_d_id tinyint not null, @@ -338,21 +358,18 @@ impl Load { o_ol_cnt tinyint, o_all_local tinyint, PRIMARY KEY(o_w_id, o_d_id, o_id) );", - )? - .done()?; - db.run("CREATE INDEX idx_orders ON orders (o_w_id,o_d_id,o_c_id,o_id);")? - .done()?; - db.run("drop table if exists new_orders;")?.done()?; - db.run( + )?; + exec.execute_batch("CREATE INDEX idx_orders ON orders (o_w_id,o_d_id,o_c_id,o_id);")?; + exec.execute_batch("drop table if exists new_orders;")?; + exec.execute_batch( "create table new_orders ( no_o_id int not null, no_d_id tinyint not null, no_w_id smallint not null, PRIMARY KEY(no_w_id, no_d_id, no_o_id));", - )? - .done()?; - db.run("drop table if exists order_line;")?.done()?; - db.run( + )?; + exec.execute_batch("drop table if exists order_line;")?; + exec.execute_batch( "create table order_line ( ol_o_id int not null, ol_d_id tinyint not null, @@ -365,21 +382,28 @@ impl Load { ol_amount decimal(6,2), ol_dist_info char(24), PRIMARY KEY(ol_w_id, ol_d_id, ol_o_id, ol_number) );", - )? - .done()?; - db.run("CREATE INDEX fkey_order_line_1 ON order_line (ol_o_id, ol_d_id, ol_w_id);")? - .done()?; - db.run("CREATE INDEX fkey_order_line_2 ON order_line (ol_supply_w_id,ol_i_id);")? - .done()?; + )?; + exec.execute_batch( + "CREATE INDEX fkey_order_line_1 ON order_line (ol_o_id, ol_d_id, ol_w_id);", + )?; + exec.execute_batch( + "CREATE INDEX fkey_order_line_2 ON order_line (ol_supply_w_id, ol_i_id);", + )?; for w_id in 1..num_ware + 1 { for d_id in 1..DIST_PER_WARE + 1 { - Self::load_orders(rng, db, d_id, w_id)?; + log_phase( + "Orders", + d_id, + DIST_PER_WARE, + &format!("Warehouse {}/{}", w_id, num_ware), + ); + Self::load_orders(rng, exec, d_id, w_id)?; } } println!("[Analyze Table: orders & order_line & new_order]"); - db.run("analyze table orders")?.done()?; - db.run("analyze table order_line")?.done()?; - db.run("analyze table new_orders")?.done()?; + exec.execute_batch("analyze table orders")?; + exec.execute_batch("analyze table order_line")?; + exec.execute_batch("analyze table new_orders")?; Ok(()) } @@ -405,7 +429,11 @@ impl Load { /// s_data varchar(50) /// /// primary key(s_w_id, s_i_id) - pub fn stock(rng: &mut ThreadRng, db: &Database, w_id: usize) -> Result<(), TpccError> { + pub fn stock( + rng: &mut ThreadRng, + exec: &impl SimpleExecutor, + w_id: usize, + ) -> Result<(), TpccError> { let pb = ProgressBar::new(MAX_ITEMS as u64); pb.set_style( ProgressStyle::default_bar() @@ -435,7 +463,7 @@ impl Load { } else { generate_string(rng, 26, 50) }; - db.run(format!( + exec.execute_batch(&format!( "insert into stock values({}, {}, {}, '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', {}, {}, {}, '{}')", s_i_id, s_w_id, @@ -454,10 +482,10 @@ impl Load { 0, 0, s_data, - ))?.done()?; + ))?; pb.set_position(s_i_id as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, None); Ok(()) } @@ -496,7 +524,11 @@ impl Load { /// /// /// primary key (d_w_id, d_id) - pub fn district(rng: &mut ThreadRng, db: &Database, w_id: usize) -> Result<(), TpccError> { + pub fn district( + rng: &mut ThreadRng, + exec: &impl SimpleExecutor, + w_id: usize, + ) -> Result<(), TpccError> { let pb = ProgressBar::new(DIST_PER_WARE as u64); pb.set_style( ProgressStyle::default_bar() @@ -521,7 +553,7 @@ impl Load { .unwrap() .round_dp(2); - db.run(format!( + exec.execute_batch(&format!( "insert into district values({}, {}, '{}', '{}', '{}', '{}', '{}', '{}', {}, {}, {})", d_id, d_w_id, @@ -534,10 +566,10 @@ impl Load { d_tax, d_ytd, d_next_o_id, - ))?.done()?; + ))?; pb.set_position(d_id as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, None); Ok(()) } @@ -581,7 +613,7 @@ impl Load { /// h_data varchar(24) pub fn load_customers( rng: &mut ThreadRng, - db: &Database, + exec: &impl SimpleExecutor, d_id: usize, w_id: usize, ) -> Result<(), TpccError> { @@ -629,7 +661,7 @@ impl Load { let c_data = generate_string(rng, 300, 500); - db.run(format!( + exec.execute_batch(&format!( "insert into customer values({}, {}, {}, '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', '{}', {}, {}, {}, {}, {}, {}, '{}')", c_id, c_d_id, @@ -652,20 +684,19 @@ impl Load { c_payment_cnt, c_delivery_cnt, c_data, - ))?.done()?; + ))?; let h_date = &date; let h_amount = Decimal::from_f64_retain(10.0).unwrap().round_dp(2); let h_data = generate_string(rng, 12, 24); - db.run(format!( + exec.execute_batch(&format!( "insert into history values({}, {}, {}, {}, {}, '{}', {}, '{}')", c_id, c_d_id, c_w_id, c_d_id, c_w_id, h_date, h_amount, h_data, - ))? - .done()?; + ))?; pb.set_position(c_id as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, None); Ok(()) } @@ -709,7 +740,7 @@ impl Load { /// primary key(ol_w_id, ol_d_id, ol_o_id, ol_number) pub fn load_orders( rng: &mut ThreadRng, - db: &Database, + exec: &impl SimpleExecutor, d_id: usize, w_id: usize, ) -> Result<(), TpccError> { @@ -734,34 +765,31 @@ impl Load { let date = format!("'{}'", Utc::now().format("%Y-%m-%d %H:%M:%S")); let o_carrier_id = if o_id > 2100 { - db.run(format!( + exec.execute_batch(&format!( "insert into new_orders values({}, {}, {})", o_id, o_d_id, o_w_id, - ))? - .done()?; + ))?; "null".to_string() } else { o_carrier_id.to_string() }; - db.run(format!( + exec.execute_batch(&format!( "insert into orders values({}, {}, {}, {}, {}, {}, {}, {})", o_id, o_d_id, o_w_id, o_c_id, date, o_carrier_id, o_ol_cnt, "1", - ))? - .done()?; + ))?; for ol in 1..o_ol_cnt + 1 { let ol_i_id = rng.gen_range(1..MAX_ITEMS); let ol_supply_w_id = o_w_id; let ol_quantity = 4; - let ol_amount = 0.0; let ol_dist_info = generate_string(rng, 24, 24); let (ol_delivery_d, ol_amount) = if o_id > 2100 { - ("null", ol_amount) + ("null", "0.00".to_string()) } else { - (date.as_str(), rng.gen_range(0.1..100.0)) + (date.as_str(), format!("{:.2}", rng.gen_range(0.1..100.0))) }; - db.run(format!( + exec.execute_batch(&format!( "insert into order_line values({}, {}, {}, {}, {}, {}, {}, {}, {}, '{}')", o_id, o_d_id, @@ -773,12 +801,11 @@ impl Load { ol_quantity, ol_amount, ol_dist_info, - ))? - .done()?; + ))?; } pb.set_position((o_id - 1) as u64); } - pb.finish_with_message("load completed!"); + finish_progress(&pb, None); Ok(()) } diff --git a/tpcc/src/main.rs b/tpcc/src/main.rs index c7aefae1..72c389ed 100644 --- a/tpcc/src/main.rs +++ b/tpcc/src/main.rs @@ -12,6 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::dual::DualBackend; +use crate::backend::kite::KiteBackend; +use crate::backend::{ + BackendControl, BackendTransaction, ColumnType, PreparedStatement, StatementSpec, +}; use crate::delivery::DeliveryTest; use crate::load::Load; use crate::new_ord::NewOrdTest; @@ -20,16 +25,16 @@ use crate::payment::PaymentTest; use crate::rt_hist::RtHist; use crate::slev::SlevTest; use crate::utils::SeqGen; -use clap::Parser; -use kite_sql::db::{DBTransaction, DataBaseBuilder, Statement}; +use clap::{Parser, ValueEnum}; +use indicatif::{ProgressBar, ProgressStyle}; use kite_sql::errors::DatabaseError; -use kite_sql::storage::Storage; use rand::prelude::ThreadRng; use rand::Rng; use std::fs; use std::path::Path; use std::time::{Duration, Instant}; +mod backend; mod delivery; mod load; mod new_ord; @@ -48,27 +53,36 @@ pub(crate) const RT_LIMITS: [Duration; 5] = [ Duration::from_secs(8), Duration::from_secs(2), ]; +const TX_NAMES: [&str; 5] = [ + "New-Order", + "Payment", + "Order-Status", + "Delivery", + "Stock-Level", +]; +pub(crate) const STOCK_LEVEL_DISTINCT_SQL: &str = "SELECT DISTINCT ol_i_id FROM order_line WHERE ol_w_id = ?1 AND ol_d_id = ?2 AND ol_o_id < ?3 AND ol_o_id >= (?4 - 20)"; +pub(crate) const STOCK_LEVEL_DISTINCT_SQLITE: &str = "SELECT DISTINCT ol_i_id FROM (SELECT ol_i_id FROM order_line WHERE ol_w_id = ?1 AND ol_d_id = ?2 AND ol_o_id < ?3 AND ol_o_id >= (?4 - 20) ORDER BY ol_w_id, ol_d_id, ol_o_id)"; -pub(crate) trait TpccTransaction { +pub(crate) trait TpccTransaction { type Args; fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError>; } -pub(crate) trait TpccTest { +pub(crate) trait TpccTest { fn name(&self) -> &'static str; fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, args: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError>; } @@ -83,6 +97,8 @@ struct Args { joins: bool, #[clap(long, default_value = "kite_sql_tpcc")] path: String, + #[clap(long, value_enum, default_value = "kite")] + backend: BackendKind, #[clap(long, default_value = "5")] max_retry: usize, #[clap(long, default_value = "720")] @@ -91,120 +107,58 @@ struct Args { num_ware: usize, } +#[derive(Copy, Clone, Debug, ValueEnum)] +enum BackendKind { + Kite, + Dual, +} + // TODO: Support multi-threaded TPCC fn main() -> Result<(), TpccError> { let args = Args::parse(); - let db_path = Path::new(&args.path); - if db_path.exists() { - fs::remove_dir_all(db_path)?; + let mut rng = rand::thread_rng(); + + match args.backend { + BackendKind::Kite => { + let db_path = Path::new(&args.path); + if db_path.exists() { + fs::remove_dir_all(db_path)?; + } + let backend = KiteBackend::new(&args.path)?; + run_tpcc(&backend, &args, &mut rng)?; + } + BackendKind::Dual => { + let db_path = Path::new(&args.path); + if db_path.exists() { + fs::remove_dir_all(db_path)?; + } + let backend = DualBackend::new(&args.path)?; + run_tpcc(&backend, &args, &mut rng)?; + } } - let mut rng = rand::thread_rng(); - let database = DataBaseBuilder::path(&args.path).build()?; + Ok(()) +} - Load::load_items(&mut rng, &database)?; - Load::load_warehouses(&mut rng, &database, args.num_ware)?; - Load::load_custs(&mut rng, &database, args.num_ware)?; - Load::load_ord(&mut rng, &database, args.num_ware)?; +fn run_tpcc( + backend: &B, + args: &Args, + rng: &mut ThreadRng, +) -> Result<(), TpccError> { + Load::load_items(rng, backend)?; + Load::load_warehouses(rng, backend, args.num_ware)?; + Load::load_custs(rng, backend, args.num_ware)?; + Load::load_ord(rng, backend, args.num_ware)?; - let test_statements = vec![ - // New-Order: order creation and stock reservation. - vec![ - // Hot join fetching customer credit + warehouse tax. - database.prepare("SELECT c.c_discount, c.c_last, c.c_credit, w.w_tax FROM customer AS c JOIN warehouse AS w ON c.c_w_id = w_id AND w.w_id = ?1 AND c.c_w_id = ?2 AND c.c_d_id = ?3 AND c.c_id = ?4")?, - // PK lookup of customer row. - database.prepare("SELECT c_discount, c_last, c_credit FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3")?, - // Single-row warehouse tax read. - database.prepare("SELECT w_tax FROM warehouse WHERE w_id = ?1")?, - // Next order id lookup; serialized per district. - database.prepare("SELECT d_next_o_id, d_tax FROM district WHERE d_id = ?1 AND d_w_id = ?2")?, - // Hot-row update on d_next_o_id. - database.prepare("UPDATE district SET d_next_o_id = ?1 + 1 WHERE d_id = ?2 AND d_w_id = ?3")?, - // Orders insert touches multiple indexes. - database.prepare("INSERT INTO orders (o_id, o_d_id, o_w_id, o_c_id, o_entry_d, o_ol_cnt, o_all_local) VALUES(?1, ?2, ?3, ?4, ?5, ?6, ?7)")?, - // Append to new_orders queue. - database.prepare("INSERT INTO new_orders (no_o_id, no_d_id, no_w_id) VALUES (?1,?2,?3)")?, - // Random item lookup by PK. - database.prepare("SELECT i_price, i_name, i_data FROM item WHERE i_id = ?1")?, - // Stock fetch per item/warehouse. - database.prepare("SELECT s_quantity, s_data, s_dist_01, s_dist_02, s_dist_03, s_dist_04, s_dist_05, s_dist_06, s_dist_07, s_dist_08, s_dist_09, s_dist_10 FROM stock WHERE s_i_id = ?1 AND s_w_id = ?2")?, - // Stock decrement; contention point. - database.prepare("UPDATE stock SET s_quantity = ?1 WHERE s_i_id = ?2 AND s_w_id = ?3")?, - // Order_line insert per item. - database.prepare("INSERT INTO order_line (ol_o_id, ol_d_id, ol_w_id, ol_number, ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_dist_info) VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7, ?8, ?9)")?, - ], - // Payment: apply payment and write history row. - vec![ - // Warehouse w_ytd counter update. - database.prepare("UPDATE warehouse SET w_ytd = w_ytd + ?1 WHERE w_id = ?2")?, - // Warehouse address projection. - database.prepare("SELECT w_street_1, w_street_2, w_city, w_state, w_zip, w_name FROM warehouse WHERE w_id = ?1")?, - // District d_ytd counter update. - database.prepare("UPDATE district SET d_ytd = d_ytd + ?1 WHERE d_w_id = ?2 AND d_id = ?3")?, - // District address projection. - database.prepare("SELECT d_street_1, d_street_2, d_city, d_state, d_zip, d_name FROM district WHERE d_w_id = ?1 AND d_id = ?2")?, - // COUNT over customers sharing last name (line 129). - database.prepare("SELECT count(c_id) FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3")?, - // ORDER BY c_first to pick correct customer (line 130). - database.prepare("SELECT c_id FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3 ORDER BY c_first")?, - // Full customer row read. - database.prepare("SELECT c_first, c_middle, c_last, c_street_1, c_street_2, c_city, c_state, c_zip, c_phone, c_credit, c_credit_lim, c_discount, c_balance, c_since FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3")?, - // Pull c_data LOB. - database.prepare("SELECT c_data FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3")?, - // Update balance + c_data (text concat). - database.prepare("UPDATE customer SET c_balance = ?1, c_data = ?2 WHERE c_w_id = ?3 AND c_d_id = ?4 AND c_id = ?5")?, - // Balance-only update variant. - database.prepare("UPDATE customer SET c_balance = ?1 WHERE c_w_id = ?2 AND c_d_id = ?3 AND c_id = ?4")?, - // History append writes long text payload. - database.prepare("INSERT OVERWRITE history(h_c_d_id, h_c_w_id, h_c_id, h_d_id, h_w_id, h_date, h_amount, h_data) VALUES(?1, ?2, ?3, ?4, ?5, ?6, ?7, ?8)")?, - ], - // Order-Status: inspect the most recent order of a customer. - vec![ - // COUNT on customers sharing last name (line 138). - database.prepare("SELECT count(c_id) FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3")?, - // ORDER BY c_first to pick median entry (line 139). - database.prepare("SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3 ORDER BY c_first")?, - // Direct customer lookup by id. - database.prepare("SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3")?, - // Correlated MAX(o_id) to find latest order. - database.prepare("SELECT o_id, o_entry_d, COALESCE(o_carrier_id,0) FROM orders WHERE o_w_id = ?1 AND o_d_id = ?2 AND o_c_id = ?3 AND o_id = (SELECT MAX(o_id) FROM orders WHERE o_w_id = ?4 AND o_d_id = ?5 AND o_c_id = ?6)")?, - // Fetch all order_line rows for that order. - database.prepare("SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line WHERE ol_w_id = ?1 AND ol_d_id = ?2 AND ol_o_id = ?3")? - ], - // Delivery: finish the oldest new order per district. - vec![ - // MIN(no_o_id) over new_orders (line 145). - database.prepare("SELECT COALESCE(MIN(no_o_id),0) FROM new_orders WHERE no_d_id = ?1 AND no_w_id = ?2")?, - // Delete that new_orders entry. - database.prepare("DELETE FROM new_orders WHERE no_o_id = ?1 AND no_d_id = ?2 AND no_w_id = ?3")?, - // Lookup the customer id from orders. - database.prepare("SELECT o_c_id FROM orders WHERE o_id = ?1 AND o_d_id = ?2 AND o_w_id = ?3")?, - // Update carrier id on the order. - database.prepare("UPDATE orders SET o_carrier_id = ?1 WHERE o_id = ?2 AND o_d_id = ?3 AND o_w_id = ?4")?, - // Update delivery timestamp on order_line rows. - database.prepare("UPDATE order_line SET ol_delivery_d = ?1 WHERE ol_o_id = ?2 AND ol_d_id = ?3 AND ol_w_id = ?4")?, - // SUM(ol_amount) per order (line 150). - database.prepare("SELECT SUM(ol_amount) FROM order_line WHERE ol_o_id = ?1 AND ol_d_id = ?2 AND ol_w_id = ?3")?, - // Customer balance + delivery_cnt increment. - database.prepare("UPDATE customer SET c_balance = c_balance + ?1 , c_delivery_cnt = c_delivery_cnt + 1 WHERE c_id = ?2 AND c_d_id = ?3 AND c_w_id = ?4")?, - ], - // Stock-Level: detect items that are low stock for recent orders. - vec![ - // District next_o_id read. - database.prepare("SELECT d_next_o_id FROM district WHERE d_id = ?1 AND d_w_id = ?2")?, - // DISTINCT item ids from last 20 orders (line 155). - database.prepare("SELECT DISTINCT ol_i_id FROM order_line WHERE ol_w_id = ?1 AND ol_d_id = ?2 AND ol_o_id < ?3 AND ol_o_id >= (?4 - 20)")?, - // COUNT stock rows where s_quantity falls below threshold (line 156). - database.prepare("SELECT count(*) FROM stock WHERE s_w_id = ?1 AND s_i_id = ?2 AND s_quantity < ?3")?, - ], - ]; + let statement_specs = statement_specs(); + let test_statements = backend.prepare_statements(&statement_specs)?; let mut rt_hist = RtHist::new(); let mut success = [0usize; 5]; let mut late = [0usize; 5]; let mut failure = [0usize; 5]; - let tests = vec![ - Box::new(NewOrdTest) as Box>, + let tests: Vec> = vec![ + Box::new(NewOrdTest), Box::new(PaymentTest), Box::new(OrderStatTest), Box::new(DeliveryTest), @@ -216,6 +170,17 @@ fn main() -> Result<(), TpccError> { let mut round_count = 0; let mut seq_gen = SeqGen::new(10, 10, 1, 1, 1); let tpcc_start = Instant::now(); + let progress = ProgressBar::new_spinner(); + progress.set_style(ProgressStyle::with_template("{spinner:.green} [TPCC] {msg}").unwrap()); + progress.enable_steady_tick(Duration::from_millis(120)); + update_progress_bar( + &progress, + round_count, + &success, + &late, + &failure, + tpcc_start, + ); while tpcc_start.elapsed() < duration { let i = seq_gen.get(); @@ -223,19 +188,16 @@ fn main() -> Result<(), TpccError> { let statement = &test_statements[i]; let mut is_succeed = false; - for j in 0..args.max_retry + 1 { + let mut last_error = None; + for _attempt in 0..=args.max_retry { let transaction_start = Instant::now(); - let mut tx = database.new_transaction()?; + let mut tx = backend.new_transaction()?; if let Err(err) = - tpcc_test.do_transaction(&mut rng, &mut tx, args.num_ware, &tpcc_args, &statement) + tpcc_test.do_transaction(rng, &mut tx, args.num_ware, &tpcc_args, statement) { failure[i] += 1; - eprintln!( - "[{}] Error while doing transaction: {}", - tpcc_test.name(), - err - ); + last_error = Some(err); } else { let rt = transaction_start.elapsed(); rt_hist.hist_inc(i, rt); @@ -249,82 +211,56 @@ fn main() -> Result<(), TpccError> { tx.commit()?; break; } - if j < args.max_retry { - println!("[{}] Retry for the {}th time", tpcc_test.name(), j + 1); - } } if !is_succeed { + if let Some(err) = last_error { + eprintln!( + "[{}] Error after {} retries: {}", + tpcc_test.name(), + args.max_retry, + err + ); + } return Err(TpccError::MaxRetry); } + update_progress_bar( + &progress, + round_count, + &success, + &late, + &failure, + tpcc_start, + ); + if round_count != 0 && round_count % CHECK_POINT_COUNT == 0 { - println!( - "[TPCC CheckPoint {} on round {round_count}][{}]: 90th Percentile RT: {:.3}", + let p90 = rt_hist.hist_ckp(i); + print_checkpoint( round_count / CHECK_POINT_COUNT, + round_count, tpcc_test.name(), - rt_hist.hist_ckp(i) + p90, + &success, + &late, + &failure, + tpcc_start, + &progress, ); } round_count += 1; } + progress.finish_and_clear(); let actual_tpcc_time = tpcc_start.elapsed(); - println!("---------------------------------------------------"); - // Raw Results - print_transaction(&success, &late, &failure, |name, success, late, failure| { - println!("|{}| sc: {} lt: {} fl: {}", name, success, late, failure) - }); - println!("in {} sec.", actual_tpcc_time.as_secs()); - println!(" (all must be [OK])"); - println!("[transaction percentage]"); - - let mut j = 0.0; - for i in 0..5 { - j += (success[i] + late[i]) as f64; - } - // Payment - let f = (((success[1] + late[1]) as f64 / j) * 100.0).round(); - print!(" Payment: {:.1}% (>=43.0%)", f); - if f >= 43.0 { - println!(" [Ok]"); - } else { - println!(" [NG]"); - } - // Order-Status - let f = (((success[2] + late[2]) as f64 / j) * 100.0).round(); - print!(" Order-Status: {:.1}% (>=4.0%)", f); - if f >= 4.0 { - println!(" [Ok]"); - } else { - println!(" [NG]"); - } - // Delivery - let f = (((success[3] + late[3]) as f64 / j) * 100.0).round(); - print!(" Delivery: {:.1}% (>=4.0%)", f); - if f >= 4.0 { - println!(" [Ok]"); - } else { - println!(" [NG]"); - } - // Stock-Level - let f = (((success[4] + late[4]) as f64 / j) * 100.0).round(); - print!(" Stock-Level: {:.1}% (>=4.0%)", f); - if f >= 4.0 { - println!(" [Ok]"); - } else { - println!(" [NG]"); - } - println!("[response time (at least 90%% passed)]"); - print_transaction(&success, &late, &failure, |name, success, late, _| { - let f = (success as f64 / (success + late) as f64) * 100.0; - print!(" {}: {:.1}", name, f); - if f >= 90.0 { - println!(" [OK]"); - } else { - println!(" [NG]"); - } - }); - print_transaction(&success, &late, &failure, |name, success, late, _| { - println!(" {} Total: {}", name, success + late) - }); + update_progress_bar( + &progress, + round_count, + &success, + &late, + &failure, + tpcc_start, + ); + print_summary_table(&success, &late, &failure, actual_tpcc_time); + print_constraint_checks(&success, &late); + print_response_checks(&success, &late); println!(); rt_hist.hist_report(); println!(""); @@ -334,24 +270,365 @@ fn main() -> Result<(), TpccError> { Ok(()) } -fn print_transaction( +fn statement_specs() -> Vec> { + vec![ + vec![ + stmt( + "SELECT c.c_discount, c.c_last, c.c_credit, w.w_tax FROM customer AS c JOIN warehouse AS w ON c.c_w_id = w_id AND w.w_id = ?1 AND c.c_w_id = ?2 AND c.c_d_id = ?3 AND c.c_id = ?4", + &[ColumnType::Decimal, ColumnType::Utf8, ColumnType::Utf8, ColumnType::Decimal], + ), + stmt( + "SELECT c_discount, c_last, c_credit FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3", + &[ColumnType::Decimal, ColumnType::Utf8, ColumnType::Utf8], + ), + stmt( + "SELECT w_tax FROM warehouse WHERE w_id = ?1", + &[ColumnType::Decimal], + ), + stmt( + "SELECT d_next_o_id, d_tax FROM district WHERE d_id = ?1 AND d_w_id = ?2", + &[ColumnType::Int32, ColumnType::Decimal], + ), + stmt( + "UPDATE district SET d_next_o_id = ?1 + 1 WHERE d_id = ?2 AND d_w_id = ?3", + &[], + ), + stmt( + "INSERT INTO orders (o_id, o_d_id, o_w_id, o_c_id, o_entry_d, o_ol_cnt, o_all_local) VALUES(?1, ?2, ?3, ?4, ?5, ?6, ?7)", + &[], + ), + stmt( + "INSERT INTO new_orders (no_o_id, no_d_id, no_w_id) VALUES (?1,?2,?3)", + &[], + ), + stmt( + "SELECT i_price, i_name, i_data FROM item WHERE i_id = ?1", + &[ColumnType::Decimal, ColumnType::Utf8, ColumnType::Utf8], + ), + stmt( + "SELECT s_quantity, s_data, s_dist_01, s_dist_02, s_dist_03, s_dist_04, s_dist_05, s_dist_06, s_dist_07, s_dist_08, s_dist_09, s_dist_10 FROM stock WHERE s_i_id = ?1 AND s_w_id = ?2", + &[ + ColumnType::Int16, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ], + ), + stmt( + "UPDATE stock SET s_quantity = ?1 WHERE s_i_id = ?2 AND s_w_id = ?3", + &[], + ), + stmt( + "INSERT INTO order_line (ol_o_id, ol_d_id, ol_w_id, ol_number, ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_dist_info) VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7, ?8, ?9)", + &[], + ), + ], + vec![ + stmt( + "UPDATE warehouse SET w_ytd = w_ytd + ?1 WHERE w_id = ?2", + &[], + ), + stmt( + "SELECT w_street_1, w_street_2, w_city, w_state, w_zip, w_name FROM warehouse WHERE w_id = ?1", + &[ + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ], + ), + stmt( + "UPDATE district SET d_ytd = d_ytd + ?1 WHERE d_w_id = ?2 AND d_id = ?3", + &[], + ), + stmt( + "SELECT d_street_1, d_street_2, d_city, d_state, d_zip, d_name FROM district WHERE d_w_id = ?1 AND d_id = ?2", + &[ + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ], + ), + stmt( + "SELECT count(c_id) FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3", + &[ColumnType::Int32], + ), + stmt( + "SELECT c_id FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3 ORDER BY c_first", + &[ColumnType::Int32], + ), + stmt( + "SELECT c_first, c_middle, c_last, c_street_1, c_street_2, c_city, c_state, c_zip, c_phone, c_credit, c_credit_lim, c_discount, c_balance, c_since FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3", + &[ + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Int64, + ColumnType::Decimal, + ColumnType::Decimal, + ColumnType::DateTime, + ], + ), + stmt( + "SELECT c_data FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3", + &[ColumnType::Utf8], + ), + stmt( + "UPDATE customer SET c_balance = ?1, c_data = ?2 WHERE c_w_id = ?3 AND c_d_id = ?4 AND c_id = ?5", + &[], + ), + stmt( + "UPDATE customer SET c_balance = ?1 WHERE c_w_id = ?2 AND c_d_id = ?3 AND c_id = ?4", + &[], + ), + stmt( + "INSERT INTO history(h_c_d_id, h_c_w_id, h_c_id, h_d_id, h_w_id, h_date, h_amount, h_data) VALUES(?1, ?2, ?3, ?4, ?5, ?6, ?7, ?8)", + &[], + ), + ], + vec![ + // "SELECT count(c_id) FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3" + stmt( + "SELECT count(c_id) FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3", + &[ColumnType::Int32], + ), + // "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE ... ORDER BY c_first" + stmt( + "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_last = ?3 ORDER BY c_first", + &[ + ColumnType::Decimal, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ], + ), + // "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3" + stmt( + "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ?1 AND c_d_id = ?2 AND c_id = ?3", + &[ + ColumnType::Decimal, + ColumnType::Utf8, + ColumnType::Utf8, + ColumnType::Utf8, + ], + ), + // "SELECT o_id, o_entry_d, COALESCE(o_carrier_id,0) FROM orders ..." + stmt( + "SELECT o_id, o_entry_d, COALESCE(o_carrier_id,0) FROM orders WHERE o_w_id = ?1 AND o_d_id = ?2 AND o_c_id = ?3 AND o_id = (SELECT MAX(o_id) FROM orders WHERE o_w_id = ?4 AND o_d_id = ?5 AND o_c_id = ?6)", + &[ColumnType::Int32, ColumnType::DateTime, ColumnType::Int32], + ), + // "SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line ..." + stmt( + "SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line WHERE ol_w_id = ?1 AND ol_d_id = ?2 AND ol_o_id = ?3", + &[ + ColumnType::Int32, + ColumnType::Int16, + ColumnType::Int8, + ColumnType::Decimal, + ColumnType::NullableDateTime, + ], + ), + ], + vec![ + // "SELECT COALESCE(MIN(no_o_id),0) FROM new_orders WHERE no_d_id = ?1 AND no_w_id = ?2" + stmt( + "SELECT COALESCE(MIN(no_o_id),0) FROM new_orders WHERE no_d_id = ?1 AND no_w_id = ?2", + &[ColumnType::Int32], + ), + // "DELETE FROM new_orders WHERE no_o_id = ?1 AND no_d_id = ?2 AND no_w_id = ?3" + stmt( + "DELETE FROM new_orders WHERE no_o_id = ?1 AND no_d_id = ?2 AND no_w_id = ?3", + &[], + ), + // "SELECT o_c_id FROM orders WHERE o_id = ?1 AND o_d_id = ?2 AND o_w_id = ?3" + stmt( + "SELECT o_c_id FROM orders WHERE o_id = ?1 AND o_d_id = ?2 AND o_w_id = ?3", + &[ColumnType::Int32], + ), + // "UPDATE orders SET o_carrier_id = ?1 WHERE o_id = ?2 AND o_d_id = ?3 AND o_w_id = ?4" + stmt( + "UPDATE orders SET o_carrier_id = ?1 WHERE o_id = ?2 AND o_d_id = ?3 AND o_w_id = ?4", + &[], + ), + // "UPDATE order_line SET ol_delivery_d = ?1 WHERE ol_o_id = ?2 AND ol_d_id = ?3 AND ol_w_id = ?4" + stmt( + "UPDATE order_line SET ol_delivery_d = ?1 WHERE ol_o_id = ?2 AND ol_d_id = ?3 AND ol_w_id = ?4", + &[], + ), + // "SELECT SUM(ol_amount) FROM order_line WHERE ol_o_id = ?1 AND ol_d_id = ?2 AND ol_w_id = ?3" + stmt( + "SELECT SUM(ol_amount) FROM order_line WHERE ol_o_id = ?1 AND ol_d_id = ?2 AND ol_w_id = ?3", + &[ColumnType::Decimal], + ), + // "UPDATE customer SET c_balance = c_balance + ?1 , c_delivery_cnt = c_delivery_cnt + 1 WHERE c_id = ?2 ..." + stmt( + "UPDATE customer SET c_balance = c_balance + ?1 , c_delivery_cnt = c_delivery_cnt + 1 WHERE c_id = ?2 AND c_d_id = ?3 AND c_w_id = ?4", + &[], + ), + ], + vec![ + // "SELECT d_next_o_id FROM district WHERE d_id = ?1 AND d_w_id = ?2" + stmt( + "SELECT d_next_o_id FROM district WHERE d_id = ?1 AND d_w_id = ?2", + &[ColumnType::Int32], + ), + stmt(STOCK_LEVEL_DISTINCT_SQL, &[ColumnType::Int32]), + // "SELECT count(*) FROM stock WHERE s_w_id = ?1 AND s_i_id = ?2 AND s_quantity < ?3" + stmt( + "SELECT count(*) FROM stock WHERE s_w_id = ?1 AND s_i_id = ?2 AND s_quantity < ?3", + &[ColumnType::Int32], + ), + ], + ] +} + +fn stmt(sql: &'static str, result_types: &'static [ColumnType]) -> StatementSpec { + StatementSpec { sql, result_types } +} + +fn print_summary_table(success: &[usize], late: &[usize], failure: &[usize], elapsed: Duration) { + println!("---------------------------------------------------"); + println!( + "Transaction Summary (elapsed {:.1}s)", + elapsed.as_secs_f32() + ); + println!("+--------------+---------+------+---------+-------+"); + println!("| Transaction | Success | Late | Failure | Total |"); + println!("+--------------+---------+------+---------+-------+"); + for (idx, name) in TX_NAMES.iter().enumerate() { + let total = success[idx] + late[idx] + failure[idx]; + println!( + "| {:<12} | {:>7} | {:>4} | {:>7} | {:>5} |", + name, success[idx], late[idx], failure[idx], total + ); + } + println!("+--------------+---------+------+---------+-------+"); +} + +fn print_constraint_checks(success: &[usize], late: &[usize]) { + println!(" (all must be [OK])"); + println!("[transaction percentage]"); + let total: f64 = success + .iter() + .zip(late.iter()) + .map(|(s, l)| (s + l) as f64) + .sum(); + let checks = [ + (1, "Payment", 43.0), + (2, "Order-Status", 4.0), + (3, "Delivery", 4.0), + (4, "Stock-Level", 4.0), + ]; + for (idx, name, threshold) in checks { + let pct = if total > 0.0 { + ((success[idx] + late[idx]) as f64 / total) * 100.0 + } else { + 0.0 + }; + let status = if pct >= threshold { "OK" } else { "NG" }; + println!(" {name}: {pct:>5.1}% (>={threshold:.1}%) [{status}]"); + } +} + +fn print_response_checks(success: &[usize], late: &[usize]) { + println!("[response time (at least 90% passed)]"); + for (idx, name) in TX_NAMES.iter().enumerate() { + let total = success[idx] + late[idx]; + if total == 0 { + println!(" {name}: n/a [NG]"); + continue; + } + let pct = (success[idx] as f64 / total as f64) * 100.0; + let status = if pct >= 90.0 { "OK" } else { "NG" }; + println!(" {name}: {pct:>5.1}% [{status}]"); + } +} + +fn update_progress_bar( + pb: &ProgressBar, + round: usize, success: &[usize], late: &[usize], failure: &[usize], - fn_print: F, + start: Instant, ) { - for (i, name) in vec![ - "New-Order", - "Payment", - "Order-Status", - "Delivery", - "Stock-Level", - ] - .into_iter() - .enumerate() - { - fn_print(name, success[i], late[i], failure[i]); - } + let elapsed = start.elapsed(); + let total_success: usize = success.iter().sum(); + let total_late: usize = late.iter().sum(); + let total_failure: usize = failure.iter().sum(); + let total_mix: f64 = success + .iter() + .zip(late.iter()) + .map(|(s, l)| (s + l) as f64) + .sum(); + let mix_str = if total_mix > 0.0 { + let share = |idx: usize| ((success[idx] + late[idx]) as f64 / total_mix) * 100.0; + format!( + "mix NO {:>4.1}% P {:>4.1}% OS {:>4.1}% D {:>4.1}% SL {:>4.1}%", + share(0), + share(1), + share(2), + share(3), + share(4) + ) + } else { + "mix n/a".to_string() + }; + let est_tpmc = if elapsed.as_secs_f64() > 0.0 { + ((success[0] + late[0]) as f64) / (elapsed.as_secs_f64() / 60.0) + } else { + 0.0 + }; + pb.set_message(format!( + "round {:>6} | succ {:>6} late {:>5} fail {:>5} | est TpmC {:>6.0} | {}", + round, total_success, total_late, total_failure, est_tpmc, mix_str + )); +} + +fn print_checkpoint( + checkpoint_idx: usize, + round: usize, + test_name: &str, + p90: f64, + success: &[usize], + late: &[usize], + failure: &[usize], + start: Instant, + progress: &ProgressBar, +) { + let elapsed = start.elapsed(); + let total_failure: usize = failure.iter().sum(); + let est_tpmc = if elapsed.as_secs_f64() > 0.0 { + ((success[0] + late[0]) as f64) / (elapsed.as_secs_f64() / 60.0) + } else { + 0.0 + }; + + progress.println(format!( + "[CP {checkpoint_idx:>3} | round {round:>6} | {test_name} p90={p90:.3}s | \ +est TpmC {:>6.0} | total fail {:>6}]", + est_tpmc, total_failure + )); } fn other_ware(rng: &mut ThreadRng, home_ware: usize, num_ware: usize) -> usize { @@ -375,6 +652,24 @@ pub enum TpccError { #[from] DatabaseError, ), + #[error("sqlite: {0}")] + Sqlite( + #[source] + #[from] + sqlite::Error, + ), + #[error("decimal parse error: {0}")] + Decimal( + #[source] + #[from] + rust_decimal::Error, + ), + #[error("datetime parse error: {0}")] + Chrono( + #[source] + #[from] + chrono::ParseError, + ), #[error("io error: {0}")] Io( #[source] @@ -385,11 +680,20 @@ pub enum TpccError { EmptyTuples, #[error("maximum retries reached")] MaxRetry, + #[error("invalid backend usage")] + InvalidBackend, + #[error("invalid parameter name")] + InvalidParameter, + #[error("invalid datetime value")] + InvalidDateTime, + #[error("backend mismatch: {0}")] + BackendMismatch(String), } #[ignore] #[test] fn explain_tpcc() -> Result<(), DatabaseError> { + use kite_sql::db::DataBaseBuilder; use kite_sql::types::tuple::create_table; let database = DataBaseBuilder::path("./kite_sql_tpcc").build()?; diff --git a/tpcc/src/new_ord.rs b/tpcc/src/new_ord.rs index ec9d3939..f0493f4b 100644 --- a/tpcc/src/new_ord.rs +++ b/tpcc/src/new_ord.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::{BackendTransaction, PreparedStatement, TransactionExt}; use crate::load::{nu_rand, CUST_PER_DIST, DIST_PER_WARE, MAX_ITEMS, MAX_NUM_ITEMS}; use crate::{other_ware, TpccArgs, TpccError, TpccTest, TpccTransaction, ALLOW_MULTI_WAREHOUSE_TX}; use chrono::Utc; -use kite_sql::db::{DBTransaction, ResultIter, Statement}; -use kite_sql::storage::Storage; use kite_sql::types::value::DataValue; use rand::prelude::ThreadRng; use rand::Rng; @@ -65,13 +64,13 @@ impl NewOrdArgs { pub(crate) struct NewOrd; pub(crate) struct NewOrdTest; -impl TpccTransaction for NewOrd { +impl TpccTransaction for NewOrd { type Args = NewOrdArgs; fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let mut price = vec![Decimal::default(); MAX_NUM_ITEMS]; let mut iname = vec![String::new(); MAX_NUM_ITEMS]; @@ -82,18 +81,15 @@ impl TpccTransaction for NewOrd { let (c_discount, _c_last, _c_credit, w_tax) = if args.joins { // "SELECT c_discount, c_last, c_credit, w_tax FROM customer, warehouse WHERE w_id = ? AND c_w_id = w_id AND c_d_id = ? AND c_id = ?" - let tuple = tx - .execute( - &statements[0], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int16(args.w_id as i16)), - ("?3", DataValue::Int8(args.d_id as i8)), - ("?4", DataValue::Int64(args.c_id as i64)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[0], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int16(args.w_id as i16)), + ("?3", DataValue::Int8(args.d_id as i8)), + ("?4", DataValue::Int64(args.c_id as i64)), + ], + )?; let c_discount = tuple.values[0].decimal().unwrap(); let c_last = tuple.values[1].utf8().unwrap().to_string(); let c_credit = tuple.values[2].utf8().unwrap().to_string(); @@ -102,58 +98,48 @@ impl TpccTransaction for NewOrd { (c_discount, c_last, c_credit, w_tax) } else { // "SELECT c_discount, c_last, c_credit FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - let tuple = tx - .execute( - &statements[1], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int32(args.c_id as i32)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[1], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int32(args.c_id as i32)), + ], + )?; let c_discount = tuple.values[0].decimal().unwrap(); let c_last = tuple.values[1].utf8().unwrap().to_string(); let c_credit = tuple.values[2].utf8().unwrap().to_string(); // "SELECT w_tax FROM warehouse WHERE w_id = ?" - let tuple = tx - .execute( - &statements[2], - &[("?1", DataValue::Int16(args.w_id as i16))], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[2], + &[("?1", DataValue::Int16(args.w_id as i16))], + )?; let w_tax = tuple.values[0].decimal().unwrap(); (c_discount, c_last, c_credit, w_tax) }; // "SELECT d_next_o_id, d_tax FROM district WHERE d_id = ? AND d_w_id = ? FOR UPDATE" - let tuple = tx - .execute( - &statements[3], - &[ - ("?1", DataValue::Int8(args.d_id as i8)), - ("?2", DataValue::Int16(args.w_id as i16)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[3], + &[ + ("?1", DataValue::Int8(args.d_id as i8)), + ("?2", DataValue::Int16(args.w_id as i16)), + ], + )?; let d_next_o_id = tuple.values[0].i32().unwrap(); let d_tax = tuple.values[1].decimal().unwrap(); // "UPDATE district SET d_next_o_id = ? + 1 WHERE d_id = ? AND d_w_id = ?" - tx.execute( + tx.execute_drain( &statements[4], &[ ("?1", DataValue::Int32(d_next_o_id)), ("?2", DataValue::Int8(args.d_id as i8)), ("?3", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; let o_id = d_next_o_id; // "INSERT INTO orders (o_id, o_d_id, o_w_id, o_c_id, o_entry_d, o_ol_cnt, o_all_local) VALUES(?, ?, ?, ?, ?, ?, ?)" - tx.execute( + tx.execute_drain( &statements[5], &[ ("?1", DataValue::Int32(o_id)), @@ -164,18 +150,16 @@ impl TpccTransaction for NewOrd { ("?6", DataValue::Int8(args.o_ol_cnt as i8)), ("?7", DataValue::Int8(args.o_all_local as i8)), ], - )? - .done()?; - // "INSERT INTO new_orders (no_o_id, no_d_id, no_w_id) VALUES (?,?,?) - tx.execute( + )?; + // "INSERT INTO new_orders (no_o_id, no_d_id, no_w_id) VALUES (?,?,?)" + tx.execute_drain( &statements[6], &[ ("?1", DataValue::Int32(o_id)), ("?2", DataValue::Int8(args.d_id as i8)), ("?3", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; let mut ol_num_seq = vec![0; MAX_NUM_ITEMS]; for i in 0..args.o_ol_cnt { @@ -204,16 +188,8 @@ impl TpccTransaction for NewOrd { let ol_i_id = args.item_id[ol_num_seq[ol_number - 1]]; let ol_quantity = args.qty[ol_num_seq[ol_number - 1]]; // "SELECT i_price, i_name, i_data FROM item WHERE i_id = ?" - let tuple = tx - .execute( - &statements[7], - vec![("?1", DataValue::Int32(ol_i_id as i32))], - )? - .next(); - let Some(tuple) = tuple else { - return Err(TpccError::EmptyTuples); - }; - let tuple = tuple?; + let params = [("?1", DataValue::Int32(ol_i_id as i32))]; + let tuple = tx.query_one(&statements[7], ¶ms)?; let i_price = tuple.values[0].decimal().unwrap(); let i_name = tuple.values[1].utf8().unwrap(); let i_data = tuple.values[2].utf8().unwrap(); @@ -222,16 +198,11 @@ impl TpccTransaction for NewOrd { iname[ol_num_seq[ol_number - 1]] = i_name.to_string(); // "SELECT s_quantity, s_data, s_dist_01, s_dist_02, s_dist_03, s_dist_04, s_dist_05, s_dist_06, s_dist_07, s_dist_08, s_dist_09, s_dist_10 FROM stock WHERE s_i_id = ? AND s_w_id = ? FOR UPDATE" - let tuple = tx - .execute( - &statements[8], - vec![ - ("?1", DataValue::Int32(ol_i_id as i32)), - ("?2", DataValue::Int16(ol_supply_w_id as i16)), - ], - )? - .next() - .unwrap()?; + let params = [ + ("?1", DataValue::Int32(ol_i_id as i32)), + ("?2", DataValue::Int16(ol_supply_w_id as i16)), + ]; + let tuple = tx.query_one(&statements[8], ¶ms)?; let mut s_quantity = tuple.values[0].i16().unwrap(); let s_data = tuple.values[1].utf8().unwrap(); let s_dist_01 = tuple.values[2].utf8().unwrap(); @@ -263,15 +234,12 @@ impl TpccTransaction for NewOrd { s_quantity - ol_quantity as i16 + 91 }; // "UPDATE stock SET s_quantity = ? WHERE s_i_id = ? AND s_w_id = ?" - tx.execute( - &statements[9], - vec![ - ("?1", DataValue::Int16(s_quantity)), - ("?2", DataValue::Int32(ol_i_id as i32)), - ("?3", DataValue::Int16(ol_supply_w_id as i16)), - ], - )? - .done()?; + let params = [ + ("?1", DataValue::Int16(s_quantity)), + ("?2", DataValue::Int32(ol_i_id as i32)), + ("?3", DataValue::Int16(ol_supply_w_id as i16)), + ]; + tx.execute_drain(&statements[9], ¶ms)?; // Tips: Integers always have 7 digits, so divide by 10 here let mut ol_amount = Decimal::from(ol_quantity) @@ -284,28 +252,25 @@ impl TpccTransaction for NewOrd { amt[ol_num_seq[ol_number - 1]] = ol_amount; // "INSERT INTO order_line (ol_o_id, ol_d_id, ol_w_id, ol_number, ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_dist_info) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)" - tx.execute( - &statements[10], - vec![ - ("?1", DataValue::Int32(o_id)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int16(args.w_id as i16)), - ("?4", DataValue::Int8(ol_number as i8)), - ("?5", DataValue::Int32(ol_i_id as i32)), - ("?6", DataValue::Int16(ol_supply_w_id as i16)), - ("?7", DataValue::Int8(ol_quantity as i8)), - ("?8", DataValue::Decimal(ol_amount.round_dp(2))), - ("?9", DataValue::from(ol_dist_info)), - ], - )? - .done()?; + let params = [ + ("?1", DataValue::Int32(o_id)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int16(args.w_id as i16)), + ("?4", DataValue::Int8(ol_number as i8)), + ("?5", DataValue::Int32(ol_i_id as i32)), + ("?6", DataValue::Int16(ol_supply_w_id as i16)), + ("?7", DataValue::Int8(ol_quantity as i8)), + ("?8", DataValue::Decimal(ol_amount.round_dp(2))), + ("?9", DataValue::from(ol_dist_info)), + ]; + tx.execute_drain(&statements[10], ¶ms)?; } Ok(()) } } -impl TpccTest for NewOrdTest { +impl TpccTest for NewOrdTest { fn name(&self) -> &'static str { "New-Order" } @@ -313,10 +278,10 @@ impl TpccTest for NewOrdTest { fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, args: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let mut all_local = 1; let notfound = MAX_ITEMS + 1; diff --git a/tpcc/src/order_stat.rs b/tpcc/src/order_stat.rs index e32595d2..70902ab8 100644 --- a/tpcc/src/order_stat.rs +++ b/tpcc/src/order_stat.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::{BackendTransaction, PreparedStatement, TransactionExt}; use crate::load::{last_name, nu_rand, CUST_PER_DIST, DIST_PER_WARE}; use crate::{TpccArgs, TpccError, TpccTest, TpccTransaction}; -use kite_sql::db::{DBTransaction, Statement}; -use kite_sql::storage::Storage; use kite_sql::types::value::DataValue; use rand::prelude::ThreadRng; use rand::Rng; @@ -51,37 +50,32 @@ impl OrderStatArgs { pub(crate) struct OrderStat; pub(crate) struct OrderStatTest; -impl TpccTransaction for OrderStat { +impl TpccTransaction for OrderStat { type Args = OrderStatArgs; fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let (_c_balance, _c_first, _c_middle, _c_last) = if args.by_name { - // SELECT count(c_id) FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ?" - let tuple = tx - .execute( - &statements[0], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::from(args.c_last.clone())), - ], - )? - .next() - .unwrap()?; - let mut name_cnt = tuple.values[0].i32().unwrap() as usize; - // SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ? ORDER BY c_first" - let mut tuple_iter = tx.execute( - &statements[1], + // "SELECT count(c_id) FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ?" + let tuple = tx.query_one( + &statements[0], &[ ("?1", DataValue::Int16(args.w_id as i16)), ("?2", DataValue::Int8(args.d_id as i8)), ("?3", DataValue::from(args.c_last.clone())), ], )?; + let mut name_cnt = tuple.values[0].i32().unwrap() as usize; + // "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ? ORDER BY c_first" + let params = [ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::from(args.c_last.clone())), + ]; + let mut tuple_iter = tx.execute(&statements[1], ¶ms)?; if name_cnt % 2 == 1 { name_cnt += 1; @@ -102,57 +96,38 @@ impl TpccTransaction for OrderStat { (c_balance, c_first, c_middle, c_last) } else { // "SELECT c_balance, c_first, c_middle, c_last FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - let tuple = tx - .execute( - &statements[2], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int32(args.c_id as i32)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[2], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int32(args.c_id as i32)), + ], + )?; let c_balance = tuple.values[0].decimal().unwrap(); let c_first = tuple.values[1].utf8().unwrap().to_string(); let c_middle = tuple.values[2].utf8().unwrap().to_string(); let c_last = tuple.values[3].utf8().unwrap().to_string(); (c_balance, c_first, c_middle, c_last) }; - // TODO: Join Eq // "SELECT o_id, o_entry_d, COALESCE(o_carrier_id,0) FROM orders WHERE o_w_id = ? AND o_d_id = ? AND o_c_id = ? AND o_id = (SELECT MAX(o_id) FROM orders WHERE o_w_id = ? AND o_d_id = ? AND o_c_id = ?)" - let tuple = tx - .execute( - &statements[3], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int32(args.c_id as i32)), - ("?4", DataValue::Int16(args.w_id as i16)), - ("?5", DataValue::Int8(args.d_id as i8)), - ("?6", DataValue::Int32(args.c_id as i32)), - ], - )? - .next(); - let Some(tuple) = tuple else { - return Err(TpccError::EmptyTuples); - }; - let tuple = tuple?; + let params = [ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int32(args.c_id as i32)), + ("?4", DataValue::Int16(args.w_id as i16)), + ("?5", DataValue::Int8(args.d_id as i8)), + ("?6", DataValue::Int32(args.c_id as i32)), + ]; + let tuple = tx.query_one(&statements[3], ¶ms)?; let o_id = tuple.values[0].i32().unwrap(); - // let o_entry_d = tuple.values[1].datetime().unwrap(); - // let o_carrier_id = tuple.values[2].i32().unwrap(); // "SELECT ol_i_id, ol_supply_w_id, ol_quantity, ol_amount, ol_delivery_d FROM order_line WHERE ol_w_id = ? AND ol_d_id = ? AND ol_o_id = ?" - let _tuple = tx - .execute( - &statements[4], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int32(o_id)), - ], - )? - .next() - .unwrap()?; + let params = [ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int32(o_id)), + ]; + let _tuple = tx.query_one(&statements[4], ¶ms)?; // let ol_i_id = tuple.values[0].i32(); // let ol_supply_w_id = tuple.values[1].i16(); // let ol_quantity = tuple.values[2].i8(); @@ -163,7 +138,7 @@ impl TpccTransaction for OrderStat { } } -impl TpccTest for OrderStatTest { +impl TpccTest for OrderStatTest { fn name(&self) -> &'static str { "Order-Status" } @@ -171,10 +146,10 @@ impl TpccTest for OrderStatTest { fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, _: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let w_id = rng.gen_range(0..num_ware) + 1; let d_id = rng.gen_range(1..DIST_PER_WARE); diff --git a/tpcc/src/payment.rs b/tpcc/src/payment.rs index fb79aed6..b6ad0ba6 100644 --- a/tpcc/src/payment.rs +++ b/tpcc/src/payment.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::{BackendTransaction, PreparedStatement, TransactionExt}; use crate::load::{last_name, nu_rand, CUST_PER_DIST, DIST_PER_WARE}; use crate::{other_ware, TpccArgs, TpccError, TpccTest, TpccTransaction, ALLOW_MULTI_WAREHOUSE_TX}; use chrono::Utc; -use kite_sql::db::{DBTransaction, ResultIter, Statement}; -use kite_sql::storage::Storage; use kite_sql::types::value::DataValue; use rand::prelude::ThreadRng; use rand::Rng; @@ -62,33 +61,29 @@ impl PaymentArgs { pub(crate) struct Payment; pub(crate) struct PaymentTest; -impl TpccTransaction for Payment { +impl TpccTransaction for Payment { type Args = PaymentArgs; #[allow(unused_variables)] fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let now = Utc::now(); // "UPDATE warehouse SET w_ytd = w_ytd + ? WHERE w_id = ?" - tx.execute( + tx.execute_drain( &statements[0], &[ ("?1", DataValue::Decimal(args.h_amount)), ("?2", DataValue::Int16(args.w_id as i16)), ], - )? - .done()?; + )?; // "SELECT w_street_1, w_street_2, w_city, w_state, w_zip, w_name FROM warehouse WHERE w_id = ?" - let tuple = tx - .execute( - &statements[1], - &[("?1", DataValue::Int16(args.w_id as i16))], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[1], + &[("?1", DataValue::Int16(args.w_id as i16))], + )?; let w_street_1 = tuple.values[0].utf8().unwrap(); let w_street_2 = tuple.values[1].utf8().unwrap(); let w_city = tuple.values[2].utf8().unwrap(); @@ -97,27 +92,23 @@ impl TpccTransaction for Payment { let w_name = tuple.values[5].utf8().unwrap(); // "UPDATE district SET d_ytd = d_ytd + ? WHERE d_w_id = ? AND d_id = ?" - tx.execute( + tx.execute_drain( &statements[2], &[ ("?1", DataValue::Decimal(args.h_amount)), ("?2", DataValue::Int16(args.w_id as i16)), ("?3", DataValue::Int8(args.d_id as i8)), ], - )? - .done()?; + )?; // "SELECT d_street_1, d_street_2, d_city, d_state, d_zip, d_name FROM district WHERE d_w_id = ? AND d_id = ?" - let tuple = tx - .execute( - &statements[3], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[3], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ], + )?; let d_street_1 = tuple.values[0].utf8().unwrap(); let d_street_2 = tuple.values[1].utf8().unwrap(); let d_city = tuple.values[2].utf8().unwrap(); @@ -128,28 +119,22 @@ impl TpccTransaction for Payment { let mut c_id = args.c_id as i32; if args.by_name { // "SELECT count(c_id) FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ?" - let tuple = tx - .execute( - &statements[4], - &[ - ("?1", DataValue::Int16(args.c_w_id as i16)), - ("?2", DataValue::Int8(args.c_d_id as i8)), - ("?3", DataValue::from(args.c_last.clone())), - ], - )? - .next() - .unwrap()?; - let mut name_cnt = tuple.values[0].i32().unwrap(); - - // "SELECT c_id FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ? ORDER BY c_first" - let mut tuple_iter = tx.execute( - &statements[5], + let tuple = tx.query_one( + &statements[4], &[ ("?1", DataValue::Int16(args.c_w_id as i16)), ("?2", DataValue::Int8(args.c_d_id as i8)), ("?3", DataValue::from(args.c_last.clone())), ], )?; + let mut name_cnt = tuple.values[0].i32().unwrap(); + // "SELECT c_id FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_last = ? ORDER BY c_first" + let params = [ + ("?1", DataValue::Int16(args.c_w_id as i16)), + ("?2", DataValue::Int8(args.c_d_id as i8)), + ("?3", DataValue::from(args.c_last.clone())), + ]; + let mut tuple_iter = tx.execute(&statements[5], ¶ms)?; if name_cnt % 2 == 1 { name_cnt += 1; } @@ -159,17 +144,14 @@ impl TpccTransaction for Payment { } } // "SELECT c_first, c_middle, c_last, c_street_1, c_street_2, c_city, c_state, c_zip, c_phone, c_credit, c_credit_lim, c_discount, c_balance, c_since FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_id = ? FOR UPDATE" - let tuple = tx - .execute( - &statements[6], - &[ - ("?1", DataValue::Int16(args.c_w_id as i16)), - ("?2", DataValue::Int8(args.c_d_id as i8)), - ("?3", DataValue::Int32(c_id)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[6], + &[ + ("?1", DataValue::Int16(args.c_w_id as i16)), + ("?2", DataValue::Int8(args.c_d_id as i8)), + ("?3", DataValue::Int32(c_id)), + ], + )?; let c_first = tuple.values[0].utf8().unwrap(); let c_middle = tuple.values[1].utf8().unwrap(); let c_last = tuple.values[2].utf8().unwrap(); @@ -189,24 +171,21 @@ impl TpccTransaction for Payment { if let Some(c_credit) = c_credit { if c_credit.contains("BC") { // "SELECT c_data FROM customer WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - let tuple = tx - .execute( - &statements[7], - &[ - ("?1", DataValue::Int16(args.c_w_id as i16)), - ("?2", DataValue::Int8(args.c_d_id as i8)), - ("?3", DataValue::Int32(c_id)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[7], + &[ + ("?1", DataValue::Int16(args.c_w_id as i16)), + ("?2", DataValue::Int8(args.c_d_id as i8)), + ("?3", DataValue::Int32(c_id)), + ], + )?; let c_data = tuple.values[0].utf8().unwrap(); // https://github.com/AgilData/tpcc/blob/dfbabe1e35cc93b2bf2e107fc699eb29c2097e24/src/main/java/com/codefutures/tpcc/Payment.java#L284 // let c_new_data = format!("| {} {} {} {} {} {} {}", c_id, args.c_d_id, args.c_w_id, args.d_id, args.w_id, args.h_amount, ) // "UPDATE customer SET c_balance = ?, c_data = ? WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - tx.execute( + tx.execute_drain( &statements[8], &[ ("?1", DataValue::Decimal(c_balance)), @@ -215,11 +194,10 @@ impl TpccTransaction for Payment { ("?4", DataValue::Int8(args.c_d_id as i8)), ("?5", DataValue::Int32(c_id)), ], - )? - .done()?; + )?; } else { // "UPDATE customer SET c_balance = ? WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - tx.execute( + tx.execute_drain( &statements[9], &[ ("?1", DataValue::Decimal(c_balance)), @@ -227,12 +205,11 @@ impl TpccTransaction for Payment { ("?3", DataValue::Int8(args.c_d_id as i8)), ("?4", DataValue::Int32(c_id)), ], - )? - .done()?; + )?; } } else { // "UPDATE customer SET c_balance = ? WHERE c_w_id = ? AND c_d_id = ? AND c_id = ?" - tx.execute( + tx.execute_drain( &statements[9], &[ ("?1", DataValue::Decimal(c_balance)), @@ -240,12 +217,11 @@ impl TpccTransaction for Payment { ("?3", DataValue::Int8(args.c_d_id as i8)), ("?4", DataValue::Int32(c_id)), ], - )? - .done()?; + )?; } let h_data = format!("\\0{d_name} \\0"); // "INSERT INTO history(h_c_d_id, h_c_w_id, h_c_id, h_d_id, h_w_id, h_date, h_amount, h_data) VALUES(?, ?, ?, ?, ?, ?, ?, ?)" - tx.execute( + tx.execute_drain( &statements[10], &[ ("?1", DataValue::Int8(args.c_d_id as i8)), @@ -257,14 +233,13 @@ impl TpccTransaction for Payment { ("?7", DataValue::Decimal(args.h_amount)), ("?8", DataValue::from(h_data)), ], - )? - .done()?; + )?; Ok(()) } } -impl TpccTest for PaymentTest { +impl TpccTest for PaymentTest { fn name(&self) -> &'static str { "Payment" } @@ -272,10 +247,10 @@ impl TpccTest for PaymentTest { fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, _: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let w_id = rng.gen_range(0..num_ware) + 1; let d_id = rng.gen_range(1..DIST_PER_WARE); diff --git a/tpcc/src/slev.rs b/tpcc/src/slev.rs index 54347ccc..bf7108e0 100644 --- a/tpcc/src/slev.rs +++ b/tpcc/src/slev.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::backend::{BackendTransaction, PreparedStatement, TransactionExt}; use crate::load::DIST_PER_WARE; use crate::{TpccArgs, TpccError, TpccTest, TpccTransaction}; -use kite_sql::db::{DBTransaction, Statement}; -use kite_sql::storage::Storage; use kite_sql::types::value::DataValue; use rand::prelude::ThreadRng; use rand::Rng; @@ -36,59 +35,50 @@ impl SlevArgs { pub(crate) struct Slev; pub(crate) struct SlevTest; -impl TpccTransaction for Slev { +impl TpccTransaction for Slev { type Args = SlevArgs; fn run( - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, args: &Self::Args, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { // "SELECT d_next_o_id FROM district WHERE d_id = ? AND d_w_id = ?" - let tuple = tx - .execute( - &statements[0], - &[ - ("?1", DataValue::Int8(args.d_id as i8)), - ("?2", DataValue::Int16(args.w_id as i16)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[0], + &[ + ("?1", DataValue::Int8(args.d_id as i8)), + ("?2", DataValue::Int16(args.w_id as i16)), + ], + )?; let d_next_o_id = tuple.values[0].i32().unwrap(); // "SELECT DISTINCT ol_i_id FROM order_line WHERE ol_w_id = ? AND ol_d_id = ? AND ol_o_id < ? AND ol_o_id >= (? - 20)" - let tuple = tx - .execute( - &statements[1], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(args.d_id as i8)), - ("?3", DataValue::Int32(d_next_o_id)), - ("?4", DataValue::Int32(d_next_o_id)), - ], - )? - .next() - .unwrap()?; + let tuple = tx.query_one( + &statements[1], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(args.d_id as i8)), + ("?3", DataValue::Int32(d_next_o_id)), + ("?4", DataValue::Int32(d_next_o_id)), + ], + )?; let ol_i_id = tuple.values[0].i32().unwrap(); // "SELECT count(*) FROM stock WHERE s_w_id = ? AND s_i_id = ? AND s_quantity < ?" - let _tuple = tx - .execute( - &statements[2], - &[ - ("?1", DataValue::Int16(args.w_id as i16)), - ("?2", DataValue::Int8(ol_i_id as i8)), - ("?3", DataValue::Int16(args.level as i16)), - ], - )? - .next() - .unwrap()?; + let _tuple = tx.query_one( + &statements[2], + &[ + ("?1", DataValue::Int16(args.w_id as i16)), + ("?2", DataValue::Int8(ol_i_id as i8)), + ("?3", DataValue::Int16(args.level as i16)), + ], + )?; // let i_count = tuple.values[0].i32().unwrap(); Ok(()) } } -impl TpccTest for SlevTest { +impl TpccTest for SlevTest { fn name(&self) -> &'static str { "Stock-Level" } @@ -96,10 +86,10 @@ impl TpccTest for SlevTest { fn do_transaction( &self, rng: &mut ThreadRng, - tx: &mut DBTransaction, + tx: &mut dyn BackendTransaction, num_ware: usize, _: &TpccArgs, - statements: &[Statement], + statements: &[PreparedStatement], ) -> Result<(), TpccError> { let w_id = rng.gen_range(0..num_ware) + 1; let d_id = rng.gen_range(1..DIST_PER_WARE);