diff --git a/code/.gitignore b/code/.gitignore index a9d37c5..4ca7e1d 100644 --- a/code/.gitignore +++ b/code/.gitignore @@ -1,2 +1,3 @@ target Cargo.lock +risinglight.db/ diff --git a/code/03-00/Cargo.toml b/code/03-00/Cargo.toml index 1feb335..10c0534 100644 --- a/code/03-00/Cargo.toml +++ b/code/03-00/Cargo.toml @@ -7,6 +7,7 @@ edition = "2021" [dependencies] anyhow = "1" bitvec = "1.0" +bytes = "1" enum_dispatch = "0.3" env_logger = "0.9" futures = { version = "0.3", default-features = false, features = ["alloc"] } @@ -16,8 +17,9 @@ log = "0.4" prettytable-rs = { version = "0.8", default-features = false } rustyline = "9" sqlparser = "0.13" +tempfile = "3" thiserror = "1" -tokio = { version = "1", features = ["rt", "rt-multi-thread", "sync", "macros"] } +tokio = { version = "1", features = ["rt", "rt-multi-thread", "sync", "macros", "fs"] } tokio-stream = "0.1" [dev-dependencies] diff --git a/code/03-00/src/db.rs b/code/03-00/src/db.rs index 5b29739..cae83a5 100644 --- a/code/03-00/src/db.rs +++ b/code/03-00/src/db.rs @@ -12,7 +12,7 @@ use crate::executor::{ExecuteError, ExecutorBuilder}; use crate::logical_planner::{LogicalPlanError, LogicalPlanner}; use crate::parser::{parse, ParserError}; use crate::physical_planner::{PhysicalPlanError, PhysicalPlanner}; -use crate::storage::DiskStorage; +use crate::storage::{DiskStorage, StorageOptions}; /// The database instance. pub struct Database { @@ -21,17 +21,11 @@ pub struct Database { runtime: Runtime, } -impl Default for Database { - fn default() -> Self { - Self::new() - } -} - impl Database { /// Create a new database instance. - pub fn new() -> Self { + pub fn new(options: StorageOptions) -> Self { let catalog = Arc::new(DatabaseCatalog::new()); - let storage = Arc::new(DiskStorage::new()); + let storage = Arc::new(DiskStorage::new(options)); let parallel = matches!(std::env::var("LIGHT_PARALLEL"), Ok(s) if s == "1"); let runtime = if parallel { tokio::runtime::Builder::new_multi_thread() diff --git a/code/03-00/src/executor/create.rs b/code/03-00/src/executor/create.rs index c4ad484..b06a81b 100644 --- a/code/03-00/src/executor/create.rs +++ b/code/03-00/src/executor/create.rs @@ -1,7 +1,6 @@ use super::*; use crate::catalog::TableRefId; use crate::physical_planner::PhysicalCreateTable; -use crate::storage::StorageRef; /// The executor of `CREATE TABLE` statement. pub struct CreateTableExecutor { @@ -16,11 +15,15 @@ impl CreateTableExecutor { let schema = self.catalog.get_schema(self.plan.schema_id).unwrap(); let table_id = schema.add_table(&self.plan.table_name).unwrap(); let table = schema.get_table(table_id).unwrap(); + let mut column_descs = vec![]; for (name, desc) in &self.plan.columns { table.add_column(name, desc.clone()).unwrap(); + column_descs.push(desc.clone()); } - self.storage - .add_table(TableRefId::new(self.plan.schema_id, table_id))?; + self.storage.add_table( + TableRefId::new(self.plan.schema_id, table_id), + &column_descs, + )?; yield DataChunk::single(1); } } diff --git a/code/03-00/src/executor/insert.rs b/code/03-00/src/executor/insert.rs index 0c76fc3..d40b61a 100644 --- a/code/03-00/src/executor/insert.rs +++ b/code/03-00/src/executor/insert.rs @@ -3,7 +3,6 @@ use itertools::Itertools; use super::*; use crate::array::{ArrayBuilderImpl, DataChunk}; use crate::catalog::{ColumnId, TableRefId}; -use crate::storage::StorageRef; use crate::types::{DataType, DataValue}; /// The executor of `INSERT` statement. @@ -38,12 +37,18 @@ impl InsertExecutor { ) .collect_vec(); let mut count = 0; + + let mut txn = table.write().await?; + #[for_await] for chunk in self.child { let chunk = transform_chunk(chunk?, &output_columns); count += chunk.cardinality(); - table.append(chunk)?; + txn.append(chunk).await?; } + + txn.commit().await?; + yield DataChunk::single(count as i32); } } diff --git a/code/03-00/src/executor/seq_scan.rs b/code/03-00/src/executor/seq_scan.rs index e6c0002..d1f5df9 100644 --- a/code/03-00/src/executor/seq_scan.rs +++ b/code/03-00/src/executor/seq_scan.rs @@ -13,8 +13,12 @@ impl SeqScanExecutor { #[try_stream(boxed, ok = DataChunk, error = ExecuteError)] pub async fn execute(self) { let table = self.storage.get_table(self.table_ref_id)?; - for chunk in table.all_chunks()? { + let txn = table.read().await?; + + for chunk in txn.all_chunks().await? { yield chunk; } + + txn.commit().await?; } } diff --git a/code/03-00/src/main.rs b/code/03-00/src/main.rs index fe69db4..5dd0de6 100644 --- a/code/03-00/src/main.rs +++ b/code/03-00/src/main.rs @@ -1,5 +1,6 @@ //! A simple interactive shell of the database. +use risinglight_03_00::storage::StorageOptions; use risinglight_03_00::Database; use rustyline::error::ReadlineError; use rustyline::Editor; @@ -7,7 +8,9 @@ use rustyline::Editor; fn main() { env_logger::init(); - let db = Database::new(); + let db = Database::new(StorageOptions { + base_path: "risinglight.db".into(), + }); let mut rl = Editor::<()>::new(); loop { diff --git a/code/03-00/src/storage/column.rs b/code/03-00/src/storage/column.rs new file mode 100644 index 0000000..46b3edf --- /dev/null +++ b/code/03-00/src/storage/column.rs @@ -0,0 +1,25 @@ +use anyhow::anyhow; +use bytes::{Buf, BufMut}; + +use super::StorageResult; +use crate::array::{Array, ArrayBuilder, I32Array, I32ArrayBuilder}; + +/// Encode an `I32Array` into a `Vec`. +pub fn encode_int32_column(a: &I32Array, mut buffer: impl BufMut) -> StorageResult<()> { + for item in a.iter() { + if let Some(item) = item { + buffer.put_i32_le(*item); + } else { + return Err(anyhow!("nullable encoding not supported!").into()); + } + } + Ok(()) +} + +pub fn decode_int32_column(mut data: impl Buf) -> StorageResult { + let mut builder = I32ArrayBuilder::with_capacity(data.remaining() / 4); + while data.has_remaining() { + builder.push(Some(&data.get_i32_le())); + } + Ok(builder.finish()) +} diff --git a/code/03-00/src/storage/mod.rs b/code/03-00/src/storage/mod.rs index f4fa7e3..58886eb 100644 --- a/code/03-00/src/storage/mod.rs +++ b/code/03-00/src/storage/mod.rs @@ -1,95 +1,208 @@ -//! In-memory storage. -//! -//! RisingLight's in-memory representation of data is very simple. Currently, -//! it is simple a vector of `DataChunk`. Upon insertion, users' data are -//! simply appended to the end of the vector. +//! On-disk storage + +mod column; +mod rowset; use std::collections::HashMap; -use std::sync::{Arc, Mutex, RwLock}; +use std::path::PathBuf; +use std::sync::atomic::AtomicU32; +use std::sync::{Arc, RwLock}; + +use anyhow::anyhow; +use self::rowset::{DiskRowset, RowSetBuilder}; use crate::array::DataChunk; -use crate::catalog::TableRefId; +use crate::catalog::{ColumnDesc, TableRefId}; /// The error type of storage operations. #[derive(thiserror::Error, Debug)] -pub enum StorageError { - #[error("table not found: {0:?}")] - NotFound(TableRefId), -} +#[error("{0:?}")] +pub struct StorageError(#[from] anyhow::Error); /// A specialized `Result` type for storage operations. pub type StorageResult = std::result::Result; pub type StorageRef = Arc; -pub type DiskTableRef = Arc; +pub type StorageTableRef = Arc; -/// In-memory storage. +/// On-disk storage. pub struct DiskStorage { - tables: Mutex>, + /// All tables in the current storage engine. + tables: RwLock>, + + /// Generator for RowSet id. + rowset_id_generator: Arc, + + /// The storage options. + options: Arc, } -impl Default for DiskStorage { - fn default() -> Self { - Self::new() - } +pub struct StorageOptions { + /// The directory of the storage + pub base_path: PathBuf, +} + +pub fn err(error: impl Into) -> StorageError { + StorageError(error.into()) +} + +/// An on-disk table. +pub struct DiskTable { + /// Id of the table. + id: TableRefId, + + /// Columns of the current table. + column_descs: Arc<[ColumnDesc]>, + + /// The storage options. + options: Arc, + + /// Generator for RowSet id. + rowset_id_generator: Arc, + + /// RowSets in the table + rowsets: RwLock>, } impl DiskStorage { /// Create a new in-memory storage. - pub fn new() -> Self { + pub fn new(options: StorageOptions) -> Self { DiskStorage { - tables: Mutex::new(HashMap::new()), + tables: RwLock::new(HashMap::new()), + options: Arc::new(options), + rowset_id_generator: Arc::new(AtomicU32::new(0)), } } /// Add a table. - pub fn add_table(&self, id: TableRefId) -> StorageResult<()> { - let table = Arc::new(DiskTable::new(id)); - self.tables.lock().unwrap().insert(id, table); + pub fn add_table(&self, id: TableRefId, column_descs: &[ColumnDesc]) -> StorageResult<()> { + let mut tables = self.tables.write().unwrap(); + let table = DiskTable { + id, + options: self.options.clone(), + column_descs: column_descs.into(), + rowsets: RwLock::new(Vec::new()), + rowset_id_generator: self.rowset_id_generator.clone(), + }; + let res = tables.insert(id, table.into()); + if res.is_some() { + return Err(anyhow!("table already exists: {:?}", id).into()); + } Ok(()) } /// Get a table. - pub fn get_table(&self, id: TableRefId) -> StorageResult { - self.tables - .lock() - .unwrap() + pub fn get_table(&self, id: TableRefId) -> StorageResult { + let tables = self.tables.read().unwrap(); + tables .get(&id) + .ok_or_else(|| anyhow!("table not found: {:?}", id).into()) .cloned() - .ok_or(StorageError::NotFound(id)) } } -/// A table in in-memory engine. -pub struct DiskTable { - #[allow(dead_code)] - id: TableRefId, - inner: RwLock, +impl DiskTable { + /// Start a transaction which only contains write. + pub async fn write(self: &Arc) -> StorageResult { + let rowsets = self.rowsets.read().unwrap(); + Ok(DiskTransaction { + read_only: false, + table: self.clone(), + rowset_snapshot: rowsets.clone(), + builder: None, + finished: false, + }) + } + + /// Start a transaction which only contains read. + pub async fn read(self: &Arc) -> StorageResult { + let rowsets = self.rowsets.read().unwrap(); + Ok(DiskTransaction { + read_only: true, + table: self.clone(), + rowset_snapshot: rowsets.clone(), + builder: None, + finished: false, + }) + } + + pub fn table_path(&self) -> PathBuf { + self.options.base_path.join(self.id.table_id.to_string()) + } + + pub fn rowset_path_of(&self, rowset_id: u32) -> PathBuf { + self.table_path().join(rowset_id.to_string()) + } } -#[derive(Default)] -struct DiskTableInner { - chunks: Vec, +pub struct DiskTransaction { + /// If this txn is read only. + read_only: bool, + + /// Reference to table object + table: Arc, + + /// Current snapshot of RowSets + rowset_snapshot: Vec, + + /// Builder for the RowSet + builder: Option, + + /// Indicates whether the transaction is committed or aborted. If + /// the [`SecondaryTransaction`] object is dropped without finishing, + /// the transaction will panic. + finished: bool, } -impl DiskTable { - fn new(id: TableRefId) -> Self { - Self { - id, - inner: RwLock::new(DiskTableInner::default()), +impl Drop for DiskTransaction { + fn drop(&mut self) { + if !self.finished { + warn!("Transaction dropped without committing or aborting"); } } +} +impl DiskTransaction { /// Append a chunk to the table. - pub fn append(&self, chunk: DataChunk) -> StorageResult<()> { - let mut inner = self.inner.write().unwrap(); - inner.chunks.push(chunk); + pub async fn append(&mut self, chunk: DataChunk) -> StorageResult<()> { + if self.read_only { + return Err(anyhow!("cannot append chunks in read only txn!").into()); + } + if self.builder.is_none() { + self.builder = Some(RowSetBuilder::new(self.table.column_descs.clone())); + } + let builder = self.builder.as_mut().unwrap(); + + builder.append(chunk)?; + + Ok(()) + } + + pub async fn commit(mut self) -> StorageResult<()> { + self.finished = true; + + if let Some(builder) = self.builder.take() { + use std::sync::atomic::Ordering::SeqCst; + let rowset_id = self.table.rowset_id_generator.fetch_add(1, SeqCst); + let rowset_path = self + .table + .options + .base_path + .join(self.table.rowset_path_of(rowset_id)); + let rowset = builder.flush(rowset_id, rowset_path).await?; + let mut rowsets = self.table.rowsets.write().unwrap(); + rowsets.push(rowset); + } + Ok(()) } /// Get all chunks of the table. - pub fn all_chunks(&self) -> StorageResult> { - let inner = self.inner.read().unwrap(); - Ok(inner.chunks.clone()) + pub async fn all_chunks(&self) -> StorageResult> { + let mut chunks = vec![]; + for rowset in &self.rowset_snapshot { + chunks.push(rowset.as_chunk().await?); + } + Ok(chunks) } } diff --git a/code/03-00/src/storage/rowset.rs b/code/03-00/src/storage/rowset.rs new file mode 100644 index 0000000..8961890 --- /dev/null +++ b/code/03-00/src/storage/rowset.rs @@ -0,0 +1,90 @@ +use std::path::{Path, PathBuf}; +use std::sync::Arc; + +use anyhow::anyhow; +use itertools::Itertools; + +use super::column::{decode_int32_column, encode_int32_column}; +use super::{err, StorageResult}; +use crate::array::{ArrayImpl, DataChunk}; +use crate::catalog::ColumnDesc; + +fn column_path(rowset_path: impl AsRef, column_id: usize) -> PathBuf { + rowset_path.as_ref().join(format!("{}.col", column_id)) +} + +#[derive(Clone)] +pub struct DiskRowset { + /// Columns of the current RowSet. + column_descs: Arc<[ColumnDesc]>, + + /// Id of the current rowset within the table. + #[allow(dead_code)] + rowset_id: u32, + + /// Base path of the RowSet + rowset_path: PathBuf, +} + +impl DiskRowset { + pub async fn as_chunk(&self) -> StorageResult { + let mut columns = vec![]; + for (idx, _) in self.column_descs.iter().enumerate() { + let column_path = column_path(&self.rowset_path, idx); + let data = tokio::fs::read(column_path).await.map_err(err)?; + columns.push(decode_int32_column(&data[..])?); + } + Ok(columns.into_iter().map(ArrayImpl::Int32).collect()) + } +} + +pub struct RowSetBuilder { + /// Columns of the current RowSet. + column_descs: Arc<[ColumnDesc]>, + + /// Buffer of all column data + buffer: Vec>, +} + +impl RowSetBuilder { + pub fn new(column_descs: Arc<[ColumnDesc]>) -> Self { + RowSetBuilder { + buffer: (0..column_descs.len()).map(|_| vec![]).collect_vec(), + column_descs, + } + } + + pub fn append(&mut self, chunk: DataChunk) -> StorageResult<()> { + for (idx, column) in chunk.arrays().iter().enumerate() { + if let ArrayImpl::Int32(column) = column { + encode_int32_column(column, &mut self.buffer[idx])?; + } else { + return Err(anyhow!("unsupported column type").into()); + } + } + Ok(()) + } + + pub async fn flush( + self, + rowset_id: u32, + rowset_path: impl AsRef, + ) -> StorageResult { + let rowset_path = rowset_path.as_ref(); + + tokio::fs::create_dir_all(rowset_path).await.map_err(err)?; + + for (idx, _) in self.column_descs.iter().enumerate() { + let column_path = column_path(rowset_path, idx); + tokio::fs::write(column_path, &self.buffer[idx]) + .await + .map_err(err)?; + } + + Ok(DiskRowset { + column_descs: self.column_descs, + rowset_id, + rowset_path: rowset_path.into(), + }) + } +} diff --git a/code/03-00/src/test.rs b/code/03-00/src/test.rs index 495722f..b55a2d3 100644 --- a/code/03-00/src/test.rs +++ b/code/03-00/src/test.rs @@ -1,21 +1,25 @@ use std::path::Path; +use tempfile::tempdir; use test_case::test_case; use crate::array::DataChunk; +use crate::storage::StorageOptions; use crate::types::DataValue; use crate::{Database, Error}; -#[test_case("01-01.slt")] -#[test_case("01-03.slt")] -#[test_case("01-05.slt")] -#[test_case("01-06.slt")] -#[test_case("01-07.slt")] +#[test_case("03-01.slt")] +#[test_case("03-02.slt")] fn test(name: &str) { init_logger(); let script = std::fs::read_to_string(Path::new("../sql").join(name)).unwrap(); - let mut tester = sqllogictest::Runner::new(Database::new()); - tester.run_script(&script).unwrap(); + let tempdir = tempdir().unwrap(); + let mut tester = sqllogictest::Runner::new(Database::new(StorageOptions { + base_path: tempdir.path().into(), + })); + if let Err(err) = tester.run_script(&script) { + panic!("{}", err); + } } impl sqllogictest::DB for Database { diff --git a/code/03-01/Cargo.toml b/code/03-01/Cargo.toml deleted file mode 100644 index 5d1f778..0000000 --- a/code/03-01/Cargo.toml +++ /dev/null @@ -1,25 +0,0 @@ -[package] -name = "risinglight-03-01" -version = "0.1.0" -edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - -[dependencies] -anyhow = "1" -bitvec = "1.0" -enum_dispatch = "0.3" -env_logger = "0.9" -futures = { version = "0.3", default-features = false, features = ["alloc"] } -futures-async-stream = { git = "https://github.com/taiki-e/futures-async-stream", rev = "944f407" } -itertools = "0.10" -log = "0.4" -prettytable-rs = { version = "0.8", default-features = false } -rustyline = "9" -sqlparser = "0.13" -thiserror = "1" -tokio = { version = "1", features = ["rt", "rt-multi-thread", "sync", "macros"] } -tokio-stream = "0.1" - -[dev-dependencies] -sqllogictest = "0.2" -test-case = "1.2" diff --git a/code/03-01/src/array b/code/03-01/src/array deleted file mode 120000 index a30992f..0000000 --- a/code/03-01/src/array +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/array \ No newline at end of file diff --git a/code/03-01/src/binder b/code/03-01/src/binder deleted file mode 120000 index 936cb6b..0000000 --- a/code/03-01/src/binder +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/binder/ \ No newline at end of file diff --git a/code/03-01/src/catalog b/code/03-01/src/catalog deleted file mode 120000 index e98b018..0000000 --- a/code/03-01/src/catalog +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/catalog/ \ No newline at end of file diff --git a/code/03-01/src/db.rs b/code/03-01/src/db.rs deleted file mode 120000 index ef0ca75..0000000 --- a/code/03-01/src/db.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/db.rs \ No newline at end of file diff --git a/code/03-01/src/executor b/code/03-01/src/executor deleted file mode 120000 index 991a7a7..0000000 --- a/code/03-01/src/executor +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/executor/ \ No newline at end of file diff --git a/code/03-01/src/lib.rs b/code/03-01/src/lib.rs deleted file mode 120000 index 84f6551..0000000 --- a/code/03-01/src/lib.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/lib.rs \ No newline at end of file diff --git a/code/03-01/src/logical_planner b/code/03-01/src/logical_planner deleted file mode 120000 index 80ac2b5..0000000 --- a/code/03-01/src/logical_planner +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/logical_planner \ No newline at end of file diff --git a/code/03-01/src/main.rs b/code/03-01/src/main.rs deleted file mode 100644 index abe8e95..0000000 --- a/code/03-01/src/main.rs +++ /dev/null @@ -1,40 +0,0 @@ -//! A simple interactive shell of the database. - -use risinglight_03_01::Database; -use rustyline::error::ReadlineError; -use rustyline::Editor; - -fn main() { - env_logger::init(); - - let db = Database::new(); - - let mut rl = Editor::<()>::new(); - loop { - match rl.readline("> ") { - Ok(line) => { - rl.add_history_entry(line.as_str()); - let ret = db.run(&line); - match ret { - Ok(chunks) => { - for chunk in chunks { - println!("{}", chunk); - } - } - Err(err) => println!("{}", err), - } - } - Err(ReadlineError::Interrupted) => { - println!("Interrupted"); - } - Err(ReadlineError::Eof) => { - println!("Exited"); - break; - } - Err(err) => { - println!("Error: {:?}", err); - break; - } - } - } -} diff --git a/code/03-01/src/parser.rs b/code/03-01/src/parser.rs deleted file mode 120000 index 41306c1..0000000 --- a/code/03-01/src/parser.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/parser.rs \ No newline at end of file diff --git a/code/03-01/src/physical_planner b/code/03-01/src/physical_planner deleted file mode 120000 index 844f2c0..0000000 --- a/code/03-01/src/physical_planner +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/physical_planner \ No newline at end of file diff --git a/code/03-01/src/storage/mod.rs b/code/03-01/src/storage/mod.rs deleted file mode 100644 index 7e380c1..0000000 --- a/code/03-01/src/storage/mod.rs +++ /dev/null @@ -1,118 +0,0 @@ -//! Persistent storage on disk. -//! -//! RisingLight's in-memory representation of data is very simple. Currently, -//! it is simple a vector of `DataChunk`. Upon insertion, users' data are -//! simply appended to the end of the vector. - -mod rowset; -mod table_transaction; - -use std::collections::HashMap; -use std::sync::{Arc, Mutex, RwLock}; - -use self::table_transaction::TableTransaction; -use crate::array::DataChunk; -use crate::catalog::TableRefId; - -/// The error type of storage operations. -#[derive(thiserror::Error, Debug)] -pub enum StorageError { - #[error("table not found: {0:?}")] - NotFound(TableRefId), -} - -/// A specialized `Result` type for storage operations. -pub type StorageResult = std::result::Result; - -pub type StorageRef = Arc; -pub type DiskTableRef = Arc; - -/// Persistent storage on disk. -pub struct DiskStorage { - tables: Mutex>, -} - -impl Default for DiskStorage { - fn default() -> Self { - Self::new() - } -} - -impl DiskStorage { - /// Create a new persistent storage on disk. - pub fn new() -> Self { - DiskStorage { - tables: Mutex::new(HashMap::new()), - } - } - - /// Add a table. - pub fn add_table(&self, id: TableRefId) -> StorageResult<()> { - let table = Arc::new(DiskTable::new(id)); - self.tables.lock().unwrap().insert(id, table); - Ok(()) - } - - /// Get a table. - pub fn get_table(&self, id: TableRefId) -> StorageResult { - self.tables - .lock() - .unwrap() - .get(&id) - .cloned() - .ok_or(StorageError::NotFound(id)) - } -} - -/// A table in in-memory engine. -pub struct DiskTable { - #[allow(dead_code)] - id: TableRefId, - inner: RwLock, -} - -#[derive(Default)] -struct DiskTableInner { - chunks: Vec, -} - -impl DiskTable { - fn new(id: TableRefId) -> Self { - Self { - id, - inner: RwLock::new(DiskTableInner::default()), - } - } - - #[allow(dead_code)] - async fn write(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), false, false).await?) - } - - #[allow(dead_code)] - async fn read(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), true, false).await?) - } - - #[allow(dead_code)] - async fn update(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), false, true).await?) - } - - /// Append a chunk to the table. - /// - /// This interface will be deprecated soon in this tutorial. - pub fn append(&self, chunk: DataChunk) -> StorageResult<()> { - let mut inner = self.inner.write().unwrap(); - inner.chunks.push(chunk); - Ok(()) - } - - /// Get all chunks of the table. - /// - /// This interface will be deprecated soon in this tutorial. - pub fn all_chunks(&self) -> StorageResult> { - let inner = self.inner.read().unwrap(); - Ok(inner.chunks.clone()) - } -} diff --git a/code/03-01/src/storage/rowset/mem_rowset.rs b/code/03-01/src/storage/rowset/mem_rowset.rs deleted file mode 100644 index 3b904a8..0000000 --- a/code/03-01/src/storage/rowset/mem_rowset.rs +++ /dev/null @@ -1,39 +0,0 @@ -#![allow(dead_code)] - -use std::sync::Arc; - -use itertools::Itertools; - -use crate::array::{ArrayBuilderImpl, DataChunk}; -use crate::catalog::ColumnCatalog; -use crate::storage::StorageResult; - -pub struct MemRowset { - builders: Vec, -} - -impl MemRowset { - pub fn new(columns: Arc<[ColumnCatalog]>) -> Self { - Self { - builders: columns - .iter() - .map(|column| ArrayBuilderImpl::with_capacity(0, column.desc().datatype())) - .collect_vec(), - } - } - - fn append(&mut self, columns: DataChunk) -> StorageResult<()> { - for (idx, column) in columns.arrays().iter().enumerate() { - self.builders[idx].append(column); - } - Ok(()) - } - - fn flush(self) -> StorageResult { - Ok(self - .builders - .into_iter() - .map(|builder| builder.finish()) - .collect::()) - } -} diff --git a/code/03-01/src/storage/rowset/mod.rs b/code/03-01/src/storage/rowset/mod.rs deleted file mode 100644 index 1675535..0000000 --- a/code/03-01/src/storage/rowset/mod.rs +++ /dev/null @@ -1,3 +0,0 @@ -mod mem_rowset; - -pub use mem_rowset::*; diff --git a/code/03-01/src/storage/table_transaction.rs b/code/03-01/src/storage/table_transaction.rs deleted file mode 100644 index 749bbbf..0000000 --- a/code/03-01/src/storage/table_transaction.rs +++ /dev/null @@ -1,56 +0,0 @@ -#![allow(dead_code)] - -use super::rowset::MemRowset; -use super::{DiskTableRef, StorageResult}; -use crate::array::DataChunk; - -/// [`TableTransaction`] records the state of a single table. All operations (insert, update, -/// delete) should go through [`TableTransaction`]. -pub struct TableTransaction { - mem_rowset: Option, - read_only: bool, - update: bool, - table: DiskTableRef, -} - -impl TableTransaction { - /// Start a [`WriteBatch`] - pub async fn start(table: DiskTableRef, read_only: bool, update: bool) -> StorageResult { - Ok(Self { - mem_rowset: None, - table, - update, - read_only, - }) - } - - /// Flush [`WriteBatch`] to some on-disk RowSets. - pub async fn flush(self) { - todo!() - } - - /// Add a [`DataChunk`] to the mem rowset - pub fn append(&self, _chunk: DataChunk) -> StorageResult<()> { - todo!() - } - - /// Delete a row from the table. - async fn delete(&mut self, _row_id: u64) -> StorageResult<()> { - todo!() - } - - /// Commit all changes in this transaction. - pub fn commit(self) -> StorageResult<()> { - todo!() - } - - /// Abort all changes in this transaction. - pub fn abort(self) -> StorageResult<()> { - todo!() - } - - /// Create an iterator on this table. - pub async fn scan(&self) { - todo!() - } -} diff --git a/code/03-01/src/test.rs b/code/03-01/src/test.rs deleted file mode 120000 index 05edac9..0000000 --- a/code/03-01/src/test.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/test.rs \ No newline at end of file diff --git a/code/03-01/src/types.rs b/code/03-01/src/types.rs deleted file mode 120000 index 88bb446..0000000 --- a/code/03-01/src/types.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/types.rs \ No newline at end of file diff --git a/code/03-02/Cargo.toml b/code/03-02/Cargo.toml deleted file mode 100644 index d5f856f..0000000 --- a/code/03-02/Cargo.toml +++ /dev/null @@ -1,26 +0,0 @@ -[package] -name = "risinglight-03-02" -version = "0.1.0" -edition = "2021" -# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html - -[dependencies] -anyhow = "1" -bitvec = "1.0" -bytes = "1" -enum_dispatch = "0.3" -env_logger = "0.9" -futures = { version = "0.3", default-features = false, features = ["alloc"] } -futures-async-stream = { git = "https://github.com/taiki-e/futures-async-stream", rev = "944f407" } -itertools = "0.10" -log = "0.4" -prettytable-rs = { version = "0.8", default-features = false } -rustyline = "9" -sqlparser = "0.13" -thiserror = "1" -tokio = { version = "1", features = ["rt", "rt-multi-thread", "sync", "macros"] } -tokio-stream = "0.1" - -[dev-dependencies] -sqllogictest = "0.2" -test-case = "1.2" diff --git a/code/03-02/src/array b/code/03-02/src/array deleted file mode 120000 index a30992f..0000000 --- a/code/03-02/src/array +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/array \ No newline at end of file diff --git a/code/03-02/src/binder b/code/03-02/src/binder deleted file mode 120000 index 936cb6b..0000000 --- a/code/03-02/src/binder +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/binder/ \ No newline at end of file diff --git a/code/03-02/src/catalog b/code/03-02/src/catalog deleted file mode 120000 index e98b018..0000000 --- a/code/03-02/src/catalog +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/catalog/ \ No newline at end of file diff --git a/code/03-02/src/db.rs b/code/03-02/src/db.rs deleted file mode 120000 index ef0ca75..0000000 --- a/code/03-02/src/db.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/db.rs \ No newline at end of file diff --git a/code/03-02/src/executor b/code/03-02/src/executor deleted file mode 120000 index 991a7a7..0000000 --- a/code/03-02/src/executor +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/executor/ \ No newline at end of file diff --git a/code/03-02/src/lib.rs b/code/03-02/src/lib.rs deleted file mode 120000 index 84f6551..0000000 --- a/code/03-02/src/lib.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/lib.rs \ No newline at end of file diff --git a/code/03-02/src/logical_planner b/code/03-02/src/logical_planner deleted file mode 120000 index 80ac2b5..0000000 --- a/code/03-02/src/logical_planner +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/logical_planner \ No newline at end of file diff --git a/code/03-02/src/main.rs b/code/03-02/src/main.rs deleted file mode 100644 index d8ff167..0000000 --- a/code/03-02/src/main.rs +++ /dev/null @@ -1,40 +0,0 @@ -//! A simple interactive shell of the database. - -use risinglight_03_02::Database; -use rustyline::error::ReadlineError; -use rustyline::Editor; - -fn main() { - env_logger::init(); - - let db = Database::new(); - - let mut rl = Editor::<()>::new(); - loop { - match rl.readline("> ") { - Ok(line) => { - rl.add_history_entry(line.as_str()); - let ret = db.run(&line); - match ret { - Ok(chunks) => { - for chunk in chunks { - println!("{}", chunk); - } - } - Err(err) => println!("{}", err), - } - } - Err(ReadlineError::Interrupted) => { - println!("Interrupted"); - } - Err(ReadlineError::Eof) => { - println!("Exited"); - break; - } - Err(err) => { - println!("Error: {:?}", err); - break; - } - } - } -} diff --git a/code/03-02/src/parser.rs b/code/03-02/src/parser.rs deleted file mode 120000 index 41306c1..0000000 --- a/code/03-02/src/parser.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/parser.rs \ No newline at end of file diff --git a/code/03-02/src/physical_planner b/code/03-02/src/physical_planner deleted file mode 120000 index 844f2c0..0000000 --- a/code/03-02/src/physical_planner +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/physical_planner \ No newline at end of file diff --git a/code/03-02/src/storage/block/mod.rs b/code/03-02/src/storage/block/mod.rs deleted file mode 100644 index 558e2e9..0000000 --- a/code/03-02/src/storage/block/mod.rs +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright 2022 RisingLight Project Authors. Licensed under Apache-2.0. - -//! Secondary's Block builders and iterators -//! -//! [`Block`] is the minimum managing unit in the storage engine. - -mod primitive_block_builder; -mod primitive_block_iterator; -use anyhow::{anyhow, Context, Result}; -use bytes::{Buf, BufMut, Bytes}; -pub use primitive_block_builder::*; -pub use primitive_block_iterator::*; - -use crate::array::Array; -use crate::storage::proto::*; - -/// A block is simply a [`Bytes`] array. -pub type Block = Bytes; - -/// Builds a block. All builders should implement the trait, while -/// ensuring that the format follows the block encoding scheme. -/// -/// In RisingLight, the block encoding scheme is as follows: -/// -/// ```plain -/// | block_type | cksum_type | cksum | data | -/// | 4B | 4B | 8B | variable | -/// ``` -pub trait BlockBuilder { - /// Append one data into the block. - fn append(&mut self, item: Option<&A::Item>); - - /// Get estimated size of block. Will be useful on runlength or compression encoding. - fn estimated_size(&self) -> usize; - - /// Check if we should finish the current block. If there is no item in the current - /// builder, this function must return `true`. - fn should_finish(&self, next_item: &Option<&A::Item>) -> bool; - - /// Finish a block and return encoded data. - fn finish(self) -> Vec; -} - -/// An iterator on a block. This iterator requires the block being pre-loaded in memory. -pub trait BlockIterator { - /// Get a batch from the block. A `0` return value means that this batch contains no - /// element. Some iterators might support exact size output. By using `expected_size`, - /// developers can get an array of NO MORE THAN the `expected_size`. - fn next_batch(&mut self, expected_size: Option, builder: &mut A::Builder) -> usize; - - /// Skip `cnt` items. - fn skip(&mut self, cnt: usize); - - /// Number of items remaining in this block - fn remaining_items(&self) -> usize; -} - -#[derive(Debug, Clone)] -pub struct BlockHeader { - pub block_type: BlockType, - pub checksum_type: ChecksumType, - pub checksum: u64, -} - -pub const BLOCK_HEADER_SIZE: usize = 4 + 4 + 8; - -impl BlockHeader { - pub fn encode(&self, buf: &mut impl BufMut) { - buf.put_i32(self.block_type.into()); - buf.put_i32(self.checksum_type.into()); - buf.put_u64(self.checksum); - } - - pub fn decode(&mut self, buf: &mut impl Buf) -> Result<()> { - if buf.remaining() < 4 + 4 + 8 { - return Err(anyhow!("expected 16 bytes")); - } - self.block_type = - BlockType::from_i32(buf.get_i32()).context("expected valid checksum type")?; - self.checksum_type = - ChecksumType::from_i32(buf.get_i32()).context("expected valid checksum type")?; - self.checksum = buf.get_u64(); - Ok(()) - } -} diff --git a/code/03-02/src/storage/block/primitive_block_builder.rs b/code/03-02/src/storage/block/primitive_block_builder.rs deleted file mode 100644 index 6dd6608..0000000 --- a/code/03-02/src/storage/block/primitive_block_builder.rs +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright 2022 RisingLight Project Authors. Licensed under Apache-2.0. - -use std::marker::PhantomData; - -use super::super::encode::PrimitiveFixedWidthEncode; -use super::BlockBuilder; - -/// Encodes fixed-width data into a block. The layout is simply an array of -/// little endian fixed-width data. -pub struct PlainPrimitiveBlockBuilder { - data: Vec, - target_size: usize, - _phantom: PhantomData, -} - -impl PlainPrimitiveBlockBuilder { - pub fn new(target_size: usize) -> Self { - let data = Vec::with_capacity(target_size); - Self { - data, - target_size, - _phantom: PhantomData, - } - } -} - -impl BlockBuilder for PlainPrimitiveBlockBuilder { - fn append(&mut self, item: Option<&T>) { - item.expect("nullable item found in non-nullable block builder") - .encode(&mut self.data); - } - - fn estimated_size(&self) -> usize { - self.data.len() - } - - fn should_finish(&self, _next_item: &Option<&T>) -> bool { - !self.data.is_empty() && self.estimated_size() + T::WIDTH > self.target_size - } - - fn finish(self) -> Vec { - self.data - } -} diff --git a/code/03-02/src/storage/block/primitive_block_iterator.rs b/code/03-02/src/storage/block/primitive_block_iterator.rs deleted file mode 100644 index 1a5d6fe..0000000 --- a/code/03-02/src/storage/block/primitive_block_iterator.rs +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright 2022 RisingLight Project Authors. Licensed under Apache-2.0. - -use std::marker::PhantomData; - -use super::super::PrimitiveFixedWidthEncode; -use super::{Block, BlockIterator}; -use crate::array::{Array, ArrayBuilder}; - -/// Scans one or several arrays from the block content. -pub struct PlainPrimitiveBlockIterator { - /// Block content - block: Block, - - /// Total count of elements in block - row_count: usize, - - /// Indicates the beginning row of the next batch - next_row: usize, - - _phantom: PhantomData, -} - -impl PlainPrimitiveBlockIterator { - pub fn new(block: Block, row_count: usize) -> Self { - Self { - block, - row_count, - next_row: 0, - _phantom: PhantomData, - } - } -} - -impl BlockIterator for PlainPrimitiveBlockIterator { - fn next_batch( - &mut self, - expected_size: Option, - builder: &mut ::Builder, - ) -> usize { - if self.next_row >= self.row_count { - return 0; - } - - // TODO(chi): error handling on corrupted block - - let mut cnt = 0; - let mut buffer = &self.block[self.next_row * T::WIDTH..]; - - loop { - if let Some(expected_size) = expected_size { - assert!(expected_size > 0); - if cnt >= expected_size { - break; - } - } - - if self.next_row >= self.row_count { - break; - } - - builder.push(Some(&T::decode(&mut buffer))); - cnt += 1; - self.next_row += 1; - } - - cnt - } - - fn skip(&mut self, cnt: usize) { - self.next_row += cnt; - } - - fn remaining_items(&self) -> usize { - self.row_count - self.next_row - } -} diff --git a/code/03-02/src/storage/encode.rs b/code/03-02/src/storage/encode.rs deleted file mode 100644 index c6ed52a..0000000 --- a/code/03-02/src/storage/encode.rs +++ /dev/null @@ -1,64 +0,0 @@ -// Copyright 2022 RisingLight Project Authors. Licensed under Apache-2.0. - -use bytes::{Buf, BufMut}; - -use crate::array::{Array, BoolArray, F64Array, I32Array}; - -/// Encode a primitive value into fixed-width buffer -pub trait PrimitiveFixedWidthEncode: Copy + Clone + 'static + Send + Sync { - /// Width of each element - const WIDTH: usize; - const DEAFULT_VALUE: &'static Self; - - type ArrayType: Array; - - /// Encode current primitive data to the end of an `Vec`. - fn encode(&self, buffer: &mut impl BufMut); - - /// Decode a data from a bytes array. - fn decode(buffer: &mut impl Buf) -> Self; -} - -impl PrimitiveFixedWidthEncode for bool { - const WIDTH: usize = std::mem::size_of::(); - const DEAFULT_VALUE: &'static bool = &false; - type ArrayType = BoolArray; - - fn encode(&self, buffer: &mut impl BufMut) { - buffer.put_u8(*self as u8) - } - - fn decode(buffer: &mut impl Buf) -> Self { - buffer.get_u8() != 0 - } -} - -impl PrimitiveFixedWidthEncode for i32 { - const WIDTH: usize = std::mem::size_of::(); - const DEAFULT_VALUE: &'static i32 = &0; - - type ArrayType = I32Array; - - fn encode(&self, buffer: &mut impl BufMut) { - buffer.put_i32_le(*self); - } - - fn decode(buffer: &mut impl Buf) -> Self { - buffer.get_i32_le() - } -} - -impl PrimitiveFixedWidthEncode for f64 { - const WIDTH: usize = std::mem::size_of::(); - const DEAFULT_VALUE: &'static f64 = &0.0; - - type ArrayType = F64Array; - - fn encode(&self, buffer: &mut impl BufMut) { - buffer.put_f64_le(*self); - } - - fn decode(buffer: &mut impl Buf) -> Self { - buffer.get_f64_le() - } -} diff --git a/code/03-02/src/storage/mod.rs b/code/03-02/src/storage/mod.rs deleted file mode 100644 index bf8e2ff..0000000 --- a/code/03-02/src/storage/mod.rs +++ /dev/null @@ -1,126 +0,0 @@ -//! Persistent storage on disk. -//! -//! RisingLight's in-memory representation of data is very simple. Currently, -//! it is simple a vector of `DataChunk`. Upon insertion, users' data are -//! simply appended to the end of the vector. - -mod block; -mod encode; -mod proto; -mod rowset; -mod table_transaction; - -use std::collections::HashMap; -use std::sync::{Arc, Mutex, RwLock}; - -use anyhow::anyhow; -pub use block::*; -pub use encode::*; -pub use proto::*; -pub use rowset::*; -pub use table_transaction::*; - -use crate::array::DataChunk; -use crate::catalog::TableRefId; - -/// The error type of storage operations. -#[derive(thiserror::Error, Debug)] -#[error("{0:?}")] -pub struct StorageError(anyhow::Error); - -/// A specialized `Result` type for storage operations. -pub type StorageResult = std::result::Result; - -pub type StorageRef = Arc; -pub type DiskTableRef = Arc; - -/// Persistent storage on disk. -pub struct DiskStorage { - tables: Mutex>, -} - -impl Default for DiskStorage { - fn default() -> Self { - Self::new() - } -} - -impl DiskStorage { - /// Create a new persistent storage on disk. - pub fn new() -> Self { - DiskStorage { - tables: Mutex::new(HashMap::new()), - } - } - - /// Add a table. - pub fn add_table(&self, id: TableRefId) -> StorageResult<()> { - let table = Arc::new(DiskTable::new(id)); - self.tables.lock().unwrap().insert(id, table); - Ok(()) - } - - /// Get a table. - pub fn get_table(&self, id: TableRefId) -> StorageResult { - self.tables - .lock() - .unwrap() - .get(&id) - .cloned() - .ok_or_else(|| anyhow!("table not found: {:?}", id)) - .map_err(StorageError) - } -} - -/// A table in in-memory engine. -pub struct DiskTable { - #[allow(dead_code)] - id: TableRefId, - inner: RwLock, -} - -#[derive(Default)] -struct DiskTableInner { - chunks: Vec, -} - -impl DiskTable { - fn new(id: TableRefId) -> Self { - Self { - id, - inner: RwLock::new(DiskTableInner::default()), - } - } - - #[allow(dead_code)] - async fn write(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), false, false).await?) - } - - #[allow(dead_code)] - async fn read(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), true, false).await?) - } - - #[allow(dead_code)] - async fn update(self: &Arc) -> StorageResult { - Ok(TableTransaction::start(self.clone(), false, true).await?) - } - - /// Append a chunk to the table. - /// - /// This interface will be deprecated soon in this tutorial. - pub fn append(&self, chunk: DataChunk) -> StorageResult<()> { - let mut inner = self.inner.write().unwrap(); - inner.chunks.push(chunk); - Ok(()) - } - - /// Get all chunks of the table. - /// - /// This interface will be deprecated soon in this tutorial. - pub fn all_chunks(&self) -> StorageResult> { - let inner = self.inner.read().unwrap(); - Ok(inner.chunks.clone()) - } -} diff --git a/code/03-02/src/storage/proto.rs b/code/03-02/src/storage/proto.rs deleted file mode 100644 index be74640..0000000 --- a/code/03-02/src/storage/proto.rs +++ /dev/null @@ -1,47 +0,0 @@ -//! On-disk representation of some enums - -use anyhow::{anyhow, Result}; - -#[derive(Debug, Clone, Copy)] -pub enum BlockType { - PrimitiveNonNull, -} - -impl BlockType { - pub fn from_i32(item: i32) -> Result { - match item { - 1 => Ok(Self::PrimitiveNonNull), - other => Err(anyhow!("invlid block type {}", other)), - } - } -} - -impl From for i32 { - fn from(ty: BlockType) -> Self { - match ty { - BlockType::PrimitiveNonNull => 1, - } - } -} - -#[derive(Debug, Clone, Copy)] -pub enum ChecksumType { - None, -} - -impl ChecksumType { - pub fn from_i32(item: i32) -> Result { - match item { - 1 => Ok(Self::None), - other => Err(anyhow!("invlid checksum type {}", other)), - } - } -} - -impl From for i32 { - fn from(ty: ChecksumType) -> Self { - match ty { - ChecksumType::None => 1, - } - } -} diff --git a/code/03-02/src/storage/rowset/mem_rowset.rs b/code/03-02/src/storage/rowset/mem_rowset.rs deleted file mode 100644 index 3b904a8..0000000 --- a/code/03-02/src/storage/rowset/mem_rowset.rs +++ /dev/null @@ -1,39 +0,0 @@ -#![allow(dead_code)] - -use std::sync::Arc; - -use itertools::Itertools; - -use crate::array::{ArrayBuilderImpl, DataChunk}; -use crate::catalog::ColumnCatalog; -use crate::storage::StorageResult; - -pub struct MemRowset { - builders: Vec, -} - -impl MemRowset { - pub fn new(columns: Arc<[ColumnCatalog]>) -> Self { - Self { - builders: columns - .iter() - .map(|column| ArrayBuilderImpl::with_capacity(0, column.desc().datatype())) - .collect_vec(), - } - } - - fn append(&mut self, columns: DataChunk) -> StorageResult<()> { - for (idx, column) in columns.arrays().iter().enumerate() { - self.builders[idx].append(column); - } - Ok(()) - } - - fn flush(self) -> StorageResult { - Ok(self - .builders - .into_iter() - .map(|builder| builder.finish()) - .collect::()) - } -} diff --git a/code/03-02/src/storage/rowset/mod.rs b/code/03-02/src/storage/rowset/mod.rs deleted file mode 100644 index 1675535..0000000 --- a/code/03-02/src/storage/rowset/mod.rs +++ /dev/null @@ -1,3 +0,0 @@ -mod mem_rowset; - -pub use mem_rowset::*; diff --git a/code/03-02/src/storage/table_transaction.rs b/code/03-02/src/storage/table_transaction.rs deleted file mode 100644 index 749bbbf..0000000 --- a/code/03-02/src/storage/table_transaction.rs +++ /dev/null @@ -1,56 +0,0 @@ -#![allow(dead_code)] - -use super::rowset::MemRowset; -use super::{DiskTableRef, StorageResult}; -use crate::array::DataChunk; - -/// [`TableTransaction`] records the state of a single table. All operations (insert, update, -/// delete) should go through [`TableTransaction`]. -pub struct TableTransaction { - mem_rowset: Option, - read_only: bool, - update: bool, - table: DiskTableRef, -} - -impl TableTransaction { - /// Start a [`WriteBatch`] - pub async fn start(table: DiskTableRef, read_only: bool, update: bool) -> StorageResult { - Ok(Self { - mem_rowset: None, - table, - update, - read_only, - }) - } - - /// Flush [`WriteBatch`] to some on-disk RowSets. - pub async fn flush(self) { - todo!() - } - - /// Add a [`DataChunk`] to the mem rowset - pub fn append(&self, _chunk: DataChunk) -> StorageResult<()> { - todo!() - } - - /// Delete a row from the table. - async fn delete(&mut self, _row_id: u64) -> StorageResult<()> { - todo!() - } - - /// Commit all changes in this transaction. - pub fn commit(self) -> StorageResult<()> { - todo!() - } - - /// Abort all changes in this transaction. - pub fn abort(self) -> StorageResult<()> { - todo!() - } - - /// Create an iterator on this table. - pub async fn scan(&self) { - todo!() - } -} diff --git a/code/03-02/src/test.rs b/code/03-02/src/test.rs deleted file mode 120000 index 05edac9..0000000 --- a/code/03-02/src/test.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/test.rs \ No newline at end of file diff --git a/code/03-02/src/types.rs b/code/03-02/src/types.rs deleted file mode 120000 index 88bb446..0000000 --- a/code/03-02/src/types.rs +++ /dev/null @@ -1 +0,0 @@ -../../03-00/src/types.rs \ No newline at end of file diff --git a/code/Cargo.toml b/code/Cargo.toml index c6d29a6..4aada09 100644 --- a/code/Cargo.toml +++ b/code/Cargo.toml @@ -9,6 +9,4 @@ members = [ "01-07", "01-08", "03-00", - "03-01", - "03-02", ] diff --git a/code/sql/03-01.slt b/code/sql/03-01.slt new file mode 100644 index 0000000..fb7575d --- /dev/null +++ b/code/sql/03-01.slt @@ -0,0 +1,14 @@ +# 03-01: Very simple storage test + +statement ok +CREATE TABLE t (a INT NOT NULL, b INT NOT NULL, c INT NOT NULL) + +statement ok +INSERT INTO t VALUES (1,10,100), (2,20,200), (3,30,300) + +query III +SELECT * FROM t +---- +1 10 100 +2 20 200 +3 30 300 diff --git a/code/sql/03-02.slt b/code/sql/03-02.slt new file mode 100644 index 0000000..f9037c2 --- /dev/null +++ b/code/sql/03-02.slt @@ -0,0 +1,17 @@ +# 03-01: RowSet tests + +statement ok +CREATE TABLE t (a INT NOT NULL, b INT NOT NULL, c INT NOT NULL) + +statement ok +INSERT INTO t VALUES (1,10,100) + +statement ok +INSERT INTO t VALUES (2,20,200), (3,30,300) + +query III rowsort +SELECT * FROM t +---- +1 10 100 +2 20 200 +3 30 300