diff --git a/rust/.gitignore b/rust/.gitignore index 2d00c94e4..7af3a0e31 100644 --- a/rust/.gitignore +++ b/rust/.gitignore @@ -3,4 +3,5 @@ # SPDX-License-Identifier: Apache-2.0 /target/ -/.vscode/ \ No newline at end of file +/.vscode/ +.history diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 1de7866ca..ab8d398ed 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -302,7 +302,7 @@ dependencies = [ "arrow-select", "num", "regex", - "regex-syntax", + "regex-syntax 0.8.2", ] [[package]] @@ -949,7 +949,7 @@ dependencies = [ "hashbrown 0.14.3", "itertools 0.11.0", "log", - "regex-syntax", + "regex-syntax 0.8.2", ] [[package]] @@ -1072,6 +1072,15 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "env_logger" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95b3f3e67048839cb0d0781f445682a35113da7121f7c949db0e2be96a4fbece" +dependencies = [ + "log", +] + [[package]] name = "equivalent" version = "1.0.1" @@ -1624,15 +1633,17 @@ dependencies = [ "json", "lakesoul-io", "lakesoul-metadata", - "log", "object_store", "parquet", "prost", "proto", "rand", + "rand_chacha", "serde", "serde_json", + "test-log", "tokio", + "tracing", "uuid", ] @@ -1670,6 +1681,7 @@ dependencies = [ "tokio", "tokio-stream", "tokio-util", + "tracing", "url", "whoami", ] @@ -1695,9 +1707,14 @@ dependencies = [ "postgres-types", "prost", "proto", + "rand", + "rand_chacha", "serde_json", + "test-log", "tokio", "tokio-postgres", + "tracing", + "tracing-subscriber", "url", "uuid", ] @@ -1835,6 +1852,15 @@ dependencies = [ "pkg-config", ] +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + [[package]] name = "md-5" version = "0.10.6" @@ -1884,6 +1910,16 @@ version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + [[package]] name = "num" version = "0.4.1" @@ -2050,6 +2086,12 @@ version = "6.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2355d85b9a3786f481747ced0e0ff2ba35213a1f9bd406ed906554d7af805a1" +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + [[package]] name = "packed_simd" version = "0.3.9" @@ -2460,8 +2502,17 @@ checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" dependencies = [ "aho-corasick", "memchr", - "regex-automata", - "regex-syntax", + "regex-automata 0.4.3", + "regex-syntax 0.8.2", +] + +[[package]] +name = "regex-automata" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", ] [[package]] @@ -2472,9 +2523,15 @@ checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" dependencies = [ "aho-corasick", "memchr", - "regex-syntax", + "regex-syntax 0.8.2", ] +[[package]] +name = "regex-syntax" +version = "0.6.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" + [[package]] name = "regex-syntax" version = "0.8.2" @@ -2714,6 +2771,15 @@ dependencies = [ "digest", ] +[[package]] +name = "sharded-slab" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" +dependencies = [ + "lazy_static", +] + [[package]] name = "signal-hook" version = "0.3.17" @@ -2971,12 +3037,44 @@ dependencies = [ "winapi-util", ] +[[package]] +name = "test-log" +version = "0.2.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6159ab4116165c99fc88cce31f99fa2c9dbe08d3691cb38da02fc3b45f357d2b" +dependencies = [ + "env_logger", + "test-log-macros", + "tracing-subscriber", +] + +[[package]] +name = "test-log-macros" +version = "0.2.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ba277e77219e9eea169e8508942db1bf5d8a41ff2db9b20aab5a5aadc9fa25d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + [[package]] name = "textwrap" version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" +[[package]] +name = "thread_local" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3fdd6f064ccff2d6567adcb3873ca630700f00b5ad3f060c25b5dcfd9a4ce152" +dependencies = [ + "cfg-if", + "once_cell", +] + [[package]] name = "thrift" version = "0.17.0" @@ -3165,6 +3263,36 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", + "valuable", +] + +[[package]] +name = "tracing-log" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee855f1f400bd0e5c02d150ae5de3840039a3f54b025156404e34c23c03f47c3" +dependencies = [ + "log", + "once_cell", + "tracing-core", +] + +[[package]] +name = "tracing-subscriber" +version = "0.3.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad0f048c97dbd9faa9b7df56362b8ebcaa52adb06b498c050d2f4e32f90a7a8b" +dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", + "sharded-slab", + "smallvec", + "thread_local", + "tracing", + "tracing-core", + "tracing-log", ] [[package]] @@ -3267,6 +3395,12 @@ dependencies = [ "syn 2.0.48", ] +[[package]] +name = "valuable" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" + [[package]] name = "version_check" version = "0.9.4" diff --git a/rust/Cargo.toml b/rust/Cargo.toml index a5b38b3ea..d773c74f1 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -4,11 +4,11 @@ [workspace] members = [ - "lakesoul-metadata", - "lakesoul-metadata-c", - "proto", - "lakesoul-io", - "lakesoul-io-c", + "lakesoul-metadata", + "lakesoul-metadata-c", + "proto", + "lakesoul-io", + "lakesoul-io-c", "lakesoul-datafusion" ] resolver = "2" @@ -16,18 +16,18 @@ resolver = "2" [profile.release] [workspace.dependencies] -datafusion = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-33-parquet-prefetch"} +datafusion = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-33-parquet-prefetch" } datafusion-common = { git = "https://github.com/lakesoul-io/arrow-datafusion.git", branch = "datafusion-33-parquet-prefetch" } arrow = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } arrow-schema = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } -arrow-array = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred"} +arrow-array = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } arrow-buffer = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } -parquet = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred"} +parquet = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } object_store = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "object_store_0.7_opt", features = ["aws", "http"] } tokio-stream = "0.1.9" tokio = { version = "1", features = ["full"] } -tokio-util = { version = "0.7", features = ["io", "compat"]} +tokio-util = { version = "0.7", features = ["io", "compat"] } derivative = "2.2.0" atomic_refcell = "0.1.8" futures = "0.3" @@ -36,8 +36,9 @@ async-trait = "0.1" serde_json = "1.0" log = "^0.4" prost = "0.11" -uuid = { version = "1.4.0", features = ["v4", "fast-rng", "macro-diagnostics"]} -serde = { version = "1.0", features = ["derive", "std", "rc"]} +uuid = { version = "1.4.0", features = ["v4", "fast-rng", "macro-diagnostics"] } +serde = { version = "1.0", features = ["derive", "std", "rc"] } rand = "^0.8" bytes = "1.4.0" half = "^2.1" +tracing = "0.1.40" \ No newline at end of file diff --git a/rust/lakesoul-datafusion/Cargo.toml b/rust/lakesoul-datafusion/Cargo.toml index d38439300..1f82ae5bf 100644 --- a/rust/lakesoul-datafusion/Cargo.toml +++ b/rust/lakesoul-datafusion/Cargo.toml @@ -25,10 +25,13 @@ chrono = { version = "0.4", features = ["unstable-locales"] } json = "0.12" serde = { workspace = true } serde_json = { workspace = true } -log = { workspace = true } tokio = { workspace = true } rand = { workspace = true } bytes = { workspace = true } +tracing = "0.1.40" [dev-dependencies] -ctor = "0.2" \ No newline at end of file +ctor = "0.2" +test-log = { version = "0.2.14", features = ["trace"] } +rand = "0.8.5" +rand_chacha = "0.3.1" \ No newline at end of file diff --git a/rust/lakesoul-datafusion/src/catalog/lakesoul_catalog.rs b/rust/lakesoul-datafusion/src/catalog/lakesoul_catalog.rs new file mode 100644 index 000000000..255745787 --- /dev/null +++ b/rust/lakesoul-datafusion/src/catalog/lakesoul_catalog.rs @@ -0,0 +1,156 @@ +// SPDX-FileCopyrightText: 2024 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +use crate::catalog::LakeSoulNamespace; +use datafusion::catalog::schema::SchemaProvider; +use datafusion::catalog::CatalogProvider; +use datafusion::error::DataFusionError; +use datafusion::prelude::SessionContext; +use lakesoul_metadata::MetaDataClientRef; +use proto::proto::entity::Namespace; +use std::any::Any; +use std::fmt::{Debug, Formatter}; +use std::sync::{Arc, RwLock}; +use tokio::runtime::Handle; + +/// A metadata wrapper +/// may need a lock +pub struct LakeSoulCatalog { + metadata_client: MetaDataClientRef, + context: Arc, + catalog_lock: RwLock<()>, +} + +impl Debug for LakeSoulCatalog { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("LakeSoulCatalog{..}").finish() + } +} + +impl LakeSoulCatalog { + pub fn new(meta_data_client_ref: MetaDataClientRef, context: Arc) -> Self { + Self { + metadata_client: meta_data_client_ref, + context, + catalog_lock: RwLock::new(()), + } + } + pub fn metadata_client(&self) -> MetaDataClientRef { + self.metadata_client.clone() + } + pub fn context(&self) -> Arc { + self.context.clone() + } + + fn get_all_namespace(&self) -> crate::error::Result> { + let client = self.metadata_client.clone(); + futures::executor::block_on(async move { + Handle::current() + .spawn(async move { Ok(client.get_all_namespace().await?) }) + .await? + }) + } +} + +impl CatalogProvider for LakeSoulCatalog { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema_names(&self) -> Vec { + let _guard = self.catalog_lock.read(); + if let Ok(v) = self.get_all_namespace() { + v.into_iter().map(|np| np.namespace).collect() + } else { + vec![] + } + } + + fn schema(&self, name: &str) -> Option> { + let _guard = self.catalog_lock.read(); + match self.get_all_namespace() { + Ok(v) if v.iter().any(|np| np.namespace == name) => Some(Arc::new(LakeSoulNamespace::new( + self.metadata_client.clone(), + self.context.clone(), + name, + ))), + _ => None, + } + } + + /// Adds a new schema to this catalog. + /// + /// If a schema of the same name existed before, it is replaced in + /// the catalog and returned. + fn register_schema( + &self, + name: &str, + _schema: Arc, + ) -> lakesoul_io::lakesoul_io_config::Result>> { + let _guard = self.catalog_lock.write(); + let client = self.metadata_client.clone(); + let schema: Option> = { + match self.get_all_namespace() { + Ok(v) if v.iter().any(|np| np.namespace == name) => Some(Arc::new(LakeSoulNamespace::new( + self.metadata_client.clone(), + self.context.clone(), + name, + ))), + _ => None, + } + }; + // use default value + let np = Namespace { + namespace: name.into(), + properties: "{}".into(), + comment: "created by lakesoul-datafusion".into(), + domain: "public".into(), + }; + let _ = futures::executor::block_on(async move { + Handle::current() + .spawn(async move { client.create_namespace(np).await }) + .await + .expect("tokio join error in register schema") + }); + Ok(schema) + } + + /// Removes a schema from this catalog. Implementations of this method should return + /// errors if the schema exists but cannot be dropped. For example, in DataFusion's + /// default in-memory catalog, [`MemoryCatalogProvider`], a non-empty schema + /// will only be successfully dropped when `cascade` is true. + /// This is equivalent to how DROP SCHEMA works in PostgreSQL. + /// + /// Implementations of this method should return None if schema with `name` + /// does not exist. + fn deregister_schema( + &self, + _name: &str, + _cascade: bool, + ) -> lakesoul_io::lakesoul_io_config::Result>> { + // Not supported + // let _guard = self.catalog_lock.write(); + // let client = self.metadata_client.clone(); + // let schema: Option> = { + // match self.get_all_namespace() { + // Ok(v) if v.iter().any(|np| np.namespace == name) => Some(Arc::new(LakeSoulNamespace::new( + // self.metadata_client.clone(), + // self.context.clone(), + // name, + // ))), + // _ => None, + // } + // }; + // let namespace = name.to_string(); + // if let Some(s) = schema { + // if !s.table_names().is_empty() && !cascade { + // return Err(DataFusionError::External("can not delete".into())); + // } + // // delete all tables + // return Ok(Some(s)); + // } + // return Ok(None); + Err(DataFusionError::NotImplemented("Not supported".into())) + } +} \ No newline at end of file diff --git a/rust/lakesoul-datafusion/src/catalog/lakesoul_namespace.rs b/rust/lakesoul-datafusion/src/catalog/lakesoul_namespace.rs new file mode 100644 index 000000000..cd1659f75 --- /dev/null +++ b/rust/lakesoul-datafusion/src/catalog/lakesoul_namespace.rs @@ -0,0 +1,212 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +use crate::catalog::create_io_config_builder; +use crate::error::Result; +use async_trait::async_trait; +use datafusion::catalog::schema::SchemaProvider; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::TableProvider; +use datafusion::error::DataFusionError; +use datafusion::prelude::SessionContext; +use lakesoul_io::datasource::file_format::LakeSoulParquetFormat; +use lakesoul_io::datasource::listing::LakeSoulListingTable; +use lakesoul_metadata::error::LakeSoulMetaDataError; +use lakesoul_metadata::MetaDataClientRef; +use std::any::Any; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use tokio::runtime::Handle; +use tokio::sync::RwLock; +use tracing::debug; +use tracing::field::debug; + +/// A [`SchemaProvider`] that query pg to automatically discover tables. +/// Due to the restriction of datafusion 's api, "CREATE [EXTERNAL] Table ... " is not supported. +/// May have race condition +pub struct LakeSoulNamespace { + metadata_client: MetaDataClientRef, + context: Arc, + // primary key + namespace: String, + namespace_lock: Arc>, +} + +impl LakeSoulNamespace { + pub fn new(meta_data_client_ref: MetaDataClientRef, context: Arc, namespace: &str) -> Self { + Self { + metadata_client: meta_data_client_ref, + context, + namespace: namespace.to_string(), + namespace_lock: Arc::new(RwLock::new(())), + } + } + + pub fn metadata_client(&self) -> MetaDataClientRef { + self.metadata_client.clone() + } + + pub fn context(&self) -> Arc { + self.context.clone() + } + + pub fn namespace(&self) -> &str { + &self.namespace + } + + /// Dangerous + /// Should use transaction? + fn _delete_all_tables(&self) -> Result<()> { + unimplemented!() + } +} + +impl Debug for LakeSoulNamespace { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("LakeSoulNamespace{...}").finish() + } +} + +#[async_trait] +impl SchemaProvider for LakeSoulNamespace { + fn as_any(&self) -> &dyn Any { + self + } + + /// query table_name_id by namespace + fn table_names(&self) -> Vec { + let client = self.metadata_client.clone(); + let np = self.namespace.clone(); + let lock = self.namespace_lock.clone(); + futures::executor::block_on(async move { + Handle::current() + .spawn(async move { + let _guard = lock.read().await; + client.get_all_table_name_id_by_namespace(&np).await.unwrap() + }) + .await + .unwrap() + }) + .into_iter() + .map(|v| v.table_name) + .collect() + } + + /// Search table by name + /// return LakeSoulListing table + async fn table(&self, name: &str) -> Option> { + let _guard = self.namespace_lock.read().await; + if let Ok(_) = self + .metadata_client + .get_table_info_by_table_name(name, &self.namespace) + .await + { + let config; + if let Ok(config_builder) = + create_io_config_builder(self.metadata_client.clone(), Some(name), true, self.namespace()).await + { + config = config_builder.build(); + } else { + return None; + } + // Maybe should change + let file_format = Arc::new(LakeSoulParquetFormat::new( + Arc::new(ParquetFormat::new()), + config.clone(), + )); + if let Ok(table_provider) = LakeSoulListingTable::new_with_config_and_format( + &self.context.state(), + config, + file_format, + // care this + false, + ) + .await + { + debug!("get table provider success"); + return Some(Arc::new(table_provider)); + } + debug("get table provider fail"); + return None; + } else { + debug("get table provider fail"); + None + } + } + + /// If supported by the implementation, adds a new table to this schema. + /// If a table of the same name existed before, it returns "Table already exists" error. + #[allow(unused_variables)] + fn register_table( + &self, + name: String, + table: Arc, + ) -> lakesoul_io::lakesoul_io_config::Result>> { + // the type info of dyn TableProvider is not enough or use AST?????? + unimplemented!("schema provider does not support registering tables") + } + /// If supported by the implementation, removes an existing table from this schema and returns it. + /// If no table of that name exists, returns Ok(None). + #[allow(unused_variables)] + fn deregister_table(&self, name: &str) -> lakesoul_io::lakesoul_io_config::Result>> { + let client = self.metadata_client.clone(); + let table_name = name.to_string(); + let np = self.namespace.clone(); + let cxt = self.context.clone(); + let lock = self.namespace_lock.clone(); + futures::executor::block_on(async move { + Handle::current() + .spawn(async move { + // get table info + let _guard = lock.write().await; + match client.get_table_info_by_table_name(&table_name, &np).await { + Ok(table_info) => { + let config; + if let Ok(config_builder) = + create_io_config_builder(client.clone(), Some(&table_name), true, &np).await + { + config = config_builder.build(); + } else { + return Err(DataFusionError::External("get table provider config failed".into())); + } + // Maybe should change + let file_format = Arc::new(LakeSoulParquetFormat::new( + Arc::new(ParquetFormat::new()), + config.clone(), + )); + if let Ok(table_provider) = LakeSoulListingTable::new_with_config_and_format( + &cxt.state(), + config, + file_format, + // care this + false, + ) + .await + { + debug!("get table provider success"); + client + .delete_table_by_table_info_cascade(&table_info) + .await + .map_err(|_| DataFusionError::External("delete table info failed".into()))?; + return Ok(Some(Arc::new(table_provider) as Arc)); + } + debug("get table provider fail"); + return Err(DataFusionError::External("get table provider failed".into())); + } + Err(e) => match e { + LakeSoulMetaDataError::NotFound(_) => Ok(None), + _ => Err(DataFusionError::External("get table info failed".into())), + }, + } + }) + .await + .unwrap() + }) + } + + fn table_exist(&self, name: &str) -> bool { + // table name is primary key for `table_name_id` + self.table_names().into_iter().any(|s| s == name) + } +} diff --git a/rust/lakesoul-datafusion/src/catalog/mod.rs b/rust/lakesoul-datafusion/src/catalog/mod.rs index b92c7d8f2..968925f54 100644 --- a/rust/lakesoul-datafusion/src/catalog/mod.rs +++ b/rust/lakesoul-datafusion/src/catalog/mod.rs @@ -2,22 +2,30 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::collections::HashMap; +use datafusion::catalog::TableReference; +use std::env; +use std::fmt::Debug; use std::sync::Arc; use std::time::SystemTime; -use std::{env, path::PathBuf}; use lakesoul_io::lakesoul_io_config::{LakeSoulIOConfig, LakeSoulIOConfigBuilder}; use lakesoul_metadata::MetaDataClientRef; use proto::proto::entity::{CommitOp, DataCommitInfo, DataFileOp, FileOp, TableInfo, Uuid}; use crate::lakesoul_table::helpers::create_io_config_builder_from_table_info; -use crate::serialize::arrow_java::{schema_from_metadata_str, ArrowJavaSchema}; +use crate::serialize::arrow_java::ArrowJavaSchema; // use crate::transaction::TransactionMetaInfo; use crate::error::Result; // pub mod lakesoul_sink; // pub mod lakesoul_source; +mod lakesoul_catalog; +// used in catalog_test, but still say unused_imports, i think it is a bug about rust-lint. +// this is a workaround +#[cfg(test)] +pub use lakesoul_catalog::*; +mod lakesoul_namespace; +pub use lakesoul_namespace::*; #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] pub struct LakeSoulTableProperty { @@ -55,12 +63,13 @@ pub(crate) async fn create_io_config_builder( client: MetaDataClientRef, table_name: Option<&str>, fetch_files: bool, + namespace: &str, ) -> Result { if let Some(table_name) = table_name { - let table_info = client.get_table_info_by_table_name(table_name, "default").await?; + let table_info = client.get_table_info_by_table_name(table_name, namespace).await?; let data_files = if fetch_files { client - .get_data_files_by_table_name(table_name, vec![], "default") + .get_data_files_by_table_name(table_name, vec![], namespace) .await? } else { vec![] @@ -96,7 +105,10 @@ pub(crate) async fn commit_data( partitions: Vec<(String, String)>, files: &[String], ) -> Result<()> { - let table_name_id = client.get_table_name_id_by_table_name(table_name, "default").await?; + let table_ref = TableReference::from(table_name); + let table_name_id = client + .get_table_name_id_by_table_name(table_ref.table(), table_ref.schema().unwrap_or("default")) + .await?; client .commit_data_commit_info(DataCommitInfo { table_id: table_name_id.table_id, diff --git a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs index 852ab026f..ba4cf8ec6 100644 --- a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs +++ b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs @@ -16,6 +16,7 @@ use datafusion::physical_plan::common::AbortOnDropSingle; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{DisplayAs, DisplayFormatType, Distribution, Partitioning, SendableRecordBatchStream}; use datafusion::scalar::ScalarValue; +use datafusion::sql::TableReference; use datafusion::{ datasource::{ file_format::{parquet::ParquetFormat, FileFormat}, @@ -263,7 +264,7 @@ impl LakeSoulHashSinkExec { .iter() .map(|(column, value)| (column.to_string(), value.to_string())) .collect::>(); - commit_data(client.clone(), &table_name, partition_desc, &files) + commit_data(client.clone(), &table_name, partition_desc, files) .await .map_err(|e| DataFusionError::External(Box::new(e)))?; } @@ -292,14 +293,12 @@ impl ExecutionPlan for LakeSoulHashSinkExec { Partitioning::UnknownPartitioning(1) } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None + fn unbounded_output(&self, _children: &[bool]) -> Result { + Ok(_children[0]) } - fn benefits_from_input_partitioning(&self) -> Vec { - // DataSink is responsible for dynamically partitioning its - // own input at execution time. - vec![false] + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None } fn required_input_distribution(&self) -> Vec { @@ -329,6 +328,12 @@ impl ExecutionPlan for LakeSoulHashSinkExec { vec![false] } + fn benefits_from_input_partitioning(&self) -> Vec { + // DataSink is responsible for dynamically partitioning its + // own input at execution time. + vec![false] + } + fn children(&self) -> Vec> { vec![self.input.clone()] } @@ -343,17 +348,13 @@ impl ExecutionPlan for LakeSoulHashSinkExec { })) } - fn unbounded_output(&self, _children: &[bool]) -> Result { - Ok(_children[0]) - } - /// Execute the plan and return a stream of `RecordBatch`es for /// the specified partition. fn execute(&self, partition: usize, context: Arc) -> Result { if partition != 0 { - return Err(DataFusionError::NotImplemented(format!( - "FileSinkExec can only be called on partition 0!" - ))); + return Err(DataFusionError::NotImplemented( + "FileSinkExec can only be called on partition 0!".to_string(), + )); } let num_input_partitions = self.input.output_partitioning().partition_count(); @@ -380,10 +381,14 @@ impl ExecutionPlan for LakeSoulHashSinkExec { join_handles.push(sink_task); } + let table_ref = TableReference::Partial { + schema: self.table_info().table_namespace.clone().into(), + table: self.table_info().table_name.clone().into(), + }; let join_handle = AbortOnDropSingle::new(tokio::spawn(Self::wait_for_commit( join_handles, self.metadata_client(), - self.table_info().table_name.clone(), + table_ref.to_string(), partitioned_file_path_and_row_count, ))); @@ -403,7 +408,7 @@ impl ExecutionPlan for LakeSoulHashSinkExec { let stream = futures::stream::once(async move { match join_handle.await { Ok(Ok(count)) => Ok(make_count_batch(count)), - other => Ok(make_count_batch(u64::MAX)), + _other => Ok(make_count_batch(u64::MAX)), } }) .boxed(); diff --git a/rust/lakesoul-datafusion/src/datasource/table_provider.rs b/rust/lakesoul-datafusion/src/datasource/table_provider.rs index 090b6e26c..d06cd02d7 100644 --- a/rust/lakesoul-datafusion/src/datasource/table_provider.rs +++ b/rust/lakesoul-datafusion/src/datasource/table_provider.rs @@ -32,7 +32,7 @@ use super::file_format::LakeSoulMetaDataParquetFormat; /// /// # Features /// -/// 1. Merges schemas if the files have compatible but not indentical schemas +/// 1. Merges schemas if the files have compatible but not identical schemas /// /// 2. Hive-style partitioning support, where a path such as /// `/files/date=1/1/2022/data.parquet` is injected as a `date` column. diff --git a/rust/lakesoul-datafusion/src/error.rs b/rust/lakesoul-datafusion/src/error.rs index 89fd60069..62576cb38 100644 --- a/rust/lakesoul-datafusion/src/error.rs +++ b/rust/lakesoul-datafusion/src/error.rs @@ -3,6 +3,7 @@ // SPDX-License-Identifier: Apache-2.0 use std::{error::Error, fmt::Display, result, sync::Arc}; +use tokio::task::JoinError; use lakesoul_io::lakesoul_reader::{ArrowError, DataFusionError}; use lakesoul_metadata::error::LakeSoulMetaDataError; @@ -22,6 +23,7 @@ pub enum LakeSoulError { DataFusionError(DataFusionError), ArrowError(ArrowError), SerdeJsonError(serde_json::Error), + TokioJoinError(tokio::task::JoinError), Internal(String), } @@ -49,6 +51,12 @@ impl From for LakeSoulError { } } +impl From for LakeSoulError { + fn from(err: JoinError) -> Self { + Self::TokioJoinError(err) + } +} + impl Display for LakeSoulError { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match *self { @@ -56,6 +64,7 @@ impl Display for LakeSoulError { LakeSoulError::DataFusionError(ref desc) => write!(f, "DataFusion error: {desc}"), LakeSoulError::SerdeJsonError(ref desc) => write!(f, "serde_json error: {desc}"), LakeSoulError::ArrowError(ref desc) => write!(f, "arrow error: {desc}"), + LakeSoulError::TokioJoinError(ref desc) => write!(f, "tokio error: {desc}"), LakeSoulError::Internal(ref desc) => { write!( f, @@ -74,6 +83,7 @@ impl Error for LakeSoulError { LakeSoulError::DataFusionError(e) => Some(e), LakeSoulError::SerdeJsonError(e) => Some(e), LakeSoulError::ArrowError(e) => Some(e), + LakeSoulError::TokioJoinError(e) => Some(e), LakeSoulError::Internal(_) => None, } } diff --git a/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs b/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs index 601966ea4..ee21938b3 100644 --- a/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs +++ b/rust/lakesoul-datafusion/src/lakesoul_table/helpers.rs @@ -27,6 +27,6 @@ pub(crate) fn create_io_config_builder_from_table_info(table_info: Arc Vec<(String, ScalarValue)> { +pub fn get_columnar_value(_batch: &RecordBatch) -> Vec<(String, ScalarValue)> { vec![] } diff --git a/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs b/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs index 4d7e33417..00477248f 100644 --- a/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs +++ b/rust/lakesoul-datafusion/src/lakesoul_table/mod.rs @@ -6,13 +6,14 @@ pub mod helpers; use std::{ops::Deref, sync::Arc}; -use arrow::{datatypes::SchemaRef, util::pretty::print_batches}; +use arrow::datatypes::SchemaRef; use datafusion::{ dataframe::DataFrame, datasource::TableProvider, execution::context::{SessionContext, SessionState}, logical_expr::LogicalPlanBuilder, }; +use datafusion::sql::TableReference; use lakesoul_io::{lakesoul_io_config::create_session_context_with_planner, lakesoul_reader::RecordBatch}; use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; use proto::proto::entity::TableInfo; @@ -75,21 +76,21 @@ impl LakeSoulTable { pub async fn execute_upsert(&self, record_batch: RecordBatch) -> Result<()> { let client = Arc::new(MetaDataClient::from_env().await?); - let builder = create_io_config_builder(client, None, false).await?; + let builder = create_io_config_builder(client, None, false, self.table_namespace()).await?; let sess_ctx = create_session_context_with_planner(&mut builder.clone().build(), Some(LakeSoulQueryPlanner::new_ref()))?; let schema = record_batch.schema(); let logical_plan = LogicalPlanBuilder::insert_into( sess_ctx.read_batch(record_batch)?.into_unoptimized_plan(), - self.table_name().to_string(), + TableReference::partial(self.table_namespace().to_string(),self.table_name().to_string()), schema.deref(), false, )? .build()?; let dataframe = DataFrame::new(sess_ctx.state(), logical_plan); - let results = dataframe + let _results = dataframe // .explain(true, false)? .collect() .await?; @@ -99,7 +100,8 @@ impl LakeSoulTable { } pub async fn to_dataframe(&self, context: &SessionContext) -> Result { - let config_builder = create_io_config_builder(self.client(), Some(self.table_name()), true).await?; + let config_builder = + create_io_config_builder(self.client(), Some(self.table_name()), true, self.table_namespace()).await?; let provider = Arc::new( LakeSoulTableProvider::try_new(&context.state(), config_builder.build(), self.table_info(), false).await?, ); @@ -107,9 +109,10 @@ impl LakeSoulTable { } pub async fn as_sink_provider(&self, session_state: &SessionState) -> Result> { - let config_builder = create_io_config_builder(self.client(), Some(self.table_name()), false) - .await? - .with_prefix(self.table_info.table_path.clone()); + let config_builder = + create_io_config_builder(self.client(), Some(self.table_name()), false, self.table_namespace()) + .await? + .with_prefix(self.table_info.table_path.clone()); Ok(Arc::new( LakeSoulTableProvider::try_new(session_state, config_builder.build(), self.table_info(), true).await?, )) @@ -132,7 +135,11 @@ impl LakeSoulTable { } pub fn hash_bucket_num(&self) -> usize { - self.properties.hash_bucket_num.unwrap_or_else(|| 1) + self.properties.hash_bucket_num.unwrap_or(1) + } + + pub fn table_namespace(&self) -> &str { + &self.table_info.table_namespace } pub fn schema(&self) -> SchemaRef { diff --git a/rust/lakesoul-datafusion/src/lib.rs b/rust/lakesoul-datafusion/src/lib.rs index 102580e16..654a205ac 100644 --- a/rust/lakesoul-datafusion/src/lib.rs +++ b/rust/lakesoul-datafusion/src/lib.rs @@ -1,6 +1,10 @@ // SPDX-FileCopyrightText: 2023 LakeSoul Contributors // // SPDX-License-Identifier: Apache-2.0 +#![allow(dead_code)] +#![allow(clippy::type_complexity)] +// after finished. remove above attr +extern crate core; mod catalog; mod datasource; diff --git a/rust/lakesoul-datafusion/src/planner/physical_planner.rs b/rust/lakesoul-datafusion/src/planner/physical_planner.rs index f0d0b9c75..acd939a10 100644 --- a/rust/lakesoul-datafusion/src/planner/physical_planner.rs +++ b/rust/lakesoul-datafusion/src/planner/physical_planner.rs @@ -2,7 +2,6 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::ops::Deref; use std::sync::Arc; use arrow::datatypes::Schema; @@ -10,16 +9,16 @@ use arrow::datatypes::Schema; use datafusion::common::{DFSchema, SchemaExt}; use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; -use datafusion::logical_expr::{Expr, LogicalPlan, Partitioning as LogicalPartitioning, Repartition, Sort}; +use datafusion::logical_expr::{Expr, LogicalPlan}; use datafusion::physical_expr::PhysicalExpr; -use datafusion::physical_plan::repartition::RepartitionExec; + use datafusion::physical_plan::sorts::sort::SortExec; -use datafusion::physical_plan::{ExecutionPlan, Partitioning}; -use datafusion::physical_planner::{create_physical_sort_expr, DefaultPhysicalPlanner, PhysicalPlanner}; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; use async_trait::async_trait; -use datafusion::logical_expr::{DmlStatement, LogicalPlanBuilder, WriteOp}; +use datafusion::logical_expr::{DmlStatement, WriteOp}; use lakesoul_io::helpers::{create_hash_partitioning, create_sort_exprs}; use lakesoul_io::repartition::RepartitionByRangeAndHashExec; @@ -53,11 +52,14 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { .. }) => { let name = table_name.table(); + let schema = table_name.schema(); // let schema = session_state.schema_for_ref(table_name)?; - let lakesoul_table = LakeSoulTable::for_name(name).await.unwrap(); + let lakesoul_table = LakeSoulTable::for_namespace_and_name(schema.unwrap_or("default"), name) + .await + .unwrap(); match lakesoul_table.as_sink_provider(session_state).await { Ok(provider) => { - let physical_input = self.create_physical_plan(&input, session_state).await?; + let physical_input = self.create_physical_plan(input, session_state).await?; let physical_input = if lakesoul_table.primary_keys().is_empty() { if !lakesoul_table @@ -66,7 +68,7 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { { return Err(DataFusionError::Plan( // Return an error if schema of the input query does not match with the table schema. - format!("Inserting query must have the same schema with the table."), + "Inserting query must have the same schema with the table.".to_string(), )); } physical_input @@ -74,13 +76,13 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { let input_schema = physical_input.schema(); let input_dfschema = input.as_ref().schema(); let sort_expr = create_sort_exprs( - &lakesoul_table.primary_keys(), + lakesoul_table.primary_keys(), input_dfschema, &input_schema, session_state, )?; let hash_partitioning = create_hash_partitioning( - &lakesoul_table.primary_keys(), + lakesoul_table.primary_keys(), lakesoul_table.hash_bucket_num(), input_dfschema, &input_schema, @@ -125,10 +127,7 @@ impl PhysicalPlanner for LakeSoulPhysicalPlanner { input_schema: &Schema, session_state: &SessionState, ) -> Result> { - match expr { - _ => self - .default_planner - .create_physical_expr(expr, input_dfschema, input_schema, session_state), - } + self.default_planner + .create_physical_expr(expr, input_dfschema, input_schema, session_state) } } diff --git a/rust/lakesoul-datafusion/src/serialize/arrow_java.rs b/rust/lakesoul-datafusion/src/serialize/arrow_java.rs index 3d7160282..cc96fad77 100644 --- a/rust/lakesoul-datafusion/src/serialize/arrow_java.rs +++ b/rust/lakesoul-datafusion/src/serialize/arrow_java.rs @@ -349,10 +349,10 @@ impl From<&ArrowJavaField> for Field { ArrowJavaType::Decimal { precision, scale, - bit_width, + bit_width: _, } => DataType::Decimal128(*precision, *scale), ArrowJavaType::Date { unit } if unit == "DAY" => DataType::Date32, - ArrowJavaType::Date { unit } => DataType::Date64, + ArrowJavaType::Date { unit: _ } => DataType::Date64, ArrowJavaType::Time { bit_width, unit } => { let time_unit = match unit.as_str() { "SECOND" => TimeUnit::Second, diff --git a/rust/lakesoul-datafusion/src/test/catalog_tests.rs b/rust/lakesoul-datafusion/src/test/catalog_tests.rs new file mode 100644 index 000000000..5b0c55dd0 --- /dev/null +++ b/rust/lakesoul-datafusion/src/test/catalog_tests.rs @@ -0,0 +1,271 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +#[cfg(test)] +mod catalog_tests { + use crate::catalog::{LakeSoulCatalog, LakeSoulNamespace, LakeSoulTableProperty}; + use crate::lakesoul_table::LakeSoulTable; + use crate::serialize::arrow_java::ArrowJavaSchema; + use arrow::array::{ArrayRef, Int32Array, RecordBatch}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion::assert_batches_eq; + use datafusion::catalog::schema::SchemaProvider; + use datafusion::catalog::CatalogProvider; + use lakesoul_io::lakesoul_io_config::create_session_context; + use lakesoul_io::lakesoul_io_config::LakeSoulIOConfigBuilder; + use lakesoul_metadata::{MetaDataClient, MetaDataClientRef}; + use proto::proto::entity::{Namespace, TableInfo}; + use rand::distributions::Alphanumeric; + use rand::{thread_rng, Rng, SeedableRng}; + use rand_chacha::ChaCha8Rng; + use std::env; + use std::sync::Arc; + use test_log::test; + use tokio::runtime::Runtime; + use tracing::debug; + + fn create_batch_i32(names: Vec<&str>, values: Vec<&[i32]>) -> RecordBatch { + let values = values + .into_iter() + .map(|vec| Arc::new(Int32Array::from(Vec::from(vec))) as ArrayRef) + .collect::>(); + let iter = names + .into_iter() + .zip(values) + .map(|(name, array)| (name, array, true)) + .collect::>(); + RecordBatch::try_from_iter_with_nullable(iter).unwrap() + } + + async fn get_client() -> MetaDataClientRef { + Arc::new(MetaDataClient::from_env().await.unwrap()) + } + + fn random_namespace(hash_bucket_num: usize) -> Vec { + let mut rng = ChaCha8Rng::from_rng(thread_rng()).unwrap(); + (0..rng.gen_range(1..10)) + .map(|_| Namespace { + namespace: { + let mut v = String::with_capacity(5); + for _ in 0..5 { + v.push((&mut rng).gen_range('a'..'z')); + } + v + }, + properties: serde_json::to_string(&LakeSoulTableProperty { + hash_bucket_num: Some(hash_bucket_num), + }) + .unwrap(), + comment: "this is comment".to_string(), + domain: "public".to_string(), + }) + .collect() + } + + fn random_tables(nps: Vec, schema: SchemaRef) -> Vec<(Namespace, Vec)> { + let mut ret = Vec::with_capacity(nps.len()); + let mut rng = ChaCha8Rng::from_rng(thread_rng()).unwrap(); + let schema = serde_json::to_string::(&schema.into()).unwrap(); + for np in nps { + let n = rng.gen_range(1usize..10); + let mut v = Vec::with_capacity(n); + for _ in 0..n { + let table_name = { + let mut v = String::with_capacity(8); + for _ in 0..5 { + v.push((&mut rng).gen_range('a'..'z')); + } + v + }; + let path = format!("{}{}/{}", env::temp_dir().to_str().unwrap(), &np.namespace, &table_name); + v.push(TableInfo { + table_id: (&mut rng).sample_iter(&Alphanumeric).take(12).map(char::from).collect(), + table_namespace: np.namespace.clone(), + table_name, + table_path: format!("file://{}", path.clone()), + table_schema: schema.clone(), + properties: np.properties.clone(), + partitions: ";range,hash".to_string(), + domain: np.domain.clone(), + }) + } + ret.push((np, v)); + } + ret + } + + fn table_info(table_name: &str, namespace: &str, schema: SchemaRef) -> TableInfo { + let path = format!("{}{}/{}", env::temp_dir().to_str().unwrap(), namespace, table_name); + let schema = serde_json::to_string::(&schema.into()).unwrap(); + TableInfo { + table_id: "table_000000001".into(), + table_namespace: "hello".to_string(), + table_name: table_name.to_string(), + table_path: format!("file://{}", path), + table_schema: schema.clone(), + properties: "{}".into(), + partitions: ";range,hash".to_string(), + domain: "public".to_string(), + } + } + + #[test] + fn test_catalog_api() { + let rt = Runtime::new().unwrap(); + rt.block_on(async { + let client = Arc::new(MetaDataClient::from_env().await.unwrap()); + // insert data; + let batch = create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ); + let pks = vec!["range".to_string(), "hash".to_string()]; + let schema = SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )); + + let mut config = LakeSoulIOConfigBuilder::new() + .with_schema(schema.clone()) + .with_primary_keys(pks) + .build(); + + let sc = Arc::new(create_session_context(&mut config).unwrap()); + let data = random_tables(random_namespace(4), schema.clone()); + + let catalog = Arc::new(LakeSoulCatalog::new(client.clone(), sc.clone())); + let dummy_schema_provider = Arc::new(LakeSoulNamespace::new(client.clone(), sc.clone(), "dummy")); + // id, path, name must be unique + for (np, tables) in data.iter() { + // client.create_namespace(np.clone()).await.unwrap(); + let old = catalog + .register_schema(&np.namespace, dummy_schema_provider.clone()) + .unwrap(); + assert!(old.is_none()); + for t in tables { + client.create_table(t.clone()).await.unwrap(); + let lakesoul_table = LakeSoulTable::for_namespace_and_name(&np.namespace, &t.table_name) + .await + .unwrap(); + lakesoul_table.execute_upsert(batch.clone()).await.unwrap(); + } + } + assert!(sc.register_catalog("lakesoul", catalog.clone()).is_none()); + for (np, tables) in data.iter() { + let schema = LakeSoulNamespace::new(client.clone(), sc.clone(), &np.namespace); + let names = schema.table_names(); + debug!("{names:?}"); + assert_eq!(names.len(), tables.len()); + for name in names { + assert!(schema.table_exist(&name)); + assert!(schema.table(&name).await.is_some()); + assert!(schema.deregister_table(&name).unwrap().is_some()); + } + } + }); + } + + #[test] + fn test_catalog_sql() { + let rt = Runtime::new().unwrap(); + rt.block_on(async { + let client = Arc::new(MetaDataClient::from_env().await.unwrap()); + // insert data; + let batch = create_batch_i32( + vec!["range", "hash", "value"], + vec![&[20201101, 20201101, 20201101, 20201102], &[1, 2, 3, 4], &[1, 2, 3, 4]], + ); + let pks = vec!["range".to_string(), "hash".to_string()]; + let schema = SchemaRef::new(Schema::new( + ["range", "hash", "value"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect::>(), + )); + + let mut config = LakeSoulIOConfigBuilder::new() + .with_schema(schema.clone()) + .with_primary_keys(pks) + .build(); + + let sc = Arc::new(create_session_context(&mut config).unwrap()); + + let expected = &[ + "+----------+------+-------+", + "| range | hash | value |", + "+----------+------+-------+", + "| 20201101 | 1 | 1 |", + "| 20201101 | 2 | 2 |", + "| 20201101 | 3 | 3 |", + "| 20201102 | 4 | 4 |", + "+----------+------+-------+", + ]; + + let catalog = Arc::new(LakeSoulCatalog::new(client.clone(), sc.clone())); + { + let before = { + let sql = "show tables"; + let df = sc.sql(sql).await.unwrap(); + df.collect().await.unwrap() + }; + sc.register_catalog("lakesoul", catalog.clone()); + let after = { + let sql = "show tables"; + let df = sc.sql(sql).await.unwrap(); + df.collect().await.unwrap() + }; + assert_ne!(after, before); + } + let data = random_tables(random_namespace(4), schema.clone()); + for (np, tables) in data.iter() { + { + // create schema + let sql = format!("create schema lakesoul.{}", np.namespace); + let df = sc.sql(&sql).await.unwrap(); + df.collect().await.unwrap(); + let ret = client.get_namespace_by_namespace(&np.namespace).await.unwrap(); + assert_eq!(np.namespace, ret.namespace); + } + for t in tables { + client.create_table(t.clone()).await.unwrap(); + let lakesoul_table = LakeSoulTable::for_namespace_and_name(&np.namespace, &t.table_name) + .await + .unwrap(); + lakesoul_table.execute_upsert(batch.clone()).await.unwrap(); + } + } + for (np, tables) in data.iter() { + let schema = LakeSoulNamespace::new(client.clone(), sc.clone(), &np.namespace); + let names = schema.table_names(); + debug!("{names:?}"); + assert_eq!(names.len(), tables.len()); + for name in names { + assert!(schema.table(&name).await.is_some()); + { + // test select + let q = format!("select * from lakesoul.{}.{}", np.namespace, name); + let df = sc.sql(&q).await.unwrap(); + let record = df.collect().await.unwrap(); + assert_batches_eq!(expected, &record); + } + { + // test show columns + let q = format!("show columns from lakesoul.{}.{}", np.namespace, name); + let df = sc.sql(&q).await.unwrap(); + let record = df.collect().await.unwrap(); + assert!(record.len() > 0); + } + { + // drop table + let sql = format!("drop table lakesoul.{}.{}", np.namespace, name); + let df = sc.sql(&sql).await.unwrap(); + assert!(df.collect().await.is_ok()) + } + } + } + }); + } +} diff --git a/rust/lakesoul-datafusion/src/test/hash_tests.rs b/rust/lakesoul-datafusion/src/test/hash_tests.rs index b426f528a..65858bab2 100644 --- a/rust/lakesoul-datafusion/src/test/hash_tests.rs +++ b/rust/lakesoul-datafusion/src/test/hash_tests.rs @@ -7,7 +7,6 @@ mod hash_tests { #[test] fn hash_value_test() { - // let hash = "321".hash_one(HASH_SEED) as i32; // dbg!(hash); assert_eq!(1.hash_one(HASH_SEED) as i32, -559580957); @@ -20,7 +19,6 @@ mod hash_tests { assert_eq!(3u64.hash_one(HASH_SEED) as i32, 519220707); assert_eq!(4u64.hash_one(HASH_SEED) as i32, 1344313940); - assert_eq!(1.0f32.hash_one(HASH_SEED) as i32, -466301895); assert_eq!(2.0f32.hash_one(HASH_SEED) as i32, 1199227445); assert_eq!(3.0f32.hash_one(HASH_SEED) as i32, 1710391653); @@ -51,7 +49,5 @@ mod hash_tests { assert_eq!(false.hash_one(HASH_SEED) as i32, 933211791); assert_eq!(1065353216.hash_one(HASH_SEED) as i32, -466301895); - } - -} \ No newline at end of file +} diff --git a/rust/lakesoul-datafusion/src/test/insert_tests.rs b/rust/lakesoul-datafusion/src/test/insert_tests.rs index bbe7ca0af..1f9a09a00 100644 --- a/rust/lakesoul-datafusion/src/test/insert_tests.rs +++ b/rust/lakesoul-datafusion/src/test/insert_tests.rs @@ -56,7 +56,7 @@ mod insert_tests { ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; - let builder = create_io_config_builder(client, None, false).await?; + let builder = create_io_config_builder(client, None, false, "default").await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; @@ -510,7 +510,8 @@ mod insert_tests { ]).await } - #[tokio::test] + // #[tokio::test] + #[test_log::test(tokio::test)] async fn test_all_cases() -> Result<()> { test_insert_into_append().await?; test_insert_into_append_by_position().await?; diff --git a/rust/lakesoul-datafusion/src/test/mod.rs b/rust/lakesoul-datafusion/src/test/mod.rs index 738329082..439d1eea7 100644 --- a/rust/lakesoul-datafusion/src/test/mod.rs +++ b/rust/lakesoul-datafusion/src/test/mod.rs @@ -3,15 +3,19 @@ // SPDX-License-Identifier: Apache-2.0 use std::sync::Arc; +use tracing::debug; use lakesoul_metadata::MetaDataClient; +mod hash_tests; mod insert_tests; mod upsert_tests; -mod hash_tests; // mod compaction_tests; // mod streaming_tests; +mod catalog_tests; + +// in cargo test, this executed only once #[ctor::ctor] fn init() { tokio::runtime::Builder::new_multi_thread() @@ -21,6 +25,6 @@ fn init() { .block_on(async { let client = Arc::new(MetaDataClient::from_env().await.unwrap()); client.meta_cleanup().await.unwrap(); - println!("clean metadata"); + debug!("clean metadata"); }) } diff --git a/rust/lakesoul-datafusion/src/test/upsert_tests.rs b/rust/lakesoul-datafusion/src/test/upsert_tests.rs index e7c0c6baf..463bddb38 100644 --- a/rust/lakesoul-datafusion/src/test/upsert_tests.rs +++ b/rust/lakesoul-datafusion/src/test/upsert_tests.rs @@ -17,11 +17,9 @@ mod upsert_with_io_config_tests { use lakesoul_io::lakesoul_writer::SyncSendableMutableLakeSoulWriter; use tokio::runtime::Builder; - use arrow::array::Int64Array; - - enum str_or_i32 { - v1(&'static str), - v2(i32), + enum StrOrI32 { + V1(&'static str), + V2(i32), } fn init_table(batch: RecordBatch, table_name: &str, pks: Vec) -> LakeSoulIOConfigBuilder { @@ -77,7 +75,6 @@ mod upsert_with_io_config_tests { .duration_since(SystemTime::UNIX_EPOCH) .unwrap() .as_millis() - .to_string() ) .as_str(), ] @@ -150,27 +147,27 @@ mod upsert_with_io_config_tests { RecordBatch::try_from_iter_with_nullable(iter).unwrap() } - fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[str_or_i32]>) -> RecordBatch { + fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[StrOrI32]>) -> RecordBatch { let values = values .into_iter() .map(|vec| match vec[0] { - str_or_i32::v1(_) => { + StrOrI32::V1(_) => { let vec = vec - .into_iter() + .iter() .map(|val| match val { - str_or_i32::v1(v1) => Some(*v1), - str_or_i32::v2(v2) => None, + StrOrI32::V1(v1) => Some(*v1), + StrOrI32::V2(_v2) => None, }) .map(|val| val.unwrap()) .collect::>(); Arc::new(StringArray::from(vec)) as ArrayRef } - str_or_i32::v2(_) => { + StrOrI32::V2(_) => { let vec = vec - .into_iter() + .iter() .map(|val| match val { - str_or_i32::v1(v1) => None, - str_or_i32::v2(v2) => Some(v2), + StrOrI32::V1(_v1) => None, + StrOrI32::V2(v2) => Some(v2), }) .map(|val| *val.unwrap()) .collect::>(); @@ -453,7 +450,7 @@ mod upsert_with_io_config_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) .collect::>(), @@ -497,7 +494,7 @@ mod upsert_with_io_config_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) .collect::>(), @@ -533,7 +530,7 @@ mod upsert_with_io_config_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) .collect::>(), @@ -629,8 +626,8 @@ mod upsert_with_io_config_tests { } #[test] - fn test_upsert_without_range_parqitions_i32() { - let table_name = "upsert_without_range_parqitions"; + fn test_upsert_without_range_partitions_i32() { + let table_name = "upsert_without_range_partitions"; let builder = init_table( create_batch_i32( vec!["range", "hash", "value"], @@ -668,8 +665,8 @@ mod upsert_with_io_config_tests { } #[test] - fn test_upsert_with_multiple_range_and_hash_parqitions_i32() { - let table_name = "upsert_with_multiple_range_and_hash_parqitions"; + fn test_upsert_with_multiple_range_and_hash_partitions_i32() { + let table_name = "upsert_with_multiple_range_and_hash_partitions"; let builder = init_table( create_batch_i32( vec!["range1", "range2", "hash1", "hash2", "value"], @@ -1290,14 +1287,14 @@ mod upsert_with_io_config_tests { vec!["range", "v1", "hash1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], - &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V1("a1"), StrOrI32::V1("b1"), StrOrI32::V1("c1")], + &[StrOrI32::V2(1), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("a2"), StrOrI32::V1("b2"), StrOrI32::V1("c2")], + &[StrOrI32::V1("a"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -1305,14 +1302,14 @@ mod upsert_with_io_config_tests { vec!["range", "hash1", "v1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], - &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], - &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V2(1), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("a11"), StrOrI32::V1("b11"), StrOrI32::V1("c11")], + &[StrOrI32::V1("a22"), StrOrI32::V1("b22"), StrOrI32::V1("c22")], + &[StrOrI32::V1("a"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -1320,14 +1317,14 @@ mod upsert_with_io_config_tests { vec!["range", "v1", "hash1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], - &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], - &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V1("d1"), StrOrI32::V1("b111"), StrOrI32::V1("c111")], + &[StrOrI32::V2(4), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("d2"), StrOrI32::V1("b222"), StrOrI32::V1("c222")], + &[StrOrI32::V1("d"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -1343,7 +1340,7 @@ mod upsert_with_io_config_tests { check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { @@ -1372,7 +1369,7 @@ mod upsert_with_io_config_tests { check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "V1", "hash1", "V2", "hash2"] .iter() .map(|col| { if *col == "hash1" { @@ -1401,7 +1398,7 @@ mod upsert_with_io_config_tests { check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { @@ -1430,7 +1427,7 @@ mod upsert_with_io_config_tests { check_upsert_string_or_i32( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { @@ -1596,9 +1593,9 @@ mod upsert_with_metadata_tests { use crate::catalog::{create_io_config_builder, create_table}; - enum str_or_i32 { - v1(&'static str), - v2(i32), + enum StrOrI32 { + V1(&'static str), + V2(i32), } fn create_batch_i32(names: Vec<&str>, values: Vec<&[i32]>) -> RecordBatch { @@ -1643,7 +1640,7 @@ mod upsert_with_metadata_tests { } fn create_batch_string(names: Vec<&str>, values: Vec<&[&str]>) -> RecordBatch { - let mut values = values + let values = values .into_iter() .map(|vec| Arc::new(StringArray::from(Vec::from(vec))) as ArrayRef) .collect::>(); @@ -1655,27 +1652,27 @@ mod upsert_with_metadata_tests { RecordBatch::try_from_iter_with_nullable(iter).unwrap() } - fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[str_or_i32]>) -> RecordBatch { + fn create_batch_str_or_i32(names: Vec<&str>, values: Vec<&[StrOrI32]>) -> RecordBatch { let values = values .into_iter() .map(|vec| match vec[0] { - str_or_i32::v1(_) => { + StrOrI32::V1(_) => { let vec = vec - .into_iter() + .iter() .map(|val| match val { - str_or_i32::v1(v1) => Some(*v1), - str_or_i32::v2(v2) => None, + StrOrI32::V1(v1) => Some(*v1), + StrOrI32::V2(_v2) => None, }) .map(|val| val.unwrap()) .collect::>(); Arc::new(StringArray::from(vec)) as ArrayRef } - str_or_i32::v2(_) => { + StrOrI32::V2(_) => { let vec = vec - .into_iter() + .iter() .map(|val| match val { - str_or_i32::v1(v1) => None, - str_or_i32::v2(v2) => Some(v2), + StrOrI32::V1(_v1) => None, + StrOrI32::V2(v2) => Some(v2), }) .map(|val| *val.unwrap()) .collect::>(); @@ -1706,7 +1703,7 @@ mod upsert_with_metadata_tests { ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(batch).await?; - let builder = create_io_config_builder(client, None, false).await?; + let builder = create_io_config_builder(client, None, false, "default").await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; @@ -1759,7 +1756,7 @@ mod upsert_with_metadata_tests { ) -> Result<()> { let lakesoul_table = LakeSoulTable::for_name(table_name).await?; lakesoul_table.execute_upsert(batch).await?; - let builder = create_io_config_builder(client, None, false).await?; + let builder = create_io_config_builder(client, None, false, "default").await?; let sess_ctx = create_session_context(&mut builder.clone().build())?; let dataframe = lakesoul_table.to_dataframe(&sess_ctx).await?; @@ -1920,7 +1917,7 @@ mod upsert_with_metadata_tests { } async fn test_merge_different_columns_and_filter_partial_rows_i32() -> Result<()> { - let table_name = "merge-different_columns_and_filter_partial_rows_i32"; + let table_name = "merge_different_columns_and_filter_partial_rows_i32"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( @@ -1990,9 +1987,9 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() - .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .map(|col| Field::new(*col, DataType::Int32, true)) .collect::>(), ))), table_name, @@ -2045,9 +2042,9 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() - .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .map(|col| Field::new(*col, DataType::Int32, true)) .collect::>(), ))), table_name, @@ -2091,9 +2088,9 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() - .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .map(|col| Field::new(*col, DataType::Int32, true)) .collect::>(), ))), table_name, @@ -2161,9 +2158,9 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "hash", "value"] + ["range", "hash", "value"] .iter() - .map(|col| Field::new(*col, arrow::datatypes::DataType::Int32, true)) + .map(|col| Field::new(*col, DataType::Int32, true)) .collect::>(), ))), table_name, @@ -2222,8 +2219,8 @@ mod upsert_with_metadata_tests { Ok(()) } - async fn test_upsert_without_range_parqitions_i32() -> Result<()> { - let table_name = "upsert_without_range_parqitions"; + async fn test_upsert_without_range_partitions_i32() -> Result<()> { + let table_name = "upsert_without_range_partitions"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( @@ -2271,8 +2268,8 @@ mod upsert_with_metadata_tests { Ok(()) } - async fn test_upsert_with_multiple_range_and_hash_parqitions_i32() -> Result<()> { - let table_name = "upsert_with_multiple_range_and_hash_parqitions"; + async fn test_upsert_with_multiple_range_and_hash_partitions_i32() -> Result<()> { + let table_name = "upsert_with_multiple_range_and_hash_partitions"; let client = Arc::new(MetaDataClient::from_env().await?); init_table( @@ -3036,14 +3033,14 @@ mod upsert_with_metadata_tests { vec!["range", "v1", "hash1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v1("a1"), str_or_i32::v1("b1"), str_or_i32::v1("c1")], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("a2"), str_or_i32::v1("b2"), str_or_i32::v1("c2")], - &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V1("a1"), StrOrI32::V1("b1"), StrOrI32::V1("c1")], + &[StrOrI32::V2(1), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("a2"), StrOrI32::V1("b2"), StrOrI32::V1("c2")], + &[StrOrI32::V1("a"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -3051,14 +3048,14 @@ mod upsert_with_metadata_tests { vec!["range", "hash1", "v1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v2(1), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("a11"), str_or_i32::v1("b11"), str_or_i32::v1("c11")], - &[str_or_i32::v1("a22"), str_or_i32::v1("b22"), str_or_i32::v1("c22")], - &[str_or_i32::v1("a"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V2(1), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("a11"), StrOrI32::V1("b11"), StrOrI32::V1("c11")], + &[StrOrI32::V1("a22"), StrOrI32::V1("b22"), StrOrI32::V1("c22")], + &[StrOrI32::V1("a"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -3066,14 +3063,14 @@ mod upsert_with_metadata_tests { vec!["range", "v1", "hash1", "v2", "hash2"], vec![ &[ - str_or_i32::v1("range"), - str_or_i32::v1("range"), - str_or_i32::v1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), + StrOrI32::V1("range"), ], - &[str_or_i32::v1("d1"), str_or_i32::v1("b111"), str_or_i32::v1("c111")], - &[str_or_i32::v2(4), str_or_i32::v2(2), str_or_i32::v2(3)], - &[str_or_i32::v1("d2"), str_or_i32::v1("b222"), str_or_i32::v1("c222")], - &[str_or_i32::v1("d"), str_or_i32::v1("b"), str_or_i32::v1("c")], + &[StrOrI32::V1("d1"), StrOrI32::V1("b111"), StrOrI32::V1("c111")], + &[StrOrI32::V2(4), StrOrI32::V2(2), StrOrI32::V2(3)], + &[StrOrI32::V1("d2"), StrOrI32::V1("b222"), StrOrI32::V1("c222")], + &[StrOrI32::V1("d"), StrOrI32::V1("b"), StrOrI32::V1("c")], ], ); @@ -3085,9 +3082,9 @@ mod upsert_with_metadata_tests { .into_iter() .map(|name| { if name == "hash1" { - Field::new(name, arrow::datatypes::DataType::Int32, true) + Field::new(name, DataType::Int32, true) } else { - Field::new(name, arrow::datatypes::DataType::Utf8, true) + Field::new(name, DataType::Utf8, true) } }) .collect::>(), @@ -3103,13 +3100,13 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { - Field::new(*col, arrow::datatypes::DataType::Int32, true) + Field::new(*col, DataType::Int32, true) } else { - Field::new(*col, arrow::datatypes::DataType::Utf8, true) + Field::new(*col, DataType::Utf8, true) } }) .collect::>(), @@ -3133,13 +3130,13 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { - Field::new(*col, arrow::datatypes::DataType::Int32, true) + Field::new(*col, DataType::Int32, true) } else { - Field::new(*col, arrow::datatypes::DataType::Utf8, true) + Field::new(*col, DataType::Utf8, true) } }) .collect::>(), @@ -3163,13 +3160,13 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { - Field::new(*col, arrow::datatypes::DataType::Int32, true) + Field::new(*col, DataType::Int32, true) } else { - Field::new(*col, arrow::datatypes::DataType::Utf8, true) + Field::new(*col, DataType::Utf8, true) } }) .collect::>(), @@ -3193,13 +3190,13 @@ mod upsert_with_metadata_tests { check_upsert( RecordBatch::new_empty(SchemaRef::new(Schema::new( - vec!["range", "v1", "hash1", "v2", "hash2"] + ["range", "v1", "hash1", "v2", "hash2"] .iter() .map(|col| { if *col == "hash1" { - Field::new(*col, arrow::datatypes::DataType::Int32, true) + Field::new(*col, DataType::Int32, true) } else { - Field::new(*col, arrow::datatypes::DataType::Utf8, true) + Field::new(*col, DataType::Utf8, true) } }) .collect::>(), @@ -3378,8 +3375,8 @@ mod upsert_with_metadata_tests { test_merge_different_columns_and_filter_partial_rows_i32().await?; test_merge_one_file_with_empty_batch_i32().await?; test_merge_multi_files_with_empty_batch_i32().await?; - test_upsert_without_range_parqitions_i32().await?; - test_upsert_with_multiple_range_and_hash_parqitions_i32().await?; + test_upsert_without_range_partitions_i32().await?; + test_upsert_with_multiple_range_and_hash_partitions_i32().await?; test_filter_requested_columns_upsert_1_times_i32().await?; test_filter_requested_columns_upsert_2_times_i32().await?; test_filter_requested_columns_upsert_3_times_i32().await?; diff --git a/rust/lakesoul-io-c/src/lib.rs b/rust/lakesoul-io-c/src/lib.rs index 263a45697..debd62e21 100644 --- a/rust/lakesoul-io-c/src/lib.rs +++ b/rust/lakesoul-io-c/src/lib.rs @@ -632,7 +632,7 @@ pub extern "C" fn write_record_batch_blocked( writer: NonNull>, schema_addr: c_ptrdiff_t, array_addr: c_ptrdiff_t, -) -> *const c_char { +) -> *const c_char { unsafe { let writer = NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter); let mut ffi_array = FFI_ArrowArray::empty(); @@ -649,8 +649,7 @@ pub extern "C" fn write_record_batch_blocked( let result: lakesoul_io::Result<()> = result_fn(); match result { Ok(_) => std::ptr::null(), - Err(e) => - CString::new(format!("{}", e).as_str()).unwrap().into_raw(), + Err(e) => CString::new(format!("{}", e).as_str()).unwrap().into_raw(), } } } @@ -876,9 +875,9 @@ mod tests { } } - start_reader(reader, reader_callback.clone()); + start_reader(reader, reader_callback); unsafe { - assert_eq!(READER_FINISHED, false, "{:?}", READER_FAILED.as_ref()); + assert!(!READER_FINISHED, "{:?}", READER_FAILED.as_ref()); } let schema_ffi = FFI_ArrowSchema::empty(); @@ -930,7 +929,7 @@ mod tests { reader, std::ptr::addr_of!(schema_ptr) as c_ptrdiff_t, std::ptr::addr_of!(array_ptr) as c_ptrdiff_t, - reader_i32_callback.clone(), + reader_i32_callback, ); wait_callback(); @@ -951,7 +950,7 @@ mod tests { writer, std::ptr::addr_of!(schema_ptr) as c_ptrdiff_t, std::ptr::addr_of!(array_ptr) as c_ptrdiff_t, - writer_callback.clone(), + writer_callback, ); wait_callback(); @@ -1004,9 +1003,9 @@ mod tests { } } - start_reader(reader, reader_callback.clone()); + start_reader(reader, reader_callback); unsafe { - assert_eq!(READER_FINISHED, false, "{:?}", READER_FAILED.as_ref()); + assert!(!READER_FINISHED, "{:?}", READER_FAILED.as_ref()); } let schema_ffi = FFI_ArrowSchema::empty(); @@ -1062,7 +1061,7 @@ mod tests { reader, std::ptr::addr_of!(schema_ptr) as c_ptrdiff_t, std::ptr::addr_of!(array_ptr) as c_ptrdiff_t, - reader_i32_callback.clone(), + reader_i32_callback, ); wait_callback(); @@ -1083,7 +1082,7 @@ mod tests { writer, std::ptr::addr_of!(schema_ptr) as c_ptrdiff_t, std::ptr::addr_of!(array_ptr) as c_ptrdiff_t, - writer_callback.clone(), + writer_callback, ); wait_callback(); diff --git a/rust/lakesoul-io/Cargo.toml b/rust/lakesoul-io/Cargo.toml index a59b58648..9cba72585 100644 --- a/rust/lakesoul-io/Cargo.toml +++ b/rust/lakesoul-io/Cargo.toml @@ -11,12 +11,12 @@ edition = "2021" datafusion = { workspace = true } object_store = { workspace = true } -tokio-stream = {workspace = true } +tokio-stream = { workspace = true } tokio = { workspace = true } tokio-util = { workspace = true } derivative = { workspace = true } atomic_refcell = { workspace = true } -arrow = { workspace = true, features = ["prettyprint"]} +arrow = { workspace = true, features = ["prettyprint"] } arrow-schema = { workspace = true, features = ["serde"] } arrow-array = { workspace = true, features = ["chrono-tz"] } arrow-buffer = { workspace = true } @@ -33,11 +33,12 @@ hdrs = { git = "https://github.com/lakesoul-io/hdrs.git", branch = "main", featu lazy_static = "1.4.0" chrono = "0.4" serde_json = { workspace = true } -log = { workspace = true } +tracing = "0.1.40" proto = { path = "../proto" } parking_lot = "0.12.1" half = { workspace = true } +log = "0.4.20" [features] diff --git a/rust/lakesoul-io/src/datasource/file_format.rs b/rust/lakesoul-io/src/datasource/file_format.rs index b9579d8ae..fa382e90c 100644 --- a/rust/lakesoul-io/src/datasource/file_format.rs +++ b/rust/lakesoul-io/src/datasource/file_format.rs @@ -102,6 +102,7 @@ impl FileFormat for LakeSoulParquetFormat { for field in &conf.table_partition_cols { builder.push(Field::new(field.name(), field.data_type().clone(), false)); } + // files to read let (summary_conf, flatten_conf) = flatten_file_scan_config(state, self.parquet_format.clone(), conf, self.conf.primary_keys_slice()).await?; let projection = summary_conf.projection.clone(); diff --git a/rust/lakesoul-io/src/datasource/listing.rs b/rust/lakesoul-io/src/datasource/listing.rs index 1d10d654b..6e4d854d9 100644 --- a/rust/lakesoul-io/src/datasource/listing.rs +++ b/rust/lakesoul-io/src/datasource/listing.rs @@ -3,6 +3,7 @@ // SPDX-License-Identifier: Apache-2.0 use std::any::Any; +use std::fmt::{Debug, Formatter}; use std::sync::Arc; use async_trait::async_trait; @@ -18,16 +19,22 @@ use datafusion::{datasource::TableProvider, logical_expr::Expr}; use datafusion::logical_expr::{TableProviderFilterPushDown, TableType}; use datafusion_common::{FileTypeWriterOptions, Result}; +use tracing::{debug, instrument}; use crate::lakesoul_io_config::LakeSoulIOConfig; use crate::transform::uniform_schema; pub struct LakeSoulListingTable { listing_table: Arc, - lakesoul_io_config: LakeSoulIOConfig, } +impl Debug for LakeSoulListingTable { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("LakeSoulListingTable{..}").finish() + } +} + impl LakeSoulListingTable { pub fn new(listing_table: Arc, lakesoul_io_config: LakeSoulIOConfig) -> Self { Self { @@ -59,6 +66,7 @@ impl LakeSoulListingTable { // .with_table_partition_cols(table_partition_cols); let mut objects = vec![]; + for url in &table_paths { objects.push(store.head(url.prefix()).await?); } @@ -129,6 +137,19 @@ impl TableProvider for LakeSoulListingTable { TableType::Base } + #[instrument] + async fn scan( + &self, + state: &SessionState, + projection: Option<&Vec>, + // filters and limit can be used here to inject some push-down operations if needed + filters: &[Expr], + limit: Option, + ) -> Result> { + debug!("listing scan start"); + self.listing_table.scan(state, projection, filters, limit).await + } + fn supports_filters_pushdown(&self, filters: &[&Expr]) -> Result> { if self.lakesoul_io_config.primary_keys.is_empty() { if self.lakesoul_io_config.parquet_filter_pushdown { @@ -146,6 +167,7 @@ impl TableProvider for LakeSoulListingTable { .iter() .all(|col| self.lakesoul_io_config.primary_keys.contains(&col.name)) { + // use primary key Ok(TableProviderFilterPushDown::Inexact) } else { Ok(TableProviderFilterPushDown::Unsupported) @@ -158,17 +180,6 @@ impl TableProvider for LakeSoulListingTable { } } - async fn scan( - &self, - state: &SessionState, - projection: Option<&Vec>, - // filters and limit can be used here to inject some push-down operations if needed - filters: &[Expr], - limit: Option, - ) -> Result> { - self.listing_table.scan(state, projection, filters, limit).await - } - async fn insert_into( &self, state: &SessionState, @@ -201,9 +212,9 @@ impl TableProvider for LakeSoulListingTable { // .await?; // let file_groups = file_list_stream.try_collect::>().await?; - //if we are writing a single output_partition to a table backed by a single file - //we can append to that file. Otherwise, we can write new files into the directory - //adding new files to the listing table in order to insert to the table. + // if we are writing a single output_partition to a table backed by a single file + // we can append to that file. Otherwise, we can write new files into the directory + // adding new files to the listing table in order to insert to the table. let _input_partitions = input.output_partitioning().partition_count(); // let writer_mode = match self.options().insert_mode { // ListingTableInsertMode::AppendToFile => { diff --git a/rust/lakesoul-io/src/datasource/parquet_source.rs b/rust/lakesoul-io/src/datasource/parquet_source.rs index 13293fae7..77fcc8062 100644 --- a/rust/lakesoul-io/src/datasource/parquet_source.rs +++ b/rust/lakesoul-io/src/datasource/parquet_source.rs @@ -59,6 +59,7 @@ impl LakeSoulParquetProvider { pub async fn build_with_context(&self, context: &SessionContext) -> Result { let mut plans = vec![]; let mut full_schema = uniform_schema(self.config.schema.0.clone()).to_dfschema().unwrap(); + // one file is a table scan for i in 0..self.config.files.len() { let file = self.config.files[i].clone(); let df = context.read_parquet(file, Default::default()).await.unwrap(); @@ -108,27 +109,6 @@ impl TableProvider for LakeSoulParquetProvider { TableType::Base } - fn supports_filters_pushdown(&self, filters: &[&Expr]) -> Result> { - if self.config.primary_keys.is_empty() { - Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) - } else { - filters - .iter() - .map(|f| { - if let Ok(cols) = f.to_columns() { - if cols.iter().all(|col| self.config.primary_keys.contains(&col.name)) { - Ok(TableProviderFilterPushDown::Inexact) - } else { - Ok(TableProviderFilterPushDown::Unsupported) - } - } else { - Ok(TableProviderFilterPushDown::Unsupported) - } - }) - .collect() - } - } - async fn scan( &self, _state: &SessionState, @@ -158,8 +138,8 @@ impl TableProvider for LakeSoulParquetProvider { df.select(projected_cols)? }; - let phycical_plan = df.create_physical_plan().await.unwrap(); - inputs.push(phycical_plan); + let physical_plan = df.create_physical_plan().await.unwrap(); + inputs.push(physical_plan); } let full_schema = SchemaRef::new(Schema::new( @@ -185,6 +165,27 @@ impl TableProvider for LakeSoulParquetProvider { self.create_physical_plan(projections, full_schema, inputs).await } + + fn supports_filters_pushdown(&self, filters: &[&Expr]) -> Result> { + if self.config.primary_keys.is_empty() { + Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) + } else { + filters + .iter() + .map(|f| { + if let Ok(cols) = f.to_columns() { + if cols.iter().all(|col| self.config.primary_keys.contains(&col.name)) { + Ok(TableProviderFilterPushDown::Inexact) + } else { + Ok(TableProviderFilterPushDown::Unsupported) + } + } else { + Ok(TableProviderFilterPushDown::Unsupported) + } + }) + .collect() + } + } } #[derive(Debug, Clone)] @@ -239,7 +240,7 @@ impl LakeSoulParquetScanExec { } impl DisplayAs for LakeSoulParquetScanExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "LakeSoulParquetScanExec") } } @@ -329,7 +330,7 @@ pub fn merge_stream( } }) .collect::>(), - )); //merge_schema + )); // merge_schema let merge_ops = schema .fields() .iter() @@ -345,7 +346,7 @@ pub fn merge_stream( let merge_stream = SortedStreamMerger::new_from_streams( streams, merge_schema, - primary_keys.iter().map(String::clone).collect(), + primary_keys.iter().cloned().collect(), batch_size, merge_ops, ) @@ -395,6 +396,7 @@ pub async fn prune_filter_and_execute( })?; // column pruning let df = df.select(cols)?; + // return a stream df.execute_stream().await } } @@ -475,7 +477,7 @@ mod tests { async fn query(db: LakeSoulParquetProvider, filter: Option) -> Result<()> { // create local execution context let config = SessionConfig::default(); - let ctx = SessionContext::with_config(config); + let ctx = SessionContext::new_with_config(config); let db = db.build_with_context(&ctx).await.unwrap(); @@ -530,7 +532,7 @@ mod tests { let results = ctx .sql("SELECT * FROM lakesoul") .await? - .filter(col("value").gt(datafusion::prelude::Expr::Literal(ScalarValue::Int32(Some(1)))))? + .filter(col("value").gt(Expr::Literal(ScalarValue::Int32(Some(1)))))? .select(vec![col("hash")])? .explain(true, false)? .collect() diff --git a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs index 966e98e04..aa61b97a1 100644 --- a/rust/lakesoul-io/src/datasource/physical_plan/merge.rs +++ b/rust/lakesoul-io/src/datasource/physical_plan/merge.rs @@ -37,6 +37,7 @@ impl MergeParquetExec { metadata_size_hint: Option, io_config: LakeSoulIOConfig, ) -> Self { + // source file parquet scan let mut inputs = Vec::>::new(); for config in flatten_configs { let single_exec = Arc::new(ParquetExec::new(config, predicate.clone(), metadata_size_hint)); diff --git a/rust/lakesoul-io/src/filter/parser.rs b/rust/lakesoul-io/src/filter/parser.rs index dbc88dda7..732381179 100644 --- a/rust/lakesoul-io/src/filter/parser.rs +++ b/rust/lakesoul-io/src/filter/parser.rs @@ -2,7 +2,7 @@ // // SPDX-License-Identifier: Apache-2.0 -use arrow_schema::{DataType, Field, SchemaRef, Fields}; +use arrow_schema::{DataType, Field, Fields, SchemaRef}; use datafusion::logical_expr::Expr; use datafusion::prelude::col; use datafusion::scalar::ScalarValue; @@ -45,7 +45,7 @@ impl Parser { "lteq" => expr.lt_eq(value), _ => Expr::Literal(ScalarValue::Boolean(Some(true))), } - } + } } else { Expr::Literal(ScalarValue::Boolean(Some(false))) } @@ -155,35 +155,39 @@ impl Parser { fn qualified_expr(expr_str: &str, schema: SchemaRef) -> Option<(Expr, Arc)> { if let Ok(field) = schema.field_with_name(expr_str) { - Some((col(datafusion::common::Column::new_unqualified(expr_str)), Arc::new(field.clone()))) - + Some(( + col(datafusion::common::Column::new_unqualified(expr_str)), + Arc::new(field.clone()), + )) } else { let mut expr: Option<(Expr, Arc)> = None; let mut root = "".to_owned(); let mut sub_fields: &Fields = schema.fields(); - for expr_substr in expr_str.split('.').into_iter() { + for expr_substr in expr_str.split('.') { root = if root.is_empty() { expr_substr.to_owned() } else { format!("{}.{}", root, expr_substr) }; if let Some((_, field)) = sub_fields.find(&root) { - expr = if let Some((folding_exp, _)) = expr { Some((folding_exp.field(field.name()), field.clone())) } else { - Some((col(datafusion::common::Column::new_unqualified(field.name())), field.clone())) + Some(( + col(datafusion::common::Column::new_unqualified(field.name())), + field.clone(), + )) }; root = "".to_owned(); - + sub_fields = match field.data_type() { DataType::Struct(struct_sub_fields) => &struct_sub_fields, - _ => sub_fields + _ => sub_fields, }; - } + } } expr - } + } } #[cfg(test)] diff --git a/rust/lakesoul-io/src/hash_utils/mod.rs b/rust/lakesoul-io/src/hash_utils/mod.rs index 0a95bf94a..d3dbcf860 100644 --- a/rust/lakesoul-io/src/hash_utils/mod.rs +++ b/rust/lakesoul-io/src/hash_utils/mod.rs @@ -2,17 +2,15 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::sync::Arc; use std::io::Cursor; +use std::sync::Arc; use arrow::array::*; use arrow::datatypes::*; use arrow::{downcast_dictionary_array, downcast_primitive_array}; use arrow_buffer::i256; -use datafusion_common::cast::{ - as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array, -}; +use datafusion_common::cast::{as_boolean_array, as_generic_binary_array, as_primitive_array, as_string_array}; use datafusion_common::{DataFusionError, Result}; // use murmur3::murmur3_32; @@ -20,7 +18,7 @@ use datafusion_common::{DataFusionError, Result}; use self::spark_murmur3::spark_murmur3_32_for_bytes; mod spark_murmur3; -pub const HASH_SEED : u32 = 42; +pub const HASH_SEED: u32 = 42; // // Combines two hashes into one hash // #[inline] @@ -30,8 +28,10 @@ pub const HASH_SEED : u32 = 42; // } fn hash_null( - // random_state: &RandomState, - hashes_buffer: &'_ mut [u32], mul_col: bool) { + // random_state: &RandomState, + hashes_buffer: &'_ mut [u32], + mul_col: bool, +) { if mul_col { hashes_buffer.iter_mut().for_each(|hash| { // stable hash for null value @@ -104,7 +104,6 @@ impl HashValue for half::f16 { } } - impl HashValue for i256 { fn hash_one(&self, seed: u32) -> u32 { spark_murmur3_32_for_bytes(&mut Cursor::new(self.to_byte_slice()), seed).unwrap() @@ -123,7 +122,6 @@ impl HashValue for [u8] { } } - /// Builds hash values of PrimitiveArray and writes them into `hashes_buffer` /// If `rehash==true` this combines the previous hash value in the buffer /// with the new hash using `combine_hashes` @@ -228,9 +226,11 @@ fn hash_dictionary( // redundant hashing for large dictionary elements (e.g. strings) let values = Arc::clone(array.values()); let mut dict_hashes = vec![0; values.len()]; - create_hashes(&[values], - // random_state, - &mut dict_hashes)?; + create_hashes( + &[values], + // random_state, + &mut dict_hashes, + )?; // combine hash for each index in values if multi_col { @@ -261,9 +261,11 @@ where let offsets = array.value_offsets(); let nulls = array.nulls(); let mut values_hashes = vec![0u32; values.len()]; - create_hashes(&[values], - // random_state, - &mut values_hashes)?; + create_hashes( + &[values], + // random_state, + &mut values_hashes, + )?; if let Some(nulls) = nulls { for (i, (start, stop)) in offsets.iter().zip(offsets.iter().skip(1)).enumerate() { if nulls.is_valid(i) { @@ -320,21 +322,21 @@ pub fn create_hashes<'a>( hash_array_primitive(array, hashes_buffer, rehash) } DataType::Dictionary(_, _) => downcast_dictionary_array! { - array => hash_dictionary(array, - // random_state, + array => hash_dictionary(array, + // random_state, hashes_buffer, rehash)?, _ => unreachable!() } DataType::List(_) => { let array = as_list_array(array); - hash_list_array(array, - // random_state, + hash_list_array(array, + // random_state, hashes_buffer)?; } DataType::LargeList(_) => { let array = as_large_list_array(array); - hash_list_array(array, - // random_state, + hash_list_array(array, + // random_state, hashes_buffer)?; } _ => { @@ -349,7 +351,6 @@ pub fn create_hashes<'a>( Ok(hashes_buffer) } - #[cfg(test)] mod tests { use arrow::{array::*, datatypes::*}; @@ -368,9 +369,11 @@ mod tests { let array_ref = Arc::new(array); // let random_state = RandomState::with_seeds(0, 0, 0, 0); let hashes_buff = &mut vec![0; array_ref.len()]; - let hashes = create_hashes(&[array_ref], - // &random_state, - hashes_buff)?; + let hashes = create_hashes( + &[array_ref], + // &random_state, + hashes_buff, + )?; assert_eq!(hashes.len(), 4); Ok(()) } @@ -382,14 +385,18 @@ mod tests { // let random_state = RandomState::with_seeds(0, 0, 0, 0); let hashes_buff = &mut vec![0; f32_arr.len()]; - let hashes = create_hashes(&[f32_arr], - // &random_state, - hashes_buff)?; + let hashes = create_hashes( + &[f32_arr], + // &random_state, + hashes_buff, + )?; assert_eq!(hashes.len(), 4,); - let hashes = create_hashes(&[f64_arr], - // &random_state, - hashes_buff)?; + let hashes = create_hashes( + &[f64_arr], + // &random_state, + hashes_buff, + )?; assert_eq!(hashes.len(), 4,); Ok(()) @@ -397,17 +404,15 @@ mod tests { #[test] fn create_hashes_binary() -> Result<()> { - let byte_array = Arc::new(BinaryArray::from_vec(vec![ - &[4, 3, 2], - &[4, 3, 2], - &[1, 2, 3], - ])); + let byte_array = Arc::new(BinaryArray::from_vec(vec![&[4, 3, 2], &[4, 3, 2], &[1, 2, 3]])); // let random_state = RandomState::with_seeds(0, 0, 0, 0); let hashes_buff = &mut vec![0; byte_array.len()]; - let hashes = create_hashes(&[byte_array], - // &random_state, - hashes_buff)?; + let hashes = create_hashes( + &[byte_array], + // &random_state, + hashes_buff, + )?; assert_eq!(hashes.len(), 3,); Ok(()) @@ -416,15 +421,15 @@ mod tests { #[test] fn create_hashes_fixed_size_binary() -> Result<()> { let input_arg = vec![vec![1, 2], vec![5, 6], vec![5, 6]]; - let fixed_size_binary_array = - Arc::new(FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap()); + let fixed_size_binary_array = Arc::new(FixedSizeBinaryArray::try_from_iter(input_arg.into_iter()).unwrap()); // let random_state = RandomState::with_seeds(0, 0, 0, 0); let hashes_buff = &mut vec![0; fixed_size_binary_array.len()]; - let hashes = - create_hashes(&[fixed_size_binary_array], - // &random_state, - hashes_buff)?; + let hashes = create_hashes( + &[fixed_size_binary_array], + // &random_state, + hashes_buff, + )?; assert_eq!(hashes.len(), 3,); Ok(()) @@ -437,24 +442,25 @@ mod tests { let strings = [Some("foo"), None, Some("bar"), Some("foo"), None]; let string_array = Arc::new(strings.iter().cloned().collect::()); - let dict_array = Arc::new( - strings - .iter() - .cloned() - .collect::>(), - ); + let dict_array = Arc::new(strings.iter().cloned().collect::>()); // let random_state = RandomState::with_seeds(0, 0, 0, 0); let mut string_hashes = vec![0; strings.len()]; - create_hashes(&[string_array], - // &random_state, - &mut string_hashes).unwrap(); + create_hashes( + &[string_array], + // &random_state, + &mut string_hashes, + ) + .unwrap(); let mut dict_hashes = vec![0; strings.len()]; - create_hashes(&[dict_array], - // &random_state, - &mut dict_hashes).unwrap(); + create_hashes( + &[dict_array], + // &random_state, + &mut dict_hashes, + ) + .unwrap(); // Null values result in a zero hash, for (val, hash) in strings.iter().zip(string_hashes.iter()) { @@ -490,13 +496,15 @@ mod tests { None, Some(vec![Some(0), Some(1), Some(2)]), ]; - let list_array = - Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; + let list_array = Arc::new(ListArray::from_iter_primitive::(data)) as ArrayRef; // let random_state = RandomState::with_seeds(0, 0, 0, 0); let mut hashes = vec![0; list_array.len()]; - create_hashes(&[list_array], - // &random_state, - &mut hashes).unwrap(); + create_hashes( + &[list_array], + // &random_state, + &mut hashes, + ) + .unwrap(); assert_eq!(hashes[0], hashes[5]); assert_eq!(hashes[1], hashes[4]); assert_eq!(hashes[2], hashes[3]); @@ -510,19 +518,17 @@ mod tests { let strings2 = [Some("blarg"), Some("blah"), None]; let string_array = Arc::new(strings1.iter().cloned().collect::()); - let dict_array = Arc::new( - strings2 - .iter() - .cloned() - .collect::>(), - ); + let dict_array = Arc::new(strings2.iter().cloned().collect::>()); // let random_state = RandomState::with_seeds(0, 0, 0, 0); let mut one_col_hashes = vec![0; strings1.len()]; - create_hashes(&[dict_array.clone()], - // &random_state, - &mut one_col_hashes).unwrap(); + create_hashes( + &[dict_array.clone()], + // &random_state, + &mut one_col_hashes, + ) + .unwrap(); let mut two_col_hashes = vec![0; strings1.len()]; create_hashes( @@ -536,6 +542,5 @@ mod tests { assert_eq!(two_col_hashes.len(), 3); assert_ne!(one_col_hashes, two_col_hashes); - } } diff --git a/rust/lakesoul-io/src/hash_utils/spark_murmur3.rs b/rust/lakesoul-io/src/hash_utils/spark_murmur3.rs index b8b1f2d47..cce49db40 100644 --- a/rust/lakesoul-io/src/hash_utils/spark_murmur3.rs +++ b/rust/lakesoul-io/src/hash_utils/spark_murmur3.rs @@ -32,7 +32,6 @@ where } } - const C1: u32 = 0x85eb_ca6b; const C2: u32 = 0xc2b2_ae35; const R1: u32 = 16; diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index ebc6c0ce6..5681bd81d 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -280,10 +280,7 @@ pub fn register_s3_object_store(url: &Url, config: &LakeSoulIOConfig, runtime: & ))) .map_err(|e| External(Box::new(e)))?; let endpoint_s = endpoint_url.to_string(); - endpoint = endpoint_s - .strip_suffix('/') - .map(|s| s.to_string()) - .or(Some(endpoint_s)); + endpoint = endpoint_s.strip_suffix('/').map(|s| s.to_string()).or(Some(endpoint_s)); } } } @@ -415,12 +412,15 @@ pub fn create_session_context_with_planner( let mut sess_conf = SessionConfig::default() .with_batch_size(config.batch_size) .with_parquet_pruning(true) - .with_prefetch(config.prefetch_size); + .with_prefetch(config.prefetch_size) + .with_information_schema(true) + .with_create_default_catalog_and_schema(true); sess_conf.options_mut().optimizer.enable_round_robin_repartition = false; // if true, the record_batches poll from stream become unordered sess_conf.options_mut().optimizer.prefer_hash_join = false; //if true, panicked at 'range end out of bounds' sess_conf.options_mut().execution.parquet.pushdown_filters = config.parquet_filter_pushdown; sess_conf.options_mut().execution.target_partitions = 1; + // sess_conf.options_mut().catalog.default_catalog = "lakesoul".into(); let runtime = RuntimeEnv::new(RuntimeConfig::new())?; @@ -451,15 +451,20 @@ pub fn create_session_context_with_planner( SessionState::new_with_config_rt(sess_conf, Arc::new(runtime)) }; // only keep projection/filter rules as others are unnecessary - let physical_opt_rules = state.physical_optimizers().iter().filter_map(|r| { - // this rule is private mod in datafusion, so we use name to filter out it - if r.name() == "ProjectionPushdown" { - Some(r.clone()) - } else { - None - } - }).collect(); - state = state.with_analyzer_rules(vec![]) + let physical_opt_rules = state + .physical_optimizers() + .iter() + .filter_map(|r| { + // this rule is private mod in datafusion, so we use name to filter out it + if r.name() == "ProjectionPushdown" { + Some(r.clone()) + } else { + None + } + }) + .collect(); + state = state + .with_analyzer_rules(vec![]) .with_optimizer_rules(vec![Arc::new(PushDownFilter {})]) .with_physical_optimizer_rules(physical_opt_rules); diff --git a/rust/lakesoul-io/src/lakesoul_reader.rs b/rust/lakesoul-io/src/lakesoul_reader.rs index 88273ad7b..76e62e834 100644 --- a/rust/lakesoul-io/src/lakesoul_reader.rs +++ b/rust/lakesoul-io/src/lakesoul_reader.rs @@ -338,7 +338,7 @@ mod tests { Some(rb) => { let num_rows = &rb.unwrap().num_rows(); unsafe { - ROW_CNT = ROW_CNT + num_rows; + ROW_CNT += num_rows; println!("{}", ROW_CNT); } diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index 38665371a..d55690005 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -181,9 +181,10 @@ impl MultiPartAsyncWriter { Err(e) => Err(DataFusionError::External(Box::new(e))), }?; + // get underlying multipart uploader let (multipart_id, async_writer) = object_store.put_multipart(&path).await?; let in_mem_buf = InMemBuf(Arc::new(AtomicRefCell::new(VecDeque::::with_capacity( - 16 * 1024 * 1024, + 16 * 1024 * 1024, // 16kb )))); let schema: SchemaRef = config.schema.0.clone(); @@ -222,6 +223,7 @@ impl MultiPartAsyncWriter { batch: RecordBatch, arrow_writer: &mut ArrowWriter, in_mem_buf: &mut InMemBuf, + // underlying writer writer: &mut Box, ) -> Result<()> { arrow_writer.write(&batch)?; @@ -347,7 +349,6 @@ impl SortAsyncWriter { let join_handle = tokio::task::spawn(async move { let mut err = None; while let Some(batch) = sorted_stream.next().await { - match batch { Ok(batch) => { async_writer.write_record_batch(batch).await?; @@ -355,7 +356,7 @@ impl SortAsyncWriter { // received abort signal Err(e) => { err = Some(e); - break + break; } } } @@ -364,11 +365,12 @@ impl SortAsyncWriter { match result { Ok(_) => match e { Internal(ref err_msg) if err_msg == "external abort" => Ok(()), - _ => Err(e) + _ => Err(e), }, - Err(abort_err) => { - Err(Internal(format!("Abort failed {:?}, previous error {:?}", abort_err, e))) - } + Err(abort_err) => Err(Internal(format!( + "Abort failed {:?}, previous error {:?}", + abort_err, e + ))), } } else { async_writer.flush_and_close().await?; @@ -391,17 +393,13 @@ impl AsyncBatchWriter for SortAsyncWriter { if let Some(err) = &self.err { return Err(Internal(format!("SortAsyncWriter alread failed with error {:?}", err))); } - let send_result = self.sorter_sender - .send(Ok(batch)) - .await; + let send_result = self.sorter_sender.send(Ok(batch)).await; match send_result { Ok(_) => Ok(()), // channel has been closed, indicating error happened during sort write Err(e) => { if let Some(join_handle) = self.join_handle.take() { - let result = join_handle - .await - .map_err(|e| DataFusionError::External(Box::new(e)))?; + let result = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; self.err = result.err(); Err(Internal(format!("Write to SortAsyncWriter failed: {:?}", self.err))) } else { @@ -416,9 +414,7 @@ impl AsyncBatchWriter for SortAsyncWriter { if let Some(join_handle) = self.join_handle { let sender = self.sorter_sender; drop(sender); - join_handle - .await - .map_err(|e| DataFusionError::External(Box::new(e)))? + join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? } else { Err(Internal("SortAsyncWriter has been aborted, cannot flush".to_string())) } @@ -433,9 +429,7 @@ impl AsyncBatchWriter for SortAsyncWriter { .await .map_err(|e| DataFusionError::External(Box::new(e)))?; drop(sender); - join_handle - .await - .map_err(|e| DataFusionError::External(Box::new(e)))? + join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? } else { // previouse error has already aborted writer Ok(()) @@ -445,6 +439,8 @@ impl AsyncBatchWriter for SortAsyncWriter { pub type SendableWriter = Box; +// inner is sort writer +// multipart writer pub struct SyncSendableMutableLakeSoulWriter { inner: Arc>, runtime: Arc, @@ -476,8 +472,10 @@ impl SyncSendableMutableLakeSoulWriter { let schema = writer.schema.clone(); let writer: Box = if !config.primary_keys.is_empty() { + // sort primary key table Box::new(SortAsyncWriter::try_new(writer, config, runtime.clone())?) } else { + // else multipart Box::new(writer) }; @@ -541,11 +539,11 @@ mod tests { use arrow::array::{ArrayRef, Int64Array}; use arrow::record_batch::RecordBatch; use arrow_array::Array; + use arrow_schema::{DataType, Field, Schema}; use datafusion::error::Result; use parquet::arrow::arrow_reader::ParquetRecordBatchReader; use std::fs::File; use std::sync::Arc; - use arrow_schema::{DataType, Field, Schema}; use tokio::runtime::Builder; #[test] @@ -729,9 +727,7 @@ mod tests { .with_max_row_group_size(250000); let read_conf = common_conf_builder .clone() - .with_files(vec![ - "large_file.snappy.parquet".to_string() - ]) + .with_files(vec!["large_file.snappy.parquet".to_string()]) .with_schema(Arc::new(Schema::new(vec![ Arc::new(Field::new("uuid", DataType::Utf8, false)), Arc::new(Field::new("ip", DataType::Utf8, false)), @@ -750,9 +746,7 @@ mod tests { let write_conf = common_conf_builder .clone() - .with_files(vec![ - "/home/chenxu/program/data/large_file_written.parquet".to_string(), - ]) + .with_files(vec!["/home/chenxu/program/data/large_file_written.parquet".to_string()]) .with_primary_key("uuid".to_string()) .with_schema(schema) .build(); diff --git a/rust/lakesoul-io/src/lib.rs b/rust/lakesoul-io/src/lib.rs index 2d615c2ec..3b218287c 100644 --- a/rust/lakesoul-io/src/lib.rs +++ b/rust/lakesoul-io/src/lib.rs @@ -4,6 +4,7 @@ pub mod datasource; pub mod filter; +pub mod hash_utils; pub mod helpers; pub mod lakesoul_io_config; pub mod lakesoul_reader; @@ -11,7 +12,6 @@ pub mod lakesoul_writer; mod projection; pub mod repartition; pub mod sorted_merge; -pub mod hash_utils; #[cfg(feature = "hdfs")] mod hdfs; diff --git a/rust/lakesoul-io/src/repartition/mod.rs b/rust/lakesoul-io/src/repartition/mod.rs index 15c7f6115..2e2ee3d1a 100644 --- a/rust/lakesoul-io/src/repartition/mod.rs +++ b/rust/lakesoul-io/src/repartition/mod.rs @@ -32,7 +32,7 @@ use arrow_array::{builder::UInt64Builder, ArrayRef, RecordBatch}; use futures::{FutureExt, Stream, StreamExt}; use tokio::task::JoinHandle; -use crate::{repartition::distributor_channels::partition_aware_channels, hash_utils::create_hashes}; +use crate::{hash_utils::create_hashes, repartition::distributor_channels::partition_aware_channels}; use self::distributor_channels::{DistributionReceiver, DistributionSender}; @@ -440,15 +440,8 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { self.input.schema() } - fn children(&self) -> Vec> { - vec![self.input.clone()] - } - - fn with_new_children(self: Arc, mut children: Vec>) -> Result> { - let repartition = - RepartitionByRangeAndHashExec::try_new(children.swap_remove(0), self.hash_partitioning.clone())?; - - Ok(Arc::new(repartition)) + fn output_partitioning(&self) -> Partitioning { + self.hash_partitioning.clone() } /// Specifies whether this plan generates an infinite stream of records. @@ -458,10 +451,6 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { Ok(children[0]) } - fn output_partitioning(&self) -> Partitioning { - self.hash_partitioning.clone() - } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { if self.maintains_input_order()[0] { self.input().output_ordering() @@ -475,6 +464,17 @@ impl ExecutionPlan for RepartitionByRangeAndHashExec { vec![self.input().output_partitioning().partition_count() <= 1] } + fn children(&self) -> Vec> { + vec![self.input.clone()] + } + + fn with_new_children(self: Arc, mut children: Vec>) -> Result> { + let repartition = + RepartitionByRangeAndHashExec::try_new(children.swap_remove(0), self.hash_partitioning.clone())?; + + Ok(Arc::new(repartition)) + } + fn execute(&self, partition: usize, context: Arc) -> Result { trace!("Start {}::execute for partition: {}", self.name(), partition); // lock mutexes diff --git a/rust/lakesoul-io/src/sorted_merge/merge_operator.rs b/rust/lakesoul-io/src/sorted_merge/merge_operator.rs index 0aba5c470..4d722bf89 100644 --- a/rust/lakesoul-io/src/sorted_merge/merge_operator.rs +++ b/rust/lakesoul-io/src/sorted_merge/merge_operator.rs @@ -89,10 +89,16 @@ impl MergeOperator { MergeOperator::UseLastNotNull => last_non_null(ranges), MergeOperator::SumAll => sum_all_with_primitive_type(data_type, ranges, append_array_data_builder), MergeOperator::SumLast => sum_last_with_primitive_type(data_type, ranges, append_array_data_builder), - MergeOperator::JoinedLastByComma => concat_last_with_string_type(ranges, append_array_data_builder, ','), - MergeOperator::JoinedLastBySemicolon => concat_last_with_string_type(ranges, append_array_data_builder, ';'), + MergeOperator::JoinedLastByComma => { + concat_last_with_string_type(ranges, append_array_data_builder, ',') + } + MergeOperator::JoinedLastBySemicolon => { + concat_last_with_string_type(ranges, append_array_data_builder, ';') + } MergeOperator::JoinedAllByComma => concat_all_with_string_type(ranges, append_array_data_builder, ','), - MergeOperator::JoinedAllBySemicolon => concat_all_with_string_type(ranges, append_array_data_builder, ';'), + MergeOperator::JoinedAllBySemicolon => { + concat_all_with_string_type(ranges, append_array_data_builder, ';') + } }, } } @@ -132,28 +138,70 @@ fn sum_all_with_primitive_type( ) -> MergeResult { match dt { DataType::UInt8 => { - sum_all_with_primitive_type_and_append_value!(UInt8Type, u8, UInt8Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + UInt8Type, + u8, + UInt8Builder, + append_array_data_builder, + ranges + ) } DataType::UInt16 => { - sum_all_with_primitive_type_and_append_value!(UInt16Type, u16, UInt16Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + UInt16Type, + u16, + UInt16Builder, + append_array_data_builder, + ranges + ) } DataType::UInt32 => { - sum_all_with_primitive_type_and_append_value!(UInt32Type, u32, UInt32Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + UInt32Type, + u32, + UInt32Builder, + append_array_data_builder, + ranges + ) } DataType::UInt64 => { - sum_all_with_primitive_type_and_append_value!(UInt64Type, u64, UInt64Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + UInt64Type, + u64, + UInt64Builder, + append_array_data_builder, + ranges + ) } DataType::Int8 => { sum_all_with_primitive_type_and_append_value!(Int8Type, i8, Int8Builder, append_array_data_builder, ranges) } DataType::Int16 => { - sum_all_with_primitive_type_and_append_value!(Int16Type, i16, Int16Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + Int16Type, + i16, + Int16Builder, + append_array_data_builder, + ranges + ) } DataType::Int32 => { - sum_all_with_primitive_type_and_append_value!(Int32Type, i32, Int32Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + Int32Type, + i32, + Int32Builder, + append_array_data_builder, + ranges + ) } DataType::Int64 => { - sum_all_with_primitive_type_and_append_value!(Int64Type, i64, Int64Builder, append_array_data_builder, ranges) + sum_all_with_primitive_type_and_append_value!( + Int64Type, + i64, + Int64Builder, + append_array_data_builder, + ranges + ) } DataType::Float32 => sum_all_with_primitive_type_and_append_value!( Float32Type, @@ -180,28 +228,70 @@ fn sum_last_with_primitive_type( ) -> MergeResult { match dt { DataType::UInt8 => { - sum_last_with_primitive_type_and_append_value!(UInt8Type, u8, UInt8Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + UInt8Type, + u8, + UInt8Builder, + append_array_data_builder, + ranges + ) } DataType::UInt16 => { - sum_last_with_primitive_type_and_append_value!(UInt16Type, u16, UInt16Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + UInt16Type, + u16, + UInt16Builder, + append_array_data_builder, + ranges + ) } DataType::UInt32 => { - sum_last_with_primitive_type_and_append_value!(UInt32Type, u32, UInt32Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + UInt32Type, + u32, + UInt32Builder, + append_array_data_builder, + ranges + ) } DataType::UInt64 => { - sum_last_with_primitive_type_and_append_value!(UInt64Type, u64, UInt64Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + UInt64Type, + u64, + UInt64Builder, + append_array_data_builder, + ranges + ) } DataType::Int8 => { sum_last_with_primitive_type_and_append_value!(Int8Type, i8, Int8Builder, append_array_data_builder, ranges) } DataType::Int16 => { - sum_last_with_primitive_type_and_append_value!(Int16Type, i16, Int16Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + Int16Type, + i16, + Int16Builder, + append_array_data_builder, + ranges + ) } DataType::Int32 => { - sum_last_with_primitive_type_and_append_value!(Int32Type, i32, Int32Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + Int32Type, + i32, + Int32Builder, + append_array_data_builder, + ranges + ) } DataType::Int64 => { - sum_last_with_primitive_type_and_append_value!(Int64Type, i64, Int64Builder, append_array_data_builder, ranges) + sum_last_with_primitive_type_and_append_value!( + Int64Type, + i64, + Int64Builder, + append_array_data_builder, + ranges + ) } DataType::Float32 => sum_last_with_primitive_type_and_append_value!( Float32Type, @@ -221,7 +311,6 @@ fn sum_last_with_primitive_type( } } - fn concat_all_with_string_type( ranges: &SmallVec<[SortKeyArrayRange; 4]>, append_array_data_builder: &mut Box, @@ -289,18 +378,16 @@ fn concat_last_with_string_type( break; } } - } else { - if !arr.is_null(range.end_row - 1) { - if !first { - res.push(delim); - } else { - first = false; - } - res.push_str(arr.value(range.end_row - 1)); + } else if !arr.is_null(range.end_row - 1) { + if !first { + res.push(delim); } else { - is_none = true; - break; + first = false; } + res.push_str(arr.value(range.end_row - 1)); + } else { + is_none = true; + break; } } match is_none { @@ -316,7 +403,6 @@ fn concat_last_with_string_type( } } - #[macro_export] macro_rules! sum_all_with_primitive_type_and_append_value { ($primitive_type_name:ty, $native_ty:ty, $primitive_builder_type:ty, $builder:ident, $ranges:ident) => {{ @@ -400,7 +486,6 @@ macro_rules! sum_last_with_primitive_type_and_append_value { }}; } - #[cfg(test)] mod tests { use arrow::array::{PrimitiveArray, TimestampMillisecondArray}; diff --git a/rust/lakesoul-io/src/sorted_merge/sort_key_range.rs b/rust/lakesoul-io/src/sorted_merge/sort_key_range.rs index ed80ef243..95a872925 100644 --- a/rust/lakesoul-io/src/sorted_merge/sort_key_range.rs +++ b/rust/lakesoul-io/src/sorted_merge/sort_key_range.rs @@ -14,8 +14,8 @@ use arrow::{ }; use smallvec::{smallvec, SmallVec}; -// A range in one arrow::record_batch::RecordBatch with same sorted primary key -// This is the unit to be sorted in min heap +/// A range in one arrow::record_batch::RecordBatch with same sorted primary key +/// This is the unit to be sorted in min heap pub struct SortKeyBatchRange { pub(crate) begin_row: usize, // begin row in this batch, included pub(crate) end_row: usize, // not included @@ -102,7 +102,7 @@ impl SortKeyBatchRange { current } - //create a SortKeyArrayRange with specific column index of SortKeyBatchRange + /// create a SortKeyArrayRange with specific column index of SortKeyBatchRange pub fn column(&self, idx: usize) -> SortKeyArrayRange { SortKeyArrayRange { begin_row: self.begin_row, @@ -187,8 +187,8 @@ impl Clone for SortKeyArrayRange { } } -// Multiple ranges with same sorted primary key from variant source record_batch. -// These ranges will be merged into ONE row of target record_batch finnally. +/// Multiple ranges with same sorted primary key from variant source record_batch. +/// These ranges will be merged into ONE row of target record_batch finally. #[derive(Debug, Clone)] pub struct SortKeyBatchRanges { // vector with length=column_num that holds a Vector of SortKeyArrayRange to be merged for each column diff --git a/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs b/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs index ea212e443..9789a1a86 100644 --- a/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs +++ b/rust/lakesoul-io/src/sorted_merge/sorted_stream_merger.rs @@ -374,9 +374,9 @@ mod tests { let merged_result = common::collect(Box::pin(merge_stream)).await.unwrap(); let mut all_rb = Vec::new(); - for i in 0..files.len() { + for file in &files { let stream = session_ctx - .read_parquet(files[i].as_str(), Default::default()) + .read_parquet(file.as_str(), Default::default()) .await .unwrap() .sort(vec![logical_col("int0").sort(true, true)]) @@ -407,7 +407,7 @@ mod tests { ); } - ///! merge a series of record batches into a table using use_last + // merge a series of record batches into a table using use_last fn merge_with_use_last(results: &[RecordBatch]) -> Result { let mut table = Table::new(); table.load_preset("||--+-++| ++++++"); @@ -865,6 +865,7 @@ mod tests { Arc::new(schema), vec![String::from("id")], 2, + // TODO SumLast? vec![ MergeOperator::UseLast, MergeOperator::SumAll, @@ -942,7 +943,7 @@ mod tests { #[tokio::test] async fn parquet_viewer() { let session_config = SessionConfig::default().with_batch_size(2); - let session_ctx = SessionContext::with_config(session_config); + let session_ctx = SessionContext::new_with_config(session_config); let stream = session_ctx .read_parquet( "part-00000-58928ac0-5640-486e-bb94-8990262a1797_00000.c000.parquet", diff --git a/rust/lakesoul-io/src/transform.rs b/rust/lakesoul-io/src/transform.rs index 8983e1c13..18d7ea6e3 100644 --- a/rust/lakesoul-io/src/transform.rs +++ b/rust/lakesoul-io/src/transform.rs @@ -11,7 +11,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::{ new_null_array, types::*, ArrayRef, BooleanArray, PrimitiveArray, RecordBatchOptions, StringArray, StructArray, }; -use arrow_schema::{DataType, Field, Schema, SchemaBuilder, SchemaRef, TimeUnit, FieldRef, Fields}; +use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaBuilder, SchemaRef, TimeUnit}; use datafusion::error::Result; use datafusion_common::DataFusionError::{ArrowError, External}; @@ -29,7 +29,7 @@ pub fn uniform_field(orig_field: &FieldRef) -> FieldRef { DataType::Struct(fields) => Arc::new(Field::new( orig_field.name(), DataType::Struct(Fields::from(fields.iter().map(uniform_field).collect::>())), - orig_field.is_nullable() + orig_field.is_nullable(), )), _ => orig_field.clone(), } @@ -38,11 +38,7 @@ pub fn uniform_field(orig_field: &FieldRef) -> FieldRef { /// adjust time zone to UTC pub fn uniform_schema(orig_schema: SchemaRef) -> SchemaRef { Arc::new(Schema::new( - orig_schema - .fields() - .iter() - .map(uniform_field) - .collect::>(), + orig_schema.fields().iter().map(uniform_field).collect::>(), )) } @@ -121,7 +117,7 @@ pub fn transform_record_batch( transform_arrays, &RecordBatchOptions::new().with_row_count(Some(num_rows)), ) - .map_err(ArrowError) + .map_err(ArrowError) } pub fn transform_array( @@ -151,10 +147,10 @@ pub fn transform_array( .with_timezone_opt(Some(target_tz)) .into_data(), }), - DataType::Struct(target_child_fileds) => { + DataType::Struct(target_child_fields) => { let orig_array = as_struct_array(&array); let mut child_array = vec![]; - target_child_fileds.iter().try_for_each(|field| -> Result<()> { + target_child_fields.iter().try_for_each(|field| -> Result<()> { match orig_array.column_by_name(field.name()) { Some(array) => { child_array.push(( diff --git a/rust/lakesoul-metadata/Cargo.toml b/rust/lakesoul-metadata/Cargo.toml index ee5bb4e30..37f857156 100644 --- a/rust/lakesoul-metadata/Cargo.toml +++ b/rust/lakesoul-metadata/Cargo.toml @@ -11,9 +11,9 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -postgres="0.19.5" -tokio-postgres = {version = "0.7.8", features=["default", "with-serde_json-1", "with-uuid-1", "array-impls"]} -postgres-types = {version = "0.2.5", features=["derive"]} +postgres = "0.19.5" +tokio-postgres = { version = "0.7.8", features = ["default", "with-serde_json-1", "with-uuid-1", "array-impls"] } +postgres-types = { version = "0.2.5", features = ["derive"] } tokio = { workspace = true } proto = { path = "../proto" } @@ -23,4 +23,12 @@ num_enum = "0.5.1" uuid = { workspace = true } serde_json = { workspace = true } url = { workspace = true } +tracing = { workspace = true } + + +[dev-dependencies] +test-log = "0.2.14" +tracing-subscriber = "0.3.18" +rand = "0.8.5" +rand_chacha = "0.3.1" diff --git a/rust/lakesoul-metadata/src/error.rs b/rust/lakesoul-metadata/src/error.rs index 8e38b6fd9..1d81a9543 100644 --- a/rust/lakesoul-metadata/src/error.rs +++ b/rust/lakesoul-metadata/src/error.rs @@ -30,6 +30,7 @@ pub enum LakeSoulMetaDataError { ProstEncodeError(prost::EncodeError), Other(GenericError), Internal(String), + NotFound(String), } impl From for LakeSoulMetaDataError { @@ -98,6 +99,7 @@ impl Display for LakeSoulMetaDataError { LakeSoulMetaDataError::Other(ref desc) => { write!(f, "Other error: {desc}") } + LakeSoulMetaDataError::NotFound(ref desc) => write!(f, "not found err:{desc}"), LakeSoulMetaDataError::Internal(ref desc) => { write!( f, @@ -120,6 +122,7 @@ impl Error for LakeSoulMetaDataError { LakeSoulMetaDataError::ProstDecodeError(e) => Some(e), LakeSoulMetaDataError::ProstEncodeError(e) => Some(e), LakeSoulMetaDataError::Other(e) => Some(e.as_ref()), + LakeSoulMetaDataError::NotFound(_) => None, LakeSoulMetaDataError::Internal(_) => None, } } diff --git a/rust/lakesoul-metadata/src/lib.rs b/rust/lakesoul-metadata/src/lib.rs index 81af2d5b5..f6b024096 100644 --- a/rust/lakesoul-metadata/src/lib.rs +++ b/rust/lakesoul-metadata/src/lib.rs @@ -17,6 +17,7 @@ pub use tokio::runtime::{Builder, Runtime}; use postgres_types::{FromSql, ToSql}; use tokio::spawn; pub use tokio_postgres::{Client, NoTls, Statement}; +use tokio_postgres::{Error, Row}; pub use metadata_client::{MetaDataClient, MetaDataClientRef}; @@ -55,7 +56,7 @@ impl DataFileOp { fn from_proto_data_file_op(data_file_op: &entity::DataFileOp) -> Self { DataFileOp { path: data_file_op.path.clone(), - file_op: proto::proto::entity::FileOp::from_i32(data_file_op.file_op) + file_op: entity::FileOp::from_i32(data_file_op.file_op) .unwrap() .as_str_name() .to_string(), @@ -67,7 +68,7 @@ impl DataFileOp { fn as_proto_data_file_op(&self) -> entity::DataFileOp { entity::DataFileOp { path: self.path.clone(), - file_op: proto::proto::entity::FileOp::from_str_name(self.file_op.as_str()).unwrap() as i32, + file_op: entity::FileOp::from_str_name(self.file_op.as_str()).unwrap() as i32, size: self.size, file_exist_cols: self.file_exist_cols.clone(), } @@ -165,29 +166,29 @@ async fn get_prepared_statement( let result = { let statement = match dao_type { // Select Namespace - DaoType::SelectNamespaceByNamespace => + DaoType::SelectNamespaceByNamespace => "select namespace, properties, comment, domain from namespace where namespace = $1::TEXT", - DaoType::ListNamespaces => + DaoType::ListNamespaces => "select namespace, properties, comment, domain from namespace", // Select TablePathId - DaoType::SelectTablePathIdByTablePath => + DaoType::SelectTablePathIdByTablePath => "select table_path, table_id, table_namespace, domain from table_path_id where table_path = $1::TEXT", - DaoType::ListAllTablePath => + DaoType::ListAllTablePath => "select table_path, table_id, table_namespace, domain from table_path_id", - DaoType::ListAllPathTablePathByNamespace => + DaoType::ListAllPathTablePathByNamespace => "select table_path from table_path_id where table_namespace = $1::TEXT ", // Select TableNameId - DaoType::SelectTableNameIdByTableName => + DaoType::SelectTableNameIdByTableName => "select table_name, table_id, table_namespace, domain from table_name_id where table_name = $1::TEXT and table_namespace = $2::TEXT", @@ -197,11 +198,11 @@ async fn get_prepared_statement( where table_namespace = $1::TEXT", // Select TableInfo - DaoType::SelectTableInfoByTableId => + DaoType::SelectTableInfoByTableId => "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain from table_info where table_id = $1::TEXT", - DaoType::SelectTableInfoByTableNameAndNameSpace => + DaoType::SelectTableInfoByTableNameAndNameSpace => "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain from table_info where table_name = $1::TEXT and table_namespace=$2::TEXT", @@ -209,7 +210,7 @@ async fn get_prepared_statement( "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain from table_info where table_path = $1::TEXT", - DaoType::SelectTableInfoByIdAndTablePath => + DaoType::SelectTableInfoByIdAndTablePath => "select table_id, table_name, table_path, table_schema, properties, partitions, table_namespace, domain from table_info where table_id = $1::TEXT and table_path=$2::TEXT", @@ -224,12 +225,12 @@ async fn get_prepared_statement( select table_id,partition_desc,max(version) from partition_info where table_id = $1::TEXT and partition_desc = $2::TEXT group by table_id, partition_desc) t left join partition_info m on t.table_id = m.table_id - and t.partition_desc = m.partition_desc and t.max = m.version", + and t.partition_desc = m.partition_desc and t.max = m.version", DaoType::ListPartitionByTableIdAndDesc => "select table_id, partition_desc, version, commit_op, snapshot, timestamp, expression, domain from partition_info where table_id = $1::TEXT and partition_desc = $2::TEXT ", - DaoType::ListPartitionByTableId => + DaoType::ListPartitionByTableId => "select m.table_id, t.partition_desc, m.version, m.commit_op, m.snapshot, m.expression, m.domain from ( select table_id,partition_desc,max(version) @@ -256,10 +257,10 @@ async fn get_prepared_statement( "select table_id, partition_desc, commit_id, file_ops, commit_op, timestamp, committed, domain from data_commit_info where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id = $3::UUID", - + // Insert - DaoType::InsertNamespace => + DaoType::InsertNamespace => "insert into namespace( namespace, properties, @@ -277,7 +278,7 @@ async fn get_prepared_statement( table_namespace, domain) values($1::TEXT, $2::TEXT, $3::TEXT, $4::TEXT, $5::JSON, $6::TEXT, $7::TEXT, $8::TEXT)", - DaoType::InsertTableNameId => + DaoType::InsertTableNameId => "insert into table_name_id( table_id, table_name, @@ -302,7 +303,7 @@ async fn get_prepared_statement( domain ) values($1::TEXT, $2::TEXT, $3::INT, $4::TEXT, $5::_UUID, $6::TEXT, $7::TEXT)", - DaoType::InsertDataCommitInfo => + DaoType::InsertDataCommitInfo => "insert into data_commit_info( table_id, partition_desc, @@ -316,7 +317,7 @@ async fn get_prepared_statement( values($1::TEXT, $2::TEXT, $3::UUID, $4::_data_file_op, $5::TEXT, $6::BIGINT, $7::BOOL, $8::TEXT)", // Query Scalar - DaoType::GetLatestTimestampFromPartitionInfo => + DaoType::GetLatestTimestampFromPartitionInfo => "select max(timestamp) as timestamp from partition_info where table_id = $1::TEXT and partition_desc = $2::TEXT", @@ -348,25 +349,24 @@ async fn get_prepared_statement( "delete from table_name_id where table_id = $1::TEXT", - DaoType::DeleteTableInfoByIdAndPath => + DaoType::DeleteTableInfoByIdAndPath => "delete from table_info where table_id = $1::TEXT and table_path = $2::TEXT", DaoType::UpdateTableInfoPropertiesById => "update table_info set properties = $2::JSON where table_id = $1::TEXT", - - DaoType::DeleteTablePathIdByTablePath => + DaoType::DeleteTablePathIdByTablePath => "delete from table_path_id where table_path = $1::TEXT ", - DaoType::DeleteTablePathIdByTableId => + DaoType::DeleteTablePathIdByTableId => "delete from table_path_id where table_id = $1::TEXT ", - DaoType::DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId => + DaoType::DeleteOneDataCommitInfoByTableIdAndPartitionDescAndCommitId => "delete from data_commit_info where table_id = $1::TEXT and partition_desc = $2::TEXT and commit_id = $3::UUID ", - DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc => + DaoType::DeleteDataCommitInfoByTableIdAndPartitionDesc => "delete from data_commit_info where table_id = $1::TEXT and partition_desc = $2::TEXT", DaoType::DeleteDataCommitInfoByTableId => @@ -391,9 +391,8 @@ async fn get_prepared_statement( DaoType::ListDataCommitInfoByTableIdAndPartitionDescAndCommitList | DaoType::DeleteDataCommitInfoByTableIdAndPartitionDescAndCommitIdList | DaoType::ListPartitionDescByTableIdAndParList => "", - - // _ => todo!(), + /* _ => todo!(), */ }; client.prepare(statement).await }; @@ -407,6 +406,28 @@ async fn get_prepared_statement( } } +fn get_params(joined_string: String) -> Vec { + joined_string + .split(PARAM_DELIM) + .collect::>() + .into_iter() + .map(|s| s.to_string()) + .collect::>() +} + +fn separate_uuid(concated_uuid: &str) -> Result> { + let uuid_num = concated_uuid.len() / 32; + let mut uuid_list = Vec::::with_capacity(uuid_num); + let mut idx = 0; + for _ in 0..uuid_num { + let high = u64::from_str_radix(&concated_uuid[idx..idx + 16], 16)?; + let low = u64::from_str_radix(&concated_uuid[idx + 16..idx + 32], 16)?; + uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string()); + idx += 32; + } + Ok(uuid_list) +} + pub async fn execute_query( client: &Client, prepared: &mut PreparedStatementMap, @@ -420,12 +441,7 @@ pub async fn execute_query( let query_type = DaoType::try_from(query_type).unwrap(); let statement = get_prepared_statement(client, prepared, &query_type).await?; - let params = joined_string - .split(PARAM_DELIM) - .collect::>() - .iter() - .map(|str| str.to_string()) - .collect::>(); + let params = get_params(joined_string); let rows = match query_type { DaoType::ListNamespaces | DaoType::ListAllTablePath if params.len() == 1 && params[0].is_empty() => { @@ -569,18 +585,11 @@ pub async fn execute_query( if concated_uuid.len() % 32 != 0 { eprintln!("Invalid params of query_type={:?}, params={:?}", query_type, params); return Err(LakeSoulMetaDataError::from(std::io::Error::from( - std::io::ErrorKind::InvalidInput, + ErrorKind::InvalidInput, ))); } - let uuid_num = concated_uuid.len() / 32; - let mut uuid_list = Vec::::with_capacity(uuid_num); - let mut idx = 0; - for _ in 0..uuid_num { - let high = u64::from_str_radix(&concated_uuid[idx..idx + 16], 16)?; - let low = u64::from_str_radix(&concated_uuid[idx + 16..idx + 32], 16)?; - uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string()); - idx += 32; - } + + let uuid_list = separate_uuid(concated_uuid)?; let uuid_str_list = "'".to_owned() + &uuid_list.join("','") + "'"; @@ -647,14 +656,14 @@ pub async fn execute_query( ResultType::TableNameId => { let table_name_id: Vec = rows .iter() - .map(|row| proto::proto::entity::TableNameId { + .map(|row| entity::TableNameId { table_name: row.get(0), table_id: row.get(1), table_namespace: row.get(2), domain: row.get(3), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { table_name_id, ..Default::default() } @@ -662,14 +671,14 @@ pub async fn execute_query( ResultType::TablePathId => { let table_path_id: Vec = rows .iter() - .map(|row| proto::proto::entity::TablePathId { + .map(|row| entity::TablePathId { table_path: row.get(0), table_id: row.get(1), table_namespace: row.get(2), domain: row.get(3), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { table_path_id, ..Default::default() } @@ -677,12 +686,12 @@ pub async fn execute_query( ResultType::TablePathIdWithOnlyPath => { let table_path_id: Vec = rows .iter() - .map(|row| proto::proto::entity::TablePathId { + .map(|row| entity::TablePathId { table_path: row.get(0), ..Default::default() }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { table_path_id, ..Default::default() } @@ -691,14 +700,14 @@ pub async fn execute_query( ResultType::Namespace => { let namespace: Vec = rows .iter() - .map(|row| proto::proto::entity::Namespace { + .map(|row| entity::Namespace { namespace: row.get(0), properties: row.get::<_, serde_json::Value>(1).to_string(), comment: row.get::<_, Option>(2).unwrap_or(String::from("")), domain: row.get(3), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { namespace, ..Default::default() } @@ -706,7 +715,7 @@ pub async fn execute_query( ResultType::TableInfo => { let table_info: Vec = rows .iter() - .map(|row| proto::proto::entity::TableInfo { + .map(|row| entity::TableInfo { table_id: row.get(0), table_name: row.get(1), table_path: row.get(2), @@ -717,7 +726,7 @@ pub async fn execute_query( domain: row.get(7), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { table_info, ..Default::default() } @@ -725,25 +734,18 @@ pub async fn execute_query( ResultType::PartitionInfo => { let partition_info: Vec = rows .iter() - .map(|row| proto::proto::entity::PartitionInfo { + .map(|row| entity::PartitionInfo { table_id: row.get(0), partition_desc: row.get(1), version: row.get::<_, i32>(2), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, - snapshot: row - .get::<_, Vec>(4) - .iter() - .map(|uuid| { - let (high, low) = uuid.as_u64_pair(); - entity::Uuid { high, low } - }) - .collect::>(), + commit_op: entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, + snapshot: row_to_uuid_list(row), timestamp: row.get::<_, i64>(5), expression: row.get::<_, Option>(6).unwrap_or(String::from("")), domain: row.get(7), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { partition_info, ..Default::default() } @@ -752,25 +754,18 @@ pub async fn execute_query( ResultType::PartitionInfoWithoutTimestamp => { let partition_info: Vec = rows .iter() - .map(|row| proto::proto::entity::PartitionInfo { + .map(|row| entity::PartitionInfo { table_id: row.get(0), partition_desc: row.get(1), version: row.get::<_, i32>(2), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, - snapshot: row - .get::<_, Vec>(4) - .iter() - .map(|uuid| { - let (high, low) = uuid.as_u64_pair(); - entity::Uuid { high, low } - }) - .collect::>(), + commit_op: entity::CommitOp::from_str_name(row.get(3)).unwrap() as i32, + snapshot: row_to_uuid_list(row), expression: row.get::<_, Option>(5).unwrap_or(String::from("")), domain: row.get(6), ..Default::default() }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { partition_info, ..Default::default() } @@ -778,12 +773,12 @@ pub async fn execute_query( ResultType::PartitionInfoWithOnlyCommitOp => { let partition_info: Vec = rows .iter() - .map(|row| proto::proto::entity::PartitionInfo { - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(0)).unwrap() as i32, + .map(|row| entity::PartitionInfo { + commit_op: entity::CommitOp::from_str_name(row.get(0)).unwrap() as i32, ..Default::default() }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { partition_info, ..Default::default() } @@ -791,7 +786,7 @@ pub async fn execute_query( ResultType::DataCommitInfo => { let data_commit_info: Vec = rows .iter() - .map(|row| proto::proto::entity::DataCommitInfo { + .map(|row| entity::DataCommitInfo { table_id: row.get(0), partition_desc: row.get(1), commit_id: { @@ -803,13 +798,13 @@ pub async fn execute_query( .iter() .map(|data_file_op| data_file_op.as_proto_data_file_op()) .collect::>(), - commit_op: proto::proto::entity::CommitOp::from_str_name(row.get(4)).unwrap() as i32, + commit_op: entity::CommitOp::from_str_name(row.get(4)).unwrap() as i32, timestamp: row.get(5), committed: row.get(6), domain: row.get(7), }) .collect(); - proto::proto::entity::JniWrapper { + entity::JniWrapper { data_commit_info, ..Default::default() } @@ -1067,7 +1062,6 @@ pub async fn execute_insert( ], ) .await; - if let Some(e) = result.err() { eprintln!("transaction insert error, err = {:?}", e); return match transaction.rollback().await { @@ -1105,7 +1099,7 @@ pub async fn execute_update( ) -> Result { if update_type < DAO_TYPE_UPDATE_OFFSET { eprintln!("Invalid update_type_index: {:?}", update_type); - return Err(LakeSoulMetaDataError::from(std::io::ErrorKind::InvalidInput)); + return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } let update_type = DaoType::try_from(update_type).unwrap(); let statement = get_prepared_statement(client, prepared, &update_type).await?; @@ -1198,15 +1192,8 @@ pub async fn execute_update( eprintln!("Invalid params of update_type={:?}, params={:?}", update_type, params); return Err(LakeSoulMetaDataError::from(ErrorKind::InvalidInput)); } - let uuid_num = concated_uuid.len() / 32; - let mut uuid_list = Vec::::with_capacity(uuid_num); - let mut idx = 0; - for _ in 0..uuid_num { - let high = u64::from_str_radix(&concated_uuid[idx..idx + 16], 16)?; - let low = u64::from_str_radix(&concated_uuid[idx + 16..idx + 32], 16)?; - uuid_list.push(uuid::Uuid::from_u64_pair(high, low).to_string()); - idx += 32; - } + + let uuid_list = separate_uuid(concated_uuid)?; let uuid_str_list = "'".to_owned() + &uuid_list.join("','") + "'"; @@ -1230,6 +1217,20 @@ pub async fn execute_update( } } +fn ts_string(res: Result, Error>) -> Result> { + match res { + Ok(Some(row)) => { + let ts = row.get::<_, Option>(0); + match ts { + Some(ts) => Ok(Some(format!("{}", ts))), + None => Ok(None), + } + } + Err(e) => Err(LakeSoulMetaDataError::from(e)), + Ok(None) => Ok(None), + } +} + pub async fn execute_query_scalar( client: &mut Client, prepared: &mut PreparedStatementMap, @@ -1243,27 +1244,12 @@ pub async fn execute_query_scalar( let query_type = DaoType::try_from(query_type).unwrap(); let statement = get_prepared_statement(client, prepared, &query_type).await?; - let params = joined_string - .split(PARAM_DELIM) - .collect::>() - .iter() - .map(|str| str.to_string()) - .collect::>(); + let params = get_params(joined_string); match query_type { DaoType::GetLatestTimestampFromPartitionInfoWithoutPartitionDesc if params.len() == 1 => { let result = client.query_opt(&statement, &[¶ms[0]]).await; - match result { - Ok(Some(row)) => { - let ts = row.get::<_, Option>(0); - match ts { - Some(ts) => Ok(Some(format!("{}", ts))), - None => Ok(None), - } - } - Err(e) => Err(LakeSoulMetaDataError::from(e)), - Ok(None) => Ok(None), - } + ts_string(result) } DaoType::GetLatestTimestampFromPartitionInfo if params.len() == 2 => { let result = client.query_opt(&statement, &[¶ms[0], ¶ms[1]]).await; @@ -1293,17 +1279,7 @@ pub async fn execute_query_scalar( let result = client .query_opt(&statement, &[¶ms[0], ¶ms[1], &i64::from_str(¶ms[2])?]) .await; - match result { - Ok(Some(row)) => { - let ts = row.get::<_, Option>(0); - match ts { - Some(ts) => Ok(Some(format!("{}", ts))), - None => Ok(None), - } - } - Err(e) => Err(LakeSoulMetaDataError::from(e)), - Ok(None) => Ok(None), - } + ts_string(result) } _ => { @@ -1330,6 +1306,7 @@ pub async fn clean_meta_for_test(client: &Client) -> Result { } } +/// Create a pg connection, return pg client pub async fn create_connection(config: String) -> Result { let (client, connection) = match tokio_postgres::connect(config.as_str(), NoTls).await { Ok((client, connection)) => (client, connection), @@ -1348,10 +1325,21 @@ pub async fn create_connection(config: String) -> Result { Ok(client) } +fn row_to_uuid_list(row: &Row) -> Vec { + row.get::<_, Vec>(4) + .iter() + .map(|uuid| { + let (high, low) = uuid.as_u64_pair(); + entity::Uuid { high, low } + }) + .collect() +} + #[cfg(test)] mod tests { use prost::Message; use proto::proto::entity; + #[tokio::test] async fn test_entity() -> std::io::Result<()> { let namespace = entity::Namespace { @@ -1381,7 +1369,7 @@ mod tests { let meta_info = entity::MetaInfo { list_partition: vec![], - table_info: core::option::Option::None, + table_info: None, read_partition_info: vec![], }; println!("{:?}", meta_info); @@ -1398,4 +1386,7 @@ mod tests { Ok(()) } + + #[test] + fn test_client() {} } diff --git a/rust/lakesoul-metadata/src/metadata_client.rs b/rust/lakesoul-metadata/src/metadata_client.rs index 93bc12fc7..986c2c0be 100644 --- a/rust/lakesoul-metadata/src/metadata_client.rs +++ b/rust/lakesoul-metadata/src/metadata_client.rs @@ -2,23 +2,25 @@ // // SPDX-License-Identifier: Apache-2.0 +use std::fmt::{Debug, Formatter}; use std::ops::DerefMut; use std::sync::Arc; use std::{collections::HashMap, env, fs, vec}; use prost::Message; use proto::proto::entity::{ - self, CommitOp, DataCommitInfo, JniWrapper, MetaInfo, PartitionInfo, TableInfo, TableNameId, TablePathId, + self, CommitOp, DataCommitInfo, JniWrapper, MetaInfo, Namespace, PartitionInfo, TableInfo, TableNameId, TablePathId, }; use tokio::sync::Mutex; use tokio_postgres::Client; +use tracing::debug; use url::Url; -use crate::error::Result; +use crate::error::{LakeSoulMetaDataError, Result}; use crate::{ - clean_meta_for_test, create_connection, execute_insert, execute_query, DaoType, PreparedStatementMap, PARAM_DELIM, - PARTITION_DESC_DELIM, + clean_meta_for_test, create_connection, execute_insert, execute_query, execute_update, DaoType, + PreparedStatementMap, PARAM_DELIM, PARTITION_DESC_DELIM, }; pub struct MetaDataClient { @@ -27,6 +29,15 @@ pub struct MetaDataClient { max_retry: usize, } +impl Debug for MetaDataClient { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("MetaDataClient") + .field("client", &"{pg_client}") + .field("max_retry", &self.max_retry) + .finish() + } +} + pub type MetaDataClientRef = Arc; impl MetaDataClient { @@ -79,6 +90,11 @@ impl MetaDataClient { }) } + pub async fn create_namespace(&self, namespace: Namespace) -> Result<()> { + self.insert_namespace(&namespace).await?; + Ok(()) + } + pub async fn create_table(&self, table_info: TableInfo) -> Result<()> { self.insert_table_path_id(&table_path_id_from_table_info(&table_info)) .await?; @@ -88,8 +104,59 @@ impl MetaDataClient { Ok(()) } + pub async fn delete_namespace_by_namespace(&self, namespace: &str) -> Result<()> { + debug!("delete namespace {}", namespace); + self.execute_update( + DaoType::DeleteNamespaceByNamespace as i32, + [namespace].join(PARAM_DELIM), + ) + .await?; + Ok(()) + } + + // Use transaction? + pub async fn delete_table_by_table_info_cascade(&self, table_info: &TableInfo) -> Result<()> { + self.delete_table_name_id_by_table_id(&table_info.table_id).await?; + self.delete_table_path_id_by_table_id(&table_info.table_id).await?; + self.delete_partition_info_by_table_id(&table_info.table_id).await?; + self.delete_data_commit_info_by_table_id(&table_info.table_id).await?; + self.delete_table_info_by_id_and_path(&table_info.table_id, &table_info.table_path) + .await?; + Ok(()) + } + + pub async fn delete_table_path_id_by_table_id(&self, table_id: &str) -> Result { + self.execute_update(DaoType::DeleteTablePathIdByTableId as i32, [table_id].join(PARAM_DELIM)) + .await + } + + pub async fn delete_table_name_id_by_table_id(&self, table_id: &str) -> Result { + self.execute_update(DaoType::DeleteTableNameIdByTableId as i32, [table_id].join(PARAM_DELIM)) + .await + } + + pub async fn delete_partition_info_by_table_id(&self, table_id: &str) -> Result { + self.execute_update( + DaoType::DeletePartitionInfoByTableId as i32, + [table_id].join(PARAM_DELIM), + ) + .await + } + pub async fn delete_data_commit_info_by_table_id(&self, table_id: &str) -> Result { + self.execute_update( + DaoType::DeleteDataCommitInfoByTableId as i32, + [table_id].join(PARAM_DELIM), + ) + .await + } + + pub async fn delete_table_info_by_id_and_path(&self, id: &str, path: &str) -> Result { + self.execute_update(DaoType::DeleteTableInfoByIdAndPath as i32, [id, path].join(PARAM_DELIM)) + .await + } + async fn execute_insert(&self, insert_type: i32, wrapper: JniWrapper) -> Result { - for times in 0..self.max_retry { + for times in 0..self.max_retry as i64 { match execute_insert( self.client.lock().await.deref_mut(), self.prepared.lock().await.deref_mut(), @@ -99,15 +166,33 @@ impl MetaDataClient { .await { Ok(count) => return Ok(count), - Err(_) if times < self.max_retry - 1 => continue, + Err(_) if times < self.max_retry as i64 - 1 => continue, Err(e) => return Err(e), }; } - Ok(0) + Err(LakeSoulMetaDataError::Internal("unreachable".to_string())) + } + + async fn execute_update(&self, update_type: i32, joined_string: String) -> Result { + for times in 0..self.max_retry as i64 { + match execute_update( + self.client.lock().await.deref_mut(), + self.prepared.lock().await.deref_mut(), + update_type, + joined_string.clone(), + ) + .await + { + Ok(count) => return Ok(count), + Err(_) if times < self.max_retry as i64 - 1 => continue, + Err(e) => return Err(e), + }; + } + Err(LakeSoulMetaDataError::Internal("unreachable".to_string())) } async fn execute_query(&self, query_type: i32, joined_string: String) -> Result { - for times in 0..self.max_retry { + for times in 0..self.max_retry as i64 { match execute_query( self.client.lock().await.deref_mut(), self.prepared.lock().await.deref_mut(), @@ -117,11 +202,22 @@ impl MetaDataClient { .await { Ok(encoded) => return Ok(JniWrapper::decode(prost::bytes::Bytes::from(encoded))?), - Err(_) if times < self.max_retry - 1 => continue, + Err(_) if times < self.max_retry as i64 - 1 => continue, Err(e) => return Err(e), }; } - Ok(Default::default()) + Err(LakeSoulMetaDataError::Internal("unreachable".to_string())) + } + + async fn insert_namespace(&self, namespace: &Namespace) -> Result { + self.execute_insert( + DaoType::InsertNamespace as i32, + JniWrapper { + namespace: vec![namespace.clone()], + ..Default::default() + }, + ) + .await } async fn insert_table_info(&self, table_info: &TableInfo) -> Result { @@ -311,6 +407,31 @@ impl MetaDataClient { Ok("public".to_string()) } + pub async fn get_all_table_name_id_by_namespace(&self, namespace: &str) -> Result> { + match self + .execute_query(DaoType::ListTableNameByNamespace as i32, namespace.to_string()) + .await + { + Ok(wrapper) => Ok(wrapper.table_name_id), + Err(e) => Err(e), + } + } + + pub async fn get_all_namespace(&self) -> Result> { + self.execute_query(DaoType::ListNamespaces as i32, String::new()) + .await + .map(|wrapper| wrapper.namespace) + } + + pub async fn get_namespace_by_namespace(&self, namespace: &str) -> Result { + self.execute_query( + DaoType::SelectNamespaceByNamespace as i32, + [namespace].join(PARAM_DELIM), + ) + .await + .map(|wrapper| wrapper.namespace[0].clone()) + } + pub async fn get_table_name_id_by_table_name(&self, table_name: &str, namespace: &str) -> Result { match self .execute_query( @@ -332,7 +453,7 @@ impl MetaDataClient { ) .await { - Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal( + Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::NotFound( format!("Table '{}' not found", table_name), )), Ok(wrapper) => Ok(wrapper.table_info[0].clone()), @@ -345,7 +466,7 @@ impl MetaDataClient { .execute_query(DaoType::SelectTablePathIdByTablePath as i32, table_path.to_string()) .await { - Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::Internal( + Ok(wrapper) if wrapper.table_info.is_empty() => Err(crate::error::LakeSoulMetaDataError::NotFound( format!("Table '{}' not found", table_path), )), Ok(wrapper) => Ok(wrapper.table_info[0].clone()), @@ -488,6 +609,7 @@ pub fn table_path_id_from_table_info(table_info: &TableInfo) -> TablePathId { domain: table_info.domain.clone(), } } + pub fn table_name_id_from_table_info(table_info: &TableInfo) -> TableNameId { TableNameId { table_name: table_info.table_name.clone(), diff --git a/website/i18n/zh-Hans/code.json b/website/i18n/zh-Hans/code.json index abf993c14..6c7c205b4 100644 --- a/website/i18n/zh-Hans/code.json +++ b/website/i18n/zh-Hans/code.json @@ -271,7 +271,7 @@ "description": "The title of the tag list page" }, "homepage.documentBtnLink": { - "message": "/zh-Hans/docs/intro/" + "message": "/zh-Hans/docs/intro" }, "homepage.documentBtn": { "message": "文档教程"