diff --git a/.github/actions/artifact_failure/action.yml b/.github/actions/artifact_failure/action.yml index f1ef0ea0dfe2..93e5aa864437 100644 --- a/.github/actions/artifact_failure/action.yml +++ b/.github/actions/artifact_failure/action.yml @@ -19,7 +19,10 @@ runs: docker logs "$line" > .databend/docker/"$line".log done - tar -zcf target/failure-${{ inputs.name }}.tar.gz .databend + tar -zcf target/failure-${{ inputs.name }}.tar.gz \ + nohup.out \ + .databend/ \ + ./**/.databend \ - uses: actions/upload-artifact@v3 with: name: ${{ inputs.name }} diff --git a/.github/workflows/reuse.linux.yml b/.github/workflows/reuse.linux.yml index 77695a777301..99831a624058 100644 --- a/.github/workflows/reuse.linux.yml +++ b/.github/workflows/reuse.linux.yml @@ -214,6 +214,7 @@ jobs: - "ee" format: - "parquet" + - "native" handlers: - "mysql,http,clickhouse" steps: diff --git a/Cargo.lock b/Cargo.lock index 6fa858a38c94..f633c913d730 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2927,11 +2927,14 @@ dependencies = [ "common-meta-app", "common-pipeline-core", "common-pipeline-sources", + "common-pipeline-transforms", "common-storage", "dashmap", "opendal", "parking_lot 0.12.1", + "serde", "tracing", + "typetag", "uuid", ] diff --git a/benchmark/clickbench/benchmark_cloud.sh b/benchmark/clickbench/benchmark_cloud.sh index 614e520dd464..d754d8bb972c 100755 --- a/benchmark/clickbench/benchmark_cloud.sh +++ b/benchmark/clickbench/benchmark_cloud.sh @@ -70,6 +70,9 @@ until bendsql --query="SHOW WAREHOUSES LIKE '${CLOUD_WAREHOUSE}'" | grep -q "Run sleep 10 done +echo "Checking session settings..." +bendsql --query="select * from system.settings where value != default;" -o table + echo "Running queries..." # analyze table diff --git a/docs/doc/14-sql-commands/00-ddl/20-table/20-ddl-drop-table.md b/docs/doc/14-sql-commands/00-ddl/20-table/20-ddl-drop-table.md index 976d3b4b7c1d..704f7253825b 100644 --- a/docs/doc/14-sql-commands/00-ddl/20-table/20-ddl-drop-table.md +++ b/docs/doc/14-sql-commands/00-ddl/20-table/20-ddl-drop-table.md @@ -7,6 +7,7 @@ Deletes the table. **See also:** - [CREATE TABLE](./10-ddl-create-table.md) - [UNDROP TABLE](./21-ddl-undrop-table.md) +- [TRUNCATE TABLE](40-ddl-truncate-table.md) ## Syntax diff --git a/docs/doc/14-sql-commands/00-ddl/20-table/40-ddl-truncate-table.md b/docs/doc/14-sql-commands/00-ddl/20-table/40-ddl-truncate-table.md index e13012b6a650..68592fc28c60 100644 --- a/docs/doc/14-sql-commands/00-ddl/20-table/40-ddl-truncate-table.md +++ b/docs/doc/14-sql-commands/00-ddl/20-table/40-ddl-truncate-table.md @@ -2,29 +2,58 @@ title: TRUNCATE TABLE --- -Empties the table completely. +Removes all data from a table while preserving the table's schema. It deletes all rows in the table, making it an empty table with the same columns and constraints. Please note that, it does not release the disk space allocated to the table. To release the disk space, include the PURGE option, which is used to release the disk space allocated to the table when the truncate operation is performed. + +See also: [DROP TABLE](20-ddl-drop-table.md) ## Syntax ```sql -TRUNCATE TABLE [db.]name +TRUNCATE TABLE [db.]table_name [PURGE] ``` ## Examples ```sql -CREATE TABLE test(a BIGINT UNSIGNED, b VARCHAR) Engine = Fuse; +root@localhost> CREATE TABLE test_truncate(a BIGINT UNSIGNED, b VARCHAR); +Processed in (0.027 sec) -INSERT INTO test(a,b) VALUES(888, 'stars'); +root@localhost> INSERT INTO test_truncate(a,b) VALUES(1234, 'databend'); +1 rows affected in (0.060 sec) -SELECT * FROM test; -+------+---------+ -| a | b | -+------+---------+ -| 888 | stars | -+------+---------+ +root@localhost> SELECT * FROM test_truncate; -TRUNCATE TABLE test; +SELECT + * +FROM + test_truncate -SELECT * FROM test; -``` +┌───────────────────┐ +│ a │ b │ +│ UInt64 │ String │ +├────────┼──────────┤ +│ 1234 │ databend │ +└───────────────────┘ +1 row in 0.019 sec. Processed 1 rows, 1B (53.26 rows/s, 17.06 KiB/s) + +root@localhost> TRUNCATE TABLE test_truncate; + +TRUNCATE TABLE test_truncate + +0 row in 0.047 sec. Processed 0 rows, 0B (0 rows/s, 0B/s) + +root@localhost> SELECT * FROM test_truncate; + +SELECT + * +FROM + test_truncate + +0 row in 0.017 sec. Processed 0 rows, 0B (0 rows/s, 0B/s) + +root@localhost> TRUNCATE TABLE test_truncate PURGE; + +TRUNCATE TABLE test_truncate PURGE + +0 row in 0.118 sec. Processed 0 rows, 0B (0 rows/s, 0B/s) +``` \ No newline at end of file diff --git a/src/common/arrow/src/parquet_write.rs b/src/common/arrow/src/parquet_write.rs index 7446c6908cb8..039522081799 100644 --- a/src/common/arrow/src/parquet_write.rs +++ b/src/common/arrow/src/parquet_write.rs @@ -16,10 +16,12 @@ use std::io::Write; use arrow::array::Array; use arrow::chunk::Chunk; +use arrow::datatypes::DataType; use arrow::datatypes::Schema; use arrow::error::Result; use arrow::io::parquet::write::to_parquet_schema; use arrow::io::parquet::write::RowGroupIterator; +use parquet2::metadata::KeyValue; use parquet2::metadata::ThriftFileMetaData; use parquet2::write::FileWriter; use parquet2::write::WriteOptions; @@ -36,6 +38,18 @@ where A: AsRef + 'static + Send + Sync, I: Iterator>>, { + // add extension data type to parquet meta. + let mut key_values = Vec::new(); + for field in &schema.fields { + if let DataType::Extension(ty, _, _) = &field.data_type { + let key_value = KeyValue { + key: field.name.clone(), + value: Some(ty.clone()), + }; + key_values.push(key_value); + } + } + let parquet_schema = to_parquet_schema(&schema)?; // Arrow2 should be honored @@ -45,7 +59,13 @@ where for group in row_groups { file_writer.write(group?)?; } - let file_size = file_writer.end(None)?; + + let key_value_metadata = if !key_values.is_empty() { + Some(key_values) + } else { + None + }; + let file_size = file_writer.end(key_value_metadata)?; let (_meta_size, thrift_file_meta_data) = file_writer.into_inner_and_metadata(); Ok((file_size, thrift_file_meta_data)) } diff --git a/src/common/base/src/lib.rs b/src/common/base/src/lib.rs index 796e1daf36bf..fea64ad04c3a 100644 --- a/src/common/base/src/lib.rs +++ b/src/common/base/src/lib.rs @@ -32,5 +32,6 @@ pub mod mem_allocator; pub mod rangemap; pub mod runtime; +pub use runtime::dump_backtrace; pub use runtime::match_join_handle; pub use runtime::set_alloc_error_hook; diff --git a/src/common/base/src/runtime/backtrace.rs b/src/common/base/src/runtime/backtrace.rs new file mode 100644 index 000000000000..ef5ce78a406e --- /dev/null +++ b/src/common/base/src/runtime/backtrace.rs @@ -0,0 +1,79 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Write; + +#[derive(Debug)] +struct AsyncTaskItem { + stack_frames: Vec, +} + +pub fn dump_backtrace(wait_for_running_tasks: bool) -> String { + let tree = async_backtrace::taskdump_tree(wait_for_running_tasks); + + let mut tasks = vec![]; + let mut polling_tasks = vec![]; + let mut current_stack_frames = vec![]; + + let mut first = true; + let mut is_polling = false; + for line in tree.lines() { + if line.starts_with(|x: char| !x.is_ascii_whitespace()) { + if !first { + match is_polling { + true => polling_tasks.push(AsyncTaskItem { + stack_frames: std::mem::take(&mut current_stack_frames), + }), + false => tasks.push(AsyncTaskItem { + stack_frames: std::mem::take(&mut current_stack_frames), + }), + }; + + is_polling = false; + } + + first = false; + } + + if line.ends_with("[POLLING]") { + is_polling = true; + } + + current_stack_frames.push(line.to_string()); + } + + match is_polling { + true => polling_tasks.push(AsyncTaskItem { + stack_frames: std::mem::take(&mut current_stack_frames), + }), + false => tasks.push(AsyncTaskItem { + stack_frames: std::mem::take(&mut current_stack_frames), + }), + }; + + let mut output = String::new(); + for mut tasks in [tasks, polling_tasks] { + tasks.sort_by(|l, r| Ord::cmp(&l.stack_frames.len(), &r.stack_frames.len())); + + for item in tasks.into_iter().rev() { + for frame in item.stack_frames { + writeln!(output, "{}", frame).unwrap(); + } + + writeln!(output).unwrap(); + } + } + + output +} diff --git a/src/common/base/src/runtime/mod.rs b/src/common/base/src/runtime/mod.rs index 3ed7cbad38bc..1404f04607d3 100644 --- a/src/common/base/src/runtime/mod.rs +++ b/src/common/base/src/runtime/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod backtrace; mod catch_unwind; mod global_runtime; #[allow(clippy::module_inception)] @@ -20,6 +21,7 @@ mod runtime_tracker; mod thread; mod thread_pool; +pub use backtrace::dump_backtrace; pub use catch_unwind::catch_unwind; pub use catch_unwind::CatchUnwindFuture; pub use global_runtime::GlobalIORuntime; diff --git a/src/common/exception/src/exception_code.rs b/src/common/exception/src/exception_code.rs index 88ace541e140..a96139300e27 100644 --- a/src/common/exception/src/exception_code.rs +++ b/src/common/exception/src/exception_code.rs @@ -184,8 +184,7 @@ build_exceptions! { // Index related errors. UnsupportedIndex(1601), - IndexAlreadyRefreshed(1602), - RefreshIndexError(1603), + RefreshIndexError(1602), } // Meta service errors [2001, 3000]. diff --git a/src/common/http/src/debug/stack.rs b/src/common/http/src/debug/stack.rs index 457a7fa5cdda..f835df112817 100644 --- a/src/common/http/src/debug/stack.rs +++ b/src/common/http/src/debug/stack.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::Write; - +use common_base::dump_backtrace; use poem::web::Query; use poem::IntoResponse; @@ -22,68 +21,7 @@ pub struct DumpStackRequest { wait_for_running_tasks: bool, } -#[derive(Debug)] -struct AsyncTaskItem { - stack_frames: Vec, -} - #[poem::handler] pub async fn debug_dump_stack(req: Option>) -> impl IntoResponse { - let tree = - async_backtrace::taskdump_tree(req.map(|x| x.wait_for_running_tasks).unwrap_or(false)); - - let mut tasks = vec![]; - let mut polling_tasks = vec![]; - let mut current_stack_frames = vec![]; - - let mut first = true; - let mut is_polling = false; - for line in tree.lines() { - if line.starts_with(|x: char| !x.is_ascii_whitespace()) { - if !first { - match is_polling { - true => polling_tasks.push(AsyncTaskItem { - stack_frames: std::mem::take(&mut current_stack_frames), - }), - false => tasks.push(AsyncTaskItem { - stack_frames: std::mem::take(&mut current_stack_frames), - }), - }; - - is_polling = false; - } - - first = false; - } - - if line.ends_with("[POLLING]") { - is_polling = true; - } - - current_stack_frames.push(line.to_string()); - } - - match is_polling { - true => polling_tasks.push(AsyncTaskItem { - stack_frames: std::mem::take(&mut current_stack_frames), - }), - false => tasks.push(AsyncTaskItem { - stack_frames: std::mem::take(&mut current_stack_frames), - }), - }; - - let mut output = String::new(); - for mut tasks in [tasks, polling_tasks] { - tasks.sort_by(|l, r| Ord::cmp(&l.stack_frames.len(), &r.stack_frames.len())); - - for item in tasks.into_iter().rev() { - for frame in item.stack_frames { - writeln!(output, "{}", frame).unwrap(); - } - - writeln!(output).unwrap(); - } - } - - output + dump_backtrace(req.map(|x| x.wait_for_running_tasks).unwrap_or(false)) } diff --git a/src/common/metrics/src/lib.rs b/src/common/metrics/src/lib.rs index c4882d6f5894..9da179eb9f97 100644 --- a/src/common/metrics/src/lib.rs +++ b/src/common/metrics/src/lib.rs @@ -17,6 +17,7 @@ pub mod counter; mod dump; mod recorder; +mod reset; pub use dump::dump_metric_samples; pub use dump::HistogramCount; @@ -39,3 +40,4 @@ pub use recorder::label_increment_gauge_with_val_and_labels; pub use recorder::try_handle; pub use recorder::LABEL_KEY_CLUSTER; pub use recorder::LABEL_KEY_TENANT; +pub use reset::reset_metrics; diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/table_args.rs b/src/common/metrics/src/reset.rs similarity index 59% rename from src/query/storages/fuse/src/table_functions/fuse_segments/table_args.rs rename to src/common/metrics/src/reset.rs index 452289c069ed..a19ebd26ef4a 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/table_args.rs +++ b/src/common/metrics/src/reset.rs @@ -13,14 +13,19 @@ // limitations under the License. use common_exception::Result; +use metrics::gauge; +use metrics_exporter_prometheus::PrometheusHandle; -use crate::table_functions::string_value; -use crate::table_functions::TableArgs; +use crate::dump_metric_samples; +use crate::MetricValue; -pub fn parse_func_history_args(table_args: &TableArgs) -> Result<(String, String, String)> { - let args = table_args.expect_all_positioned("fuse_blocks", Some(3))?; - let db = string_value(&args[0])?; - let tbl = string_value(&args[1])?; - let snapshot_id = string_value(&args[2])?; - Ok((db, tbl, snapshot_id)) +/// Reset gauge metrics to 0. +pub fn reset_metrics(handle: PrometheusHandle) -> Result<()> { + let samples = dump_metric_samples(handle)?; + for sample in samples { + if let MetricValue::Gauge(_) = sample.value { + gauge!(sample.name, 0_f64); + } + } + Ok(()) } diff --git a/src/common/storage/src/lib.rs b/src/common/storage/src/lib.rs index 9bc1f5b6416c..455bd15f1f6b 100644 --- a/src/common/storage/src/lib.rs +++ b/src/common/storage/src/lib.rs @@ -50,6 +50,7 @@ pub use column_node::ColumnNode; pub use column_node::ColumnNodes; mod parquet; +pub use parquet::infer_schema_with_extension; pub use parquet::read_parquet_metas_in_parallel; pub use parquet::read_parquet_schema_async; diff --git a/src/common/storage/src/parquet.rs b/src/common/storage/src/parquet.rs index 1b5c1cb02591..9bd8879c2a38 100644 --- a/src/common/storage/src/parquet.rs +++ b/src/common/storage/src/parquet.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use common_arrow::arrow::datatypes::DataType as ArrowType; +use common_arrow::arrow::datatypes::Field as ArrowField; use common_arrow::arrow::datatypes::Schema as ArrowSchema; use common_arrow::arrow::io::parquet::read as pread; use common_arrow::parquet::metadata::FileMetaData; @@ -27,8 +29,35 @@ pub async fn read_parquet_schema_async(operator: &Operator, path: &str) -> Resul ErrorCode::Internal(format!("Read parquet file '{}''s meta error: {}", path, e)) })?; - let arrow_schema = pread::infer_schema(&meta)?; - Ok(arrow_schema) + infer_schema_with_extension(&meta) +} + +pub fn infer_schema_with_extension(meta: &FileMetaData) -> Result { + let arrow_schema = pread::infer_schema(meta)?; + // Convert data types to extension types using meta information. + // Mainly used for types such as Variant and Bitmap, + // as they have the same physical type as String. + if let Some(metas) = meta.key_value_metadata() { + let mut new_fields = arrow_schema.fields.clone(); + for (i, field) in arrow_schema.fields.iter().enumerate() { + for meta in metas { + if field.name == meta.key { + let data_type = ArrowType::Extension( + meta.value.clone().unwrap(), + Box::new(field.data_type.clone()), + None, + ); + let new_field = + ArrowField::new(field.name.clone(), data_type, field.is_nullable); + new_fields[i] = new_field; + break; + } + } + } + Ok(new_fields.into()) + } else { + Ok(arrow_schema) + } } async fn read_parquet_metas_batch( diff --git a/src/meta/api/src/schema_api_impl.rs b/src/meta/api/src/schema_api_impl.rs index aa4466d35623..c81a0b77c818 100644 --- a/src/meta/api/src/schema_api_impl.rs +++ b/src/meta/api/src/schema_api_impl.rs @@ -2009,6 +2009,10 @@ impl> SchemaApi for KV { let mut ret_table_infos = vec![]; for db_info in db_infos { + // ignore db create from share + if db_info.meta.from_share.is_some() { + continue; + } let table_infos = match db_info.meta.drop_on { Some(db_drop_on) => { if let Some(filter_drop_on) = filter_drop_on { @@ -2071,6 +2075,11 @@ impl> SchemaApi for KV { } }; + // ignore db create from share + if db_meta.from_share.is_some() && req.filter.is_some() { + return Ok(vec![]); + } + do_get_table_history(&self, req, db_id, &db_meta).await } diff --git a/src/meta/client/src/grpc_client.rs b/src/meta/client/src/grpc_client.rs index 8820ecfd9185..e31d92c6cb10 100644 --- a/src/meta/client/src/grpc_client.rs +++ b/src/meta/client/src/grpc_client.rs @@ -529,6 +529,8 @@ impl MetaGrpcClient { endpoints }; + debug!("healthy endpoints: {:?}", &endpoints); + let endpoints = if endpoints.is_empty() { warn!( "meta-service has no healthy endpoints, force using all(healthy or not) endpoints: {:?}", @@ -545,6 +547,7 @@ impl MetaGrpcClient { .enumerate() .map(|(i, a)| (a, i == endpoints.len() - 1)) { + debug!("make_channel to {}", addr); let channel = self.make_channel(Some(addr)).await; match channel { Ok(c) => { @@ -587,10 +590,13 @@ impl MetaGrpcClient { } Err(net_err) => { + warn!("{} when make_channel to {}", net_err, addr); + { let mut ue = self.unhealthy_endpoints.lock(); ue.insert(addr.to_string(), ()); } + if is_last { let cli_err = MetaClientError::NetworkError(net_err); return Err(cli_err); diff --git a/src/meta/service/src/meta_service/raftmeta.rs b/src/meta/service/src/meta_service/raftmeta.rs index b93f5afcd4e5..dd4f0f7cfca6 100644 --- a/src/meta/service/src/meta_service/raftmeta.rs +++ b/src/meta/service/src/meta_service/raftmeta.rs @@ -988,7 +988,7 @@ impl MetaNode { } } - #[tracing::instrument(level = "debug", skip(self, req), fields(target=%req.forward_to_leader))] + #[tracing::instrument(level = "debug", skip_all, fields(target=%req.forward_to_leader))] pub async fn handle_forwardable_request( &self, req: ForwardRequest, @@ -997,46 +997,77 @@ impl MetaNode { let forward = req.forward_to_leader; - let assume_leader_res = self.assume_leader().await; - debug!("assume_leader: is_err: {}", assume_leader_res.is_err()); + let mut n_retry = 17; + let mut slp = Duration::from_millis(200); - // Handle the request locally or return a ForwardToLeader error - let op_err = match assume_leader_res { - Ok(leader) => { - let res = leader.handle_request(req.clone()).await; - match res { - Ok(x) => return Ok(x), - Err(e) => e, + loop { + let assume_leader_res = self.assume_leader().await; + debug!("assume_leader: is_err: {}", assume_leader_res.is_err()); + + // Handle the request locally or return a ForwardToLeader error + let op_err = match assume_leader_res { + Ok(leader) => { + let res = leader.handle_request(req.clone()).await; + match res { + Ok(x) => return Ok(x), + Err(e) => e, + } } - } - Err(e) => MetaOperationError::ForwardToLeader(e), - }; + Err(e) => MetaOperationError::ForwardToLeader(e), + }; + + // If needs to forward, deal with it. Otherwise return the unhandlable error. + let to_leader = match op_err { + MetaOperationError::ForwardToLeader(err) => err, + MetaOperationError::DataError(d_err) => { + return Err(d_err.into()); + } + }; - // If needs to forward, deal with it. Otherwise return the unhandlable error. - let to_leader = match op_err { - MetaOperationError::ForwardToLeader(err) => err, - MetaOperationError::DataError(d_err) => { - return Err(d_err.into()); + if forward == 0 { + return Err(MetaAPIError::CanNotForward(AnyError::error( + "max number of forward reached", + ))); } - }; - if forward == 0 { - return Err(MetaAPIError::CanNotForward(AnyError::error( - "max number of forward reached", - ))); - } + let leader_id = to_leader.leader_id.ok_or_else(|| { + MetaAPIError::CanNotForward(AnyError::error("need to forward but no known leader")) + })?; - let leader_id = to_leader.leader_id.ok_or_else(|| { - MetaAPIError::CanNotForward(AnyError::error("need to forward but no known leader")) - })?; + let mut req_cloned = req.clone(); + // Avoid infinite forward + req_cloned.decr_forward(); - let mut r2 = req.clone(); - // Avoid infinite forward - r2.decr_forward(); + let res = self.forward_to(&leader_id, req_cloned).await; + let forward_err = match res { + Ok(x) => { + return Ok(x); + } + Err(forward_err) => forward_err, + }; - let res: ForwardResponse = self.forward_to(&leader_id, r2).await?; + match forward_err { + ForwardRPCError::NetworkError(ref net_err) => { + warn!( + "{} retries left, sleep time: {:?}; forward_to {} failed: {}", + n_retry, slp, leader_id, net_err + ); - Ok(res) + n_retry -= 1; + if n_retry == 0 { + error!("no more retry for forward_to {}", leader_id); + return Err(MetaAPIError::from(forward_err)); + } else { + tokio::time::sleep(slp).await; + slp = std::cmp::min(slp * 2, Duration::from_secs(1)); + continue; + } + } + ForwardRPCError::RemoteError(_) => { + return Err(MetaAPIError::from(forward_err)); + } + } + } } /// Return a MetaLeader if `self` believes it is the leader. diff --git a/src/meta/service/tests/it/grpc/metasrv_connection_error.rs b/src/meta/service/tests/it/grpc/metasrv_connection_error.rs new file mode 100644 index 000000000000..bd2bfbd6d420 --- /dev/null +++ b/src/meta/service/tests/it/grpc/metasrv_connection_error.rs @@ -0,0 +1,177 @@ +// Copyright 2021 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Test metasrv SchemaApi by writing to one node and then reading from another, +//! on a restarted cluster. + +use std::fmt::Display; +use std::sync::Arc; +use std::time::Duration; + +use common_base::base::tokio; +use common_base::base::Stoppable; +use common_meta_client::ClientHandle; +use common_meta_client::MetaGrpcClient; +use common_meta_kvapi::kvapi::KVApi; +use common_meta_kvapi::kvapi::UpsertKVReq; +use common_meta_types::MetaClientError; +use databend_meta::init_meta_ut; +use tracing::info; + +use crate::tests::service::start_metasrv_cluster; + +/// Ensure grpc-client will retry when failing to connect. +/// +/// - Start a cluster of 3. +/// - Shutdown node 1. +/// - Test upsert kv, expect the client auto choose the running nodes. +#[async_entry::test(worker_threads = 3, init = "init_meta_ut!()", tracing_span = "debug")] +async fn test_metasrv_connection_error() -> anyhow::Result<()> { + info!("--- Start cluster 0,1,2"); + let mut tcs = start_metasrv_cluster(&[0, 1, 2]).await?; + + let addresses = tcs + .iter() + .map(|tc| tc.config.grpc_api_address.clone()) + .collect::>(); + + let a0 = || addresses[0].clone(); + let a1 = || addresses[1].clone(); + let a2 = || addresses[2].clone(); + + let mut stopped = tcs.remove(1); + { stopped }.grpc_srv.take().unwrap().stop(None).await?; + + for addrs in [ + vec![a0(), a1(), a2()], // a1() is down + vec![a1(), a2()], // a1() is down + vec![a0(), a1()], // a1() is down + vec![a0()], + vec![a2()], + ] { + let addrs_str = addrs.join(",").to_string(); + info!("--- test write with api addresses: {}", addrs_str); + + let client = make_client(addrs.to_vec())?; + test_write_read(&client, &format!("grpc-conn-error-{}", addrs_str)).await?; + } + + info!("--- using only one crashed node won't work"); + { + let client = make_client(vec![a1()])?; + let res = test_write_read(&client, "crashed-node-1").await; + assert!(res.is_err()); + } + Ok(()) +} + +/// Ensure grpc-client will retry when failing to connect, too. +/// +/// - Start a cluster of 3. +/// - Create a client to node 1 and 2. +/// - Shutdown follower node 1. +/// - Test upsert kv, expect the client to auto choose the running nodes. +#[async_entry::test(worker_threads = 3, init = "init_meta_ut!()", tracing_span = "debug")] +async fn test_metasrv_one_client_follower_down() -> anyhow::Result<()> { + info!("--- Start cluster 0,1,2"); + let mut tcs = start_metasrv_cluster(&[0, 1, 2]).await?; + + let addresses = tcs + .iter() + .map(|tc| tc.config.grpc_api_address.clone()) + .collect::>(); + + let a1 = || addresses[1].clone(); + let a2 = || addresses[2].clone(); + + // a1() will be shut down + let client = make_client(vec![a1(), a2()])?; + + test_write_read(&client, "conn-error-one-client-node-1-running").await?; + + let mut stopped = tcs.remove(1); + { stopped }.grpc_srv.take().unwrap().stop(None).await?; + + test_write_read(&client, "conn-error-one-client-node-1-down").await?; + + Ok(()) +} + +/// Ensure internal forward RPC will retry when failing to connect to the leader. +/// +/// - Start a cluster of 3. +/// - Create a client to node 1 and 2. +/// - Shutdown leader node 0. +/// - Test upsert kv, expect the client to auto choose the running nodes. +#[async_entry::test(worker_threads = 3, init = "init_meta_ut!()", tracing_span = "debug")] +async fn test_metasrv_one_client_leader_down() -> anyhow::Result<()> { + info!("--- Start cluster 0,1,2"); + let mut tcs = start_metasrv_cluster(&[0, 1, 2]).await?; + + let addresses = tcs + .iter() + .map(|tc| tc.config.grpc_api_address.clone()) + .collect::>(); + + let a1 = || addresses[1].clone(); + let a2 = || addresses[2].clone(); + + // a0() will be shut down + let client = make_client(vec![a1(), a2()])?; + + test_write_read(&client, "conn-error-one-client-node-0-running").await?; + + let mut stopped = tcs.remove(0); + { stopped }.grpc_srv.take().unwrap().stop(None).await?; + + // Write/read operations will recover functioning after a new leader is elected. + test_write_read(&client, "conn-error-one-client-node-0-down").await?; + + Ok(()) +} + +fn make_client(addresses: Vec) -> Result, MetaClientError> { + let client = MetaGrpcClient::try_create( + addresses, // a1() will be shut down + "root", + "xxx", + None, + Some(Duration::from_secs(10)), + Duration::from_secs(10), + None, + )?; + + Ok(client) +} + +/// Test write and then read with a provided client +async fn test_write_read(client: &Arc, key: impl Display) -> anyhow::Result<()> { + info!("--- test write/read: {}", key); + + let k = key.to_string(); + let res = client.upsert_kv(UpsertKVReq::update(&k, &b(&k))).await?; + + info!("--- upsert {} res: {:?}", k, res); + + let res = client.get_kv(&k).await?; + let res = res.unwrap(); + + assert_eq!(k.into_bytes(), res.data); + + Ok(()) +} + +fn b(s: impl ToString) -> Vec { + s.to_string().into_bytes() +} diff --git a/src/meta/service/tests/it/grpc/mod.rs b/src/meta/service/tests/it/grpc/mod.rs index 744e36e77f09..4f6401d323ae 100644 --- a/src/meta/service/tests/it/grpc/mod.rs +++ b/src/meta/service/tests/it/grpc/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod metasrv_connection_error; pub mod metasrv_grpc_api; mod metasrv_grpc_export; pub mod metasrv_grpc_get_client_info; diff --git a/src/query/ast/src/ast/format/ast_format.rs b/src/query/ast/src/ast/format/ast_format.rs index 79295c86a32c..85a55b0332a3 100644 --- a/src/query/ast/src/ast/format/ast_format.rs +++ b/src/query/ast/src/ast/format/ast_format.rs @@ -1492,6 +1492,22 @@ impl<'ast> Visitor<'ast> for AstFormatVisitor { self.children.push(node); } + fn visit_vacuum_drop_table(&mut self, stmt: &'ast VacuumDropTableStmt) { + let mut children = Vec::new(); + if let Some(database) = &stmt.database { + self.visit_database_ref(&stmt.catalog, database); + } + children.push(self.children.pop().unwrap()); + let action_name = format!("Option {}", &stmt.option); + let action_format_ctx = AstFormatContext::new(action_name); + children.push(FormatTreeNode::new(action_format_ctx)); + + let name = "VacuumDropTable".to_string(); + let format_ctx = AstFormatContext::with_children(name, children.len()); + let node = FormatTreeNode::with_children(format_ctx, children); + self.children.push(node); + } + fn visit_analyze_table(&mut self, stmt: &'ast AnalyzeTableStmt) { let mut children = Vec::new(); self.visit_table_ref(&stmt.catalog, &stmt.database, &stmt.table); diff --git a/src/query/ast/src/ast/statements/statement.rs b/src/query/ast/src/ast/statements/statement.rs index 901892790c89..61318f9a7b29 100644 --- a/src/query/ast/src/ast/statements/statement.rs +++ b/src/query/ast/src/ast/statements/statement.rs @@ -108,6 +108,7 @@ pub enum Statement { DescribeTable(DescribeTableStmt), ShowTablesStatus(ShowTablesStatusStmt), ShowDropTables(ShowDropTablesStmt), + AttachTable(AttachTableStmt), CreateTable(CreateTableStmt), DropTable(DropTableStmt), UndropTable(UndropTableStmt), @@ -116,6 +117,7 @@ pub enum Statement { TruncateTable(TruncateTableStmt), OptimizeTable(OptimizeTableStmt), VacuumTable(VacuumTableStmt), + VacuumDropTable(VacuumDropTableStmt), AnalyzeTable(AnalyzeTableStmt), ExistsTable(ExistsTableStmt), // Columns @@ -377,6 +379,7 @@ impl Display for Statement { Statement::DescribeTable(stmt) => write!(f, "{stmt}")?, Statement::ShowTablesStatus(stmt) => write!(f, "{stmt}")?, Statement::ShowDropTables(stmt) => write!(f, "{stmt}")?, + Statement::AttachTable(stmt) => write!(f, "{stmt}")?, Statement::CreateTable(stmt) => write!(f, "{stmt}")?, Statement::DropTable(stmt) => write!(f, "{stmt}")?, Statement::UndropTable(stmt) => write!(f, "{stmt}")?, @@ -385,6 +388,7 @@ impl Display for Statement { Statement::TruncateTable(stmt) => write!(f, "{stmt}")?, Statement::OptimizeTable(stmt) => write!(f, "{stmt}")?, Statement::VacuumTable(stmt) => write!(f, "{stmt}")?, + Statement::VacuumDropTable(stmt) => write!(f, "{stmt}")?, Statement::AnalyzeTable(stmt) => write!(f, "{stmt}")?, Statement::ExistsTable(stmt) => write!(f, "{stmt}")?, Statement::CreateView(stmt) => write!(f, "{stmt}")?, diff --git a/src/query/ast/src/ast/statements/table.rs b/src/query/ast/src/ast/statements/table.rs index bb1dd9a90128..778bb6cfa946 100644 --- a/src/query/ast/src/ast/statements/table.rs +++ b/src/query/ast/src/ast/statements/table.rs @@ -175,6 +175,31 @@ impl Display for CreateTableStmt { } } +#[derive(Debug, Clone, PartialEq)] +pub struct AttachTableStmt { + pub catalog: Option, + pub database: Option, + pub table: Identifier, + pub uri_location: UriLocation, +} + +impl Display for AttachTableStmt { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "ATTACH TABLE ")?; + write_period_separated_list( + f, + self.catalog + .iter() + .chain(&self.database) + .chain(Some(&self.table)), + )?; + + write!(f, " FROM {0}", self.uri_location)?; + + Ok(()) + } +} + #[allow(clippy::large_enum_variant)] #[derive(Debug, Clone, PartialEq)] pub enum CreateTableSource { @@ -466,6 +491,27 @@ impl Display for VacuumTableStmt { } } +#[derive(Debug, Clone, PartialEq)] +pub struct VacuumDropTableStmt { + pub catalog: Option, + pub database: Option, + pub option: VacuumTableOption, +} + +impl Display for VacuumDropTableStmt { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + write!(f, "VACUUM DROP TABLE ")?; + if self.catalog.is_some() || self.database.is_some() { + write!(f, "FROM ")?; + write_period_separated_list(f, self.catalog.iter().chain(&self.database))?; + write!(f, " ")?; + } + write!(f, "{}", &self.option)?; + + Ok(()) + } +} + #[derive(Debug, Clone, PartialEq)] pub struct OptimizeTableStmt { pub catalog: Option, @@ -572,10 +618,14 @@ pub struct VacuumTableOption { impl Display for VacuumTableOption { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { if let Some(retain_hours) = &self.retain_hours { - write!(f, " RETAIN {} HOURS", retain_hours)?; + write!(f, "RETAIN {} HOURS", retain_hours)?; } if self.dry_run.is_some() { - write!(f, " DRY RUN")?; + if self.retain_hours.is_some() { + write!(f, " DRY RUN")?; + } else { + write!(f, "DRY RUN")?; + } } Ok(()) } diff --git a/src/query/ast/src/parser/statement.rs b/src/query/ast/src/parser/statement.rs index 62e6391180c7..6af614628c58 100644 --- a/src/query/ast/src/parser/statement.rs +++ b/src/query/ast/src/parser/statement.rs @@ -483,6 +483,20 @@ pub fn statement(i: Input) -> IResult { }) }, ); + + let attach_table = map( + rule! { + ATTACH ~ TABLE ~ #period_separated_idents_1_to_3 ~ #uri_location + }, + |(_, _, (catalog, database, table), uri_location)| { + Statement::AttachTable(AttachTableStmt { + catalog, + database, + table, + uri_location, + }) + }, + ); let create_table = map( rule! { CREATE ~ TRANSIENT? ~ TABLE ~ ( IF ~ NOT ~ EXISTS )? @@ -625,6 +639,22 @@ pub fn statement(i: Input) -> IResult { }) }, ); + let vacuum_drop_table = map( + rule! { + VACUUM ~ DROP ~ TABLE ~ (FROM ~ #period_separated_idents_1_to_2)? ~ #vacuum_table_option + }, + |(_, _, _, database_option, option)| { + let (catalog, database) = database_option.map_or_else( + || (None, None), + |(_, catalog_database)| (catalog_database.0, Some(catalog_database.1)), + ); + Statement::VacuumDropTable(VacuumDropTableStmt { + catalog, + database, + option, + }) + }, + ); let analyze_table = map( rule! { ANALYZE ~ TABLE ~ #period_separated_idents_1_to_3 @@ -1329,6 +1359,7 @@ pub fn statement(i: Input) -> IResult { | #show_fields : "`SHOW FIELDS FROM [.]`" | #show_tables_status : "`SHOW TABLES STATUS [FROM ] []`" | #show_drop_tables_status : "`SHOW DROP TABLES [FROM ]`" + | #attach_table : "`ATTACH TABLE [.]
`" | #create_table : "`CREATE TABLE [IF NOT EXISTS] [.]
[] []`" | #drop_table : "`DROP TABLE [IF EXISTS] [.]
`" | #undrop_table : "`UNDROP TABLE [.]
`" @@ -1337,6 +1368,7 @@ pub fn statement(i: Input) -> IResult { | #truncate_table : "`TRUNCATE TABLE [.]
[PURGE]`" | #optimize_table : "`OPTIMIZE TABLE [.]
(ALL | PURGE | COMPACT [SEGMENT])`" | #vacuum_table : "`VACUUM TABLE [.]
[RETAIN number HOURS] [DRY RUN]`" + | #vacuum_drop_table : "`VACUUM DROP TABLE [FROM [.]] [RETAIN number HOURS] [DRY RUN]`" | #analyze_table : "`ANALYZE TABLE [.]
`" | #exists_table : "`EXISTS TABLE [.]
`" | #show_table_functions : "`SHOW TABLE_FUNCTIONS []`" diff --git a/src/query/ast/src/parser/token.rs b/src/query/ast/src/parser/token.rs index da64d17cff46..fa1bf63204c5 100644 --- a/src/query/ast/src/parser/token.rs +++ b/src/query/ast/src/parser/token.rs @@ -387,6 +387,8 @@ pub enum TokenKind { COUNT, #[token("CREATE", ignore(ascii_case))] CREATE, + #[token("ATTACH", ignore(ascii_case))] + ATTACH, #[token("CREDENTIALS", ignore(ascii_case))] CREDENTIALS, #[token("CROSS", ignore(ascii_case))] @@ -1155,6 +1157,7 @@ impl TokenKind { // | TokenKind::CHAR | TokenKind::CHARACTER | TokenKind::CREATE + | TokenKind::ATTACH | TokenKind::EXCEPT // | TokenKind::FETCH | TokenKind::FOR @@ -1269,6 +1272,7 @@ impl TokenKind { | TokenKind::AS | TokenKind::BETWEEN | TokenKind::CREATE + | TokenKind::ATTACH | TokenKind::EXCEPT // | TokenKind::FETCH | TokenKind::FOR diff --git a/src/query/ast/src/visitors/visitor.rs b/src/query/ast/src/visitors/visitor.rs index 8694fa07452d..91ef34e30513 100644 --- a/src/query/ast/src/visitors/visitor.rs +++ b/src/query/ast/src/visitors/visitor.rs @@ -465,6 +465,8 @@ pub trait Visitor<'ast>: Sized { fn visit_vacuum_table(&mut self, _stmt: &'ast VacuumTableStmt) {} + fn visit_vacuum_drop_table(&mut self, _stmt: &'ast VacuumDropTableStmt) {} + fn visit_analyze_table(&mut self, _stmt: &'ast AnalyzeTableStmt) {} fn visit_exists_table(&mut self, _stmt: &'ast ExistsTableStmt) {} diff --git a/src/query/ast/src/visitors/visitor_mut.rs b/src/query/ast/src/visitors/visitor_mut.rs index 7d22e69efb09..bb78e4466f1a 100644 --- a/src/query/ast/src/visitors/visitor_mut.rs +++ b/src/query/ast/src/visitors/visitor_mut.rs @@ -480,6 +480,8 @@ pub trait VisitorMut: Sized { fn visit_vacuum_table(&mut self, _stmt: &mut VacuumTableStmt) {} + fn visit_vacuum_drop_table(&mut self, _stmt: &mut VacuumDropTableStmt) {} + fn visit_analyze_table(&mut self, _stmt: &mut AnalyzeTableStmt) {} fn visit_exists_table(&mut self, _stmt: &mut ExistsTableStmt) {} diff --git a/src/query/ast/src/visitors/walk.rs b/src/query/ast/src/visitors/walk.rs index c44a5fba73e8..508c66f5570d 100644 --- a/src/query/ast/src/visitors/walk.rs +++ b/src/query/ast/src/visitors/walk.rs @@ -381,6 +381,7 @@ pub fn walk_statement<'a, V: Visitor<'a>>(visitor: &mut V, statement: &'a Statem Statement::TruncateTable(stmt) => visitor.visit_truncate_table(stmt), Statement::OptimizeTable(stmt) => visitor.visit_optimize_table(stmt), Statement::VacuumTable(stmt) => visitor.visit_vacuum_table(stmt), + Statement::VacuumDropTable(stmt) => visitor.visit_vacuum_drop_table(stmt), Statement::AnalyzeTable(stmt) => visitor.visit_analyze_table(stmt), Statement::ExistsTable(stmt) => visitor.visit_exists_table(stmt), Statement::CreateView(stmt) => visitor.visit_create_view(stmt), @@ -471,5 +472,6 @@ pub fn walk_statement<'a, V: Visitor<'a>>(visitor: &mut V, statement: &'a Statem Statement::CreateDatamaskPolicy(stmt) => visitor.visit_create_data_mask_policy(stmt), Statement::DropDatamaskPolicy(stmt) => visitor.visit_drop_data_mask_policy(stmt), Statement::DescDatamaskPolicy(stmt) => visitor.visit_desc_data_mask_policy(stmt), + Statement::AttachTable(_) => {} } } diff --git a/src/query/ast/src/visitors/walk_mut.rs b/src/query/ast/src/visitors/walk_mut.rs index 360efec447e1..0e3aa7d38aaf 100644 --- a/src/query/ast/src/visitors/walk_mut.rs +++ b/src/query/ast/src/visitors/walk_mut.rs @@ -356,6 +356,7 @@ pub fn walk_statement_mut(visitor: &mut V, statement: &mut Statem Statement::TruncateTable(stmt) => visitor.visit_truncate_table(stmt), Statement::OptimizeTable(stmt) => visitor.visit_optimize_table(stmt), Statement::VacuumTable(stmt) => visitor.visit_vacuum_table(stmt), + Statement::VacuumDropTable(stmt) => visitor.visit_vacuum_drop_table(stmt), Statement::AnalyzeTable(stmt) => visitor.visit_analyze_table(stmt), Statement::ExistsTable(stmt) => visitor.visit_exists_table(stmt), Statement::CreateView(stmt) => visitor.visit_create_view(stmt), @@ -446,5 +447,6 @@ pub fn walk_statement_mut(visitor: &mut V, statement: &mut Statem Statement::CreateDatamaskPolicy(stmt) => visitor.visit_create_data_mask_policy(stmt), Statement::DropDatamaskPolicy(stmt) => visitor.visit_drop_data_mask_policy(stmt), Statement::DescDatamaskPolicy(stmt) => visitor.visit_desc_data_mask_policy(stmt), + Statement::AttachTable(_) => {} } } diff --git a/src/query/ast/tests/it/parser.rs b/src/query/ast/tests/it/parser.rs index ed17a1124cd7..a596d78bfdfd 100644 --- a/src/query/ast/tests/it/parser.rs +++ b/src/query/ast/tests/it/parser.rs @@ -192,6 +192,9 @@ fn test_statement() { r#"VACUUM TABLE t;"#, r#"VACUUM TABLE t RETAIN 4 HOURS DRY RUN;"#, r#"VACUUM TABLE t RETAIN 40 HOURS;"#, + r#"VACUUM DROP TABLE RETAIN 20 HOURS;"#, + r#"VACUUM DROP TABLE RETAIN 30 HOURS DRY RUN;"#, + r#"VACUUM DROP TABLE FROM db RETAIN 40 HOURS;"#, r#"CREATE TABLE t (a INT COMMENT 'col comment') COMMENT='table comment';"#, r#"GRANT CREATE, CREATE USER ON * TO 'test-grant';"#, r#"GRANT SELECT, CREATE ON * TO 'test-grant';"#, diff --git a/src/query/ast/tests/it/testdata/statement.txt b/src/query/ast/tests/it/testdata/statement.txt index 616c506fe075..e94df45733f2 100644 --- a/src/query/ast/tests/it/testdata/statement.txt +++ b/src/query/ast/tests/it/testdata/statement.txt @@ -7579,7 +7579,7 @@ VacuumTable( ---------- Input ---------- VACUUM TABLE t RETAIN 4 HOURS DRY RUN; ---------- Output --------- -VACUUM TABLE t RETAIN 4 HOURS DRY RUN +VACUUM TABLE t RETAIN 4 HOURS DRY RUN ---------- AST ------------ VacuumTable( VacuumTableStmt { @@ -7614,7 +7614,7 @@ VacuumTable( ---------- Input ---------- VACUUM TABLE t RETAIN 40 HOURS; ---------- Output --------- -VACUUM TABLE t RETAIN 40 HOURS +VACUUM TABLE t RETAIN 40 HOURS ---------- AST ------------ VacuumTable( VacuumTableStmt { @@ -7644,6 +7644,94 @@ VacuumTable( ) +---------- Input ---------- +VACUUM DROP TABLE RETAIN 20 HOURS; +---------- Output --------- +VACUUM DROP TABLE RETAIN 20 HOURS +---------- AST ------------ +VacuumDropTable( + VacuumDropTableStmt { + catalog: None, + database: None, + option: VacuumTableOption { + retain_hours: Some( + Literal { + span: Some( + 25..27, + ), + lit: UInt64( + 20, + ), + }, + ), + dry_run: None, + }, + }, +) + + +---------- Input ---------- +VACUUM DROP TABLE RETAIN 30 HOURS DRY RUN; +---------- Output --------- +VACUUM DROP TABLE RETAIN 30 HOURS DRY RUN +---------- AST ------------ +VacuumDropTable( + VacuumDropTableStmt { + catalog: None, + database: None, + option: VacuumTableOption { + retain_hours: Some( + Literal { + span: Some( + 25..27, + ), + lit: UInt64( + 30, + ), + }, + ), + dry_run: Some( + (), + ), + }, + }, +) + + +---------- Input ---------- +VACUUM DROP TABLE FROM db RETAIN 40 HOURS; +---------- Output --------- +VACUUM DROP TABLE FROM db RETAIN 40 HOURS +---------- AST ------------ +VacuumDropTable( + VacuumDropTableStmt { + catalog: None, + database: Some( + Identifier { + name: "db", + quote: None, + span: Some( + 23..25, + ), + }, + ), + option: VacuumTableOption { + retain_hours: Some( + Literal { + span: Some( + 33..35, + ), + lit: UInt64( + 40, + ), + }, + ), + dry_run: None, + }, + }, +) + + ---------- Input ---------- CREATE TABLE t (a INT COMMENT 'col comment') COMMENT='table comment'; ---------- Output --------- diff --git a/src/query/catalog/src/plan/datasource/datasource_info/parquet.rs b/src/query/catalog/src/plan/datasource/datasource_info/parquet.rs index e83d0c932604..654f1bb50fa9 100644 --- a/src/query/catalog/src/plan/datasource/datasource_info/parquet.rs +++ b/src/query/catalog/src/plan/datasource/datasource_info/parquet.rs @@ -34,6 +34,7 @@ pub struct ParquetTableInfo { pub arrow_schema: ArrowSchema, pub schema_descr: SchemaDescriptor, pub files_to_read: Option>, + pub schema_from: String, } impl ParquetTableInfo { diff --git a/src/query/ee-features/vacuum-handler/src/vacuum_handler.rs b/src/query/ee-features/vacuum-handler/src/vacuum_handler.rs index 883f27e74056..0430e3dc723a 100644 --- a/src/query/ee-features/vacuum-handler/src/vacuum_handler.rs +++ b/src/query/ee-features/vacuum-handler/src/vacuum_handler.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use chrono::DateTime; use chrono::Utc; use common_base::base::GlobalInstance; +use common_catalog::table::Table; use common_catalog::table_context::TableContext; use common_exception::Result; use common_storages_fuse::FuseTable; @@ -30,6 +31,12 @@ pub trait VacuumHandler: Sync + Send { retention_time: DateTime, dry_run: bool, ) -> Result>>; + + async fn do_vacuum_drop_tables( + &self, + tables: Vec>, + dry_run_limit: Option, + ) -> Result>>; } pub struct VacuumHandlerWrapper { @@ -53,6 +60,17 @@ impl VacuumHandlerWrapper { .do_vacuum(fuse_table, ctx, retention_time, dry_run) .await } + + #[async_backtrace::framed] + pub async fn do_vacuum_drop_tables( + &self, + tables: Vec>, + dry_run_limit: Option, + ) -> Result>> { + self.handler + .do_vacuum_drop_tables(tables, dry_run_limit) + .await + } } pub fn get_vacuum_handler() -> Arc { diff --git a/src/query/ee/src/storages/fuse/mod.rs b/src/query/ee/src/storages/fuse/mod.rs index 52b7bde29dd8..bb1ee60f4378 100644 --- a/src/query/ee/src/storages/fuse/mod.rs +++ b/src/query/ee/src/storages/fuse/mod.rs @@ -17,4 +17,5 @@ pub mod operations; pub use io::snapshots::get_snapshot_referenced_segments; pub use operations::vacuum::do_vacuum; +pub use operations::vacuum_drop_tables::do_vacuum_drop_tables; pub use operations::virtual_columns::do_generate_virtual_columns; diff --git a/src/query/ee/src/storages/fuse/operations/handler.rs b/src/query/ee/src/storages/fuse/operations/handler.rs index 618eb7832956..17f61979ad49 100644 --- a/src/query/ee/src/storages/fuse/operations/handler.rs +++ b/src/query/ee/src/storages/fuse/operations/handler.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use chrono::DateTime; use chrono::Utc; use common_base::base::GlobalInstance; +use common_catalog::table::Table; use common_catalog::table_context::TableContext; use common_exception::Result; use common_storages_fuse::FuseTable; @@ -24,6 +25,7 @@ use vacuum_handler::VacuumHandler; use vacuum_handler::VacuumHandlerWrapper; use crate::storages::fuse::do_vacuum; +use crate::storages::fuse::do_vacuum_drop_tables; pub struct RealVacuumHandler {} @@ -38,6 +40,14 @@ impl VacuumHandler for RealVacuumHandler { ) -> Result>> { do_vacuum(fuse_table, ctx, retention_time, dry_run).await } + + async fn do_vacuum_drop_tables( + &self, + tables: Vec>, + dry_run_limit: Option, + ) -> Result>> { + do_vacuum_drop_tables(tables, dry_run_limit).await + } } impl RealVacuumHandler { diff --git a/src/query/ee/src/storages/fuse/operations/mod.rs b/src/query/ee/src/storages/fuse/operations/mod.rs index 0cd68aad71fb..6dd601dd2b90 100644 --- a/src/query/ee/src/storages/fuse/operations/mod.rs +++ b/src/query/ee/src/storages/fuse/operations/mod.rs @@ -14,5 +14,6 @@ pub mod handler; pub mod vacuum; +pub mod vacuum_drop_tables; pub mod virtual_columns; pub use handler::RealVacuumHandler; diff --git a/src/query/ee/src/storages/fuse/operations/vacuum.rs b/src/query/ee/src/storages/fuse/operations/vacuum.rs index ff8495f55f70..c4f68bb5cd44 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum.rs @@ -122,7 +122,7 @@ pub async fn get_snapshot_referenced_files( }; let locations_referenced = fuse_table - .get_block_locations(ctx.clone(), &segments_vec, false) + .get_block_locations(ctx.clone(), &segments_vec, false, false) .await?; let mut segments = HashSet::with_capacity(segments_vec.len()); diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_drop_tables.rs b/src/query/ee/src/storages/fuse/operations/vacuum_drop_tables.rs new file mode 100644 index 000000000000..06cc4b8a9787 --- /dev/null +++ b/src/query/ee/src/storages/fuse/operations/vacuum_drop_tables.rs @@ -0,0 +1,123 @@ +// Copyright 2023 Databend Cloud +// +// Licensed under the Elastic License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.elastic.co/licensing/elastic-license +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; +use std::time::Instant; + +use common_catalog::table::Table; +use common_exception::Result; +use common_storages_fuse::FuseTable; +use futures_util::TryStreamExt; +use opendal::EntryMode; +use opendal::Metakey; +use tracing::info; + +#[async_backtrace::framed] +async fn do_vacuum_drop_table( + table: Arc, + dry_run_limit: Option, +) -> Result>> { + // only operate fuse table + if table.engine() != "FUSE" { + info!( + "ignore table {} not of FUSE engine, table engine {}", + table.get_table_info().name, + table.engine() + ); + return Ok(None); + } + let table_info = table.get_table_info(); + // storage_params is_some means it is an external table, ignore + if table_info.meta.storage_params.is_some() { + info!("ignore external table {}", table.get_table_info().name); + return Ok(None); + } + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + + let operator = fuse_table.get_operator_ref(); + + let dir = format!("{}/", FuseTable::parse_storage_prefix(table_info)?); + info!("vacuum drop table {:?} dir {:?}", table.name(), dir); + let start = Instant::now(); + + let ret = match dry_run_limit { + None => { + let _ = operator.remove_all(&dir).await; + + Ok(None) + } + Some(dry_run_limit) => { + let mut ds = operator.list_with(&dir).delimiter("").await?; + let mut list_files = Vec::new(); + while let Some(de) = ds.try_next().await? { + let meta = operator.metadata(&de, Metakey::Mode).await?; + if EntryMode::FILE == meta.mode() { + list_files.push((fuse_table.name().to_string(), de.name().to_string())); + if list_files.len() >= dry_run_limit { + break; + } + } + } + + Ok(Some(list_files)) + } + }; + + info!( + "vacuum drop table {:?} dir {:?}, cost:{} sec", + table.name(), + dir, + start.elapsed().as_secs() + ); + ret +} + +#[async_backtrace::framed] +pub async fn do_vacuum_drop_tables( + tables: Vec>, + dry_run_limit: Option, +) -> Result>> { + let start = Instant::now(); + let tables_len = tables.len(); + info!("do_vacuum_drop_tables {} tables", tables_len); + let mut list_files = Vec::new(); + let mut left_limit = dry_run_limit; + for table in tables { + let ret = do_vacuum_drop_table(table, left_limit).await?; + if let Some(ret) = ret { + list_files.extend(ret); + if list_files.len() >= dry_run_limit.unwrap() { + info!( + "do_vacuum_drop_tables {} tables, cost:{} sec", + tables_len, + start.elapsed().as_secs() + ); + return Ok(Some(list_files)); + } else { + left_limit = Some(dry_run_limit.unwrap() - list_files.len()); + } + } + } + info!( + "do_vacuum_drop_tables {} tables, cost:{} sec", + tables_len, + start.elapsed().as_secs() + ); + + Ok(if dry_run_limit.is_some() { + Some(list_files) + } else { + None + }) +} diff --git a/src/query/ee/tests/it/aggregating_index/index_scan.rs b/src/query/ee/tests/it/aggregating_index/index_scan.rs index 550bfb80deda..a30e648acdf3 100644 --- a/src/query/ee/tests/it/aggregating_index/index_scan.rs +++ b/src/query/ee/tests/it/aggregating_index/index_scan.rs @@ -12,39 +12,44 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::fmt::Display; use std::sync::Arc; -use aggregating_index::get_agg_index_handler; -use chrono::Utc; use common_base::base::tokio; use common_exception::Result; use common_expression::block_debug::pretty_format_blocks; -use common_expression::infer_table_schema; use common_expression::DataBlock; -use common_expression::DataSchemaRefExt; use common_expression::SendableDataBlockStream; use common_expression::SortColumnDescription; -use common_meta_app::schema::CreateIndexReq; -use common_meta_app::schema::IndexMeta; -use common_meta_app::schema::IndexNameIdent; -use common_meta_app::schema::IndexType; use common_sql::optimizer::SExpr; use common_sql::planner::plans::Plan; use common_sql::plans::RelOperator; use common_sql::Planner; -use common_storages_fuse::io::serialize_block; -use common_storages_fuse::io::TableMetaLocationGenerator; -use common_storages_fuse::io::WriteSettings; -use common_storages_fuse::TableContext; use databend_query::interpreters::InterpreterFactory; use databend_query::sessions::QueryContext; use databend_query::test_kits::table_test_fixture::expects_ok; use databend_query::test_kits::TestFixture; use enterprise_query::test_kits::context::create_ee_query_context; -use futures_util::StreamExt; use futures_util::TryStreamExt; +#[tokio::test(flavor = "multi_thread")] +async fn test_index_scan() -> Result<()> { + test_index_scan_impl("parquet").await?; + test_index_scan_impl("native").await +} + +#[tokio::test(flavor = "multi_thread")] +async fn test_index_scan_two_agg_funcs() -> Result<()> { + test_index_scan_two_agg_funcs_impl("parquet").await?; + test_index_scan_two_agg_funcs_impl("native").await +} + +#[tokio::test(flavor = "multi_thread")] +async fn test_fuzz() -> Result<()> { + test_fuzz_impl("parquet").await?; + test_fuzz_impl("native").await +} + async fn plan_sql(ctx: Arc, sql: &str) -> Result { let mut planner = Planner::new(ctx.clone()); let (plan, _) = planner.plan_sql(sql).await?; @@ -62,38 +67,6 @@ async fn execute_plan(ctx: Arc, plan: &Plan) -> Result, index_name: &str, query: &str) -> Result { - let sql = format!("CREATE AGGREGATING INDEX {index_name} AS {query}"); - - let plan = plan_sql(ctx.clone(), &sql).await?; - - if let Plan::CreateIndex(plan) = plan { - let catalog = ctx.get_catalog("default")?; - let create_index_req = CreateIndexReq { - if_not_exists: plan.if_not_exists, - name_ident: IndexNameIdent { - tenant: ctx.get_tenant(), - index_name: index_name.to_string(), - }, - meta: IndexMeta { - table_id: plan.table_id, - index_type: IndexType::AGGREGATING, - created_on: Utc::now(), - dropped_on: None, - updated_on: None, - query: query.to_string(), - }, - }; - - let handler = get_agg_index_handler(); - let res = handler.do_create_index(catalog, create_index_req).await?; - - return Ok(res.index_id); - } - - unreachable!() -} - async fn drop_index(ctx: Arc, index_name: &str) -> Result<()> { let sql = format!("DROP AGGREGATING INDEX {index_name}"); execute_sql(ctx, &sql).await?; @@ -101,15 +74,14 @@ async fn drop_index(ctx: Arc, index_name: &str) -> Result<()> { Ok(()) } -#[tokio::test(flavor = "multi_thread")] -async fn test_index_scan() -> Result<()> { +async fn test_index_scan_impl(format: &str) -> Result<()> { let (_guard, ctx, _) = create_ee_query_context(None).await.unwrap(); let fixture = TestFixture::new_with_ctx(_guard, ctx).await; // Create table execute_sql( fixture.ctx(), - "CREATE TABLE t (a int, b int, c int) storage_format = 'parquet'", + &format!("CREATE TABLE t (a int, b int, c int) storage_format = '{format}'"), ) .await?; @@ -123,18 +95,16 @@ async fn test_index_scan() -> Result<()> { // Create index let index_name = "index1"; - let index_id = create_index( + execute_sql( fixture.ctx(), - index_name, - "SELECT b, SUM(a) from t WHERE c > 1 GROUP BY b", + &format!("CREATE AGGREGATING INDEX {index_name} AS SELECT b, SUM(a) from t WHERE c > 1 GROUP BY b"), ) .await?; // Refresh Index - refresh_index( + execute_sql( fixture.ctx(), - "SELECT b, SUM_state(a), _block_name from t WHERE c > 1 GROUP BY b, _block_name", - index_id, + &format!("REFRESH AGGREGATING INDEX {index_name}"), ) .await?; @@ -272,15 +242,14 @@ async fn test_index_scan() -> Result<()> { Ok(()) } -#[tokio::test(flavor = "multi_thread")] -async fn test_index_scan_two_agg_funcs() -> Result<()> { +async fn test_index_scan_two_agg_funcs_impl(format: &str) -> Result<()> { let (_guard, ctx, _) = create_ee_query_context(None).await.unwrap(); let fixture = TestFixture::new_with_ctx(_guard, ctx).await; // Create table execute_sql( fixture.ctx(), - "CREATE TABLE t (a int, b int, c int) storage_format = 'parquet'", + &format!("CREATE TABLE t (a int, b int, c int) storage_format = '{format}'"), ) .await?; @@ -294,18 +263,16 @@ async fn test_index_scan_two_agg_funcs() -> Result<()> { // Create index let index_name = "index1"; - let index_id = create_index( + execute_sql( fixture.ctx(), - index_name, - "SELECT b, MAX(a), SUM(a) from t WHERE c > 1 GROUP BY b", + &format!("CREATE AGGREGATING INDEX {index_name} AS SELECT b, MAX(a), SUM(a) from t WHERE c > 1 GROUP BY b"), ) .await?; // Refresh Index - refresh_index( + execute_sql( fixture.ctx(), - "SELECT b, MAX_state(a), SUM_state(a), _block_name from t WHERE c > 1 GROUP BY b, _block_name", - index_id, + &format!("REFRESH AGGREGATING INDEX {index_name}"), ) .await?; @@ -434,128 +401,86 @@ fn is_index_scan_sexpr(s_expr: &SExpr) -> bool { } } -async fn refresh_index(ctx: Arc, sql: &str, index_id: u64) -> Result<()> { - let plan = plan_sql(ctx.clone(), sql).await?; - let mut output_fields = plan.schema().fields().to_vec(); - assert_eq!(output_fields.last().unwrap().name(), "_block_name"); - output_fields.pop(); // Pop _block_name field - let output_schema = DataSchemaRefExt::create(output_fields); - let index_schema = infer_table_schema(&output_schema)?; - - let mut buffer = vec![]; - let mut map: HashMap> = HashMap::new(); - - let mut index = 0; - let mut stream = execute_plan(ctx.clone(), &plan).await?; - while let Some(block) = stream.next().await { - let block = block?; - let block = block.convert_to_full(); - let block_name_col = block - .columns() - .last() - .unwrap() - .value - .as_column() - .unwrap() - .as_string() - .unwrap() - .clone(); - let result = block.pop_columns(1)?; - - for (row, block_name) in block_name_col.iter().enumerate() { - let name = String::from_utf8(block_name.to_vec())?; - - map.entry(name) - .and_modify(|v| v.push((index, row as u32, 1))) - .or_insert(vec![(index, row as u32, 1)]); - } - - buffer.push(result); - index += 1; - } - - let op = ctx.get_data_operator()?.operator(); - let data = buffer.iter().collect::>(); - - for (loc, indices) in map { - let index_block = DataBlock::take_blocks(&data, &indices, indices.len()); - let mut buf = vec![]; - - serialize_block( - &WriteSettings::default(), - &index_schema, - index_block, - &mut buf, - )?; - - let index_loc = - TableMetaLocationGenerator::gen_agg_index_location_from_block_location(&loc, index_id); - - op.write(&index_loc, buf).await?; - } - - Ok(()) -} - -async fn fuzz( +struct FuzzParams { num_rows_per_block: usize, num_blocks: usize, index_block_ratio: f64, - query_sql: &str, - index_sql: &str, - refresh_sql: &str, + query_sql: String, + index_sql: String, is_index_scan: bool, -) -> Result<()> { - let fuzz_info = format_fuzz_test_params( + table_format: String, +} + +impl Display for FuzzParams { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "Query: {}\nIndex: {}\nIsIndexScan: {}\nNumBlocks: {}\nNumRowsPerBlock: {}\nIndexBlockRatio: {}\nTableFormat: {}\n", + self.query_sql, + self.index_sql, + self.is_index_scan, + self.num_blocks, + self.num_rows_per_block, + self.index_block_ratio, + self.table_format, + ) + } +} + +async fn fuzz(params: FuzzParams) -> Result<()> { + let fuzz_info = params.to_string(); + let FuzzParams { + num_rows_per_block, + num_blocks, + index_block_ratio, query_sql, index_sql, - refresh_sql, is_index_scan, - num_blocks, - num_rows_per_block, - index_block_ratio, - ); + table_format: format, + } = params; let (_guard, ctx, _) = create_ee_query_context(None).await.unwrap(); let fixture = TestFixture::new_with_ctx(_guard, ctx).await; - let ctx = fixture.ctx(); // Prepare table and data // Create random engine table execute_sql( - ctx.clone(), + fixture.ctx(), "CREATE TABLE rt (a int, b int, c int) ENGINE = RANDOM", ) .await?; execute_sql( - ctx.clone(), - "CREATE TABLE t (a int, b int, c int) storage_format = 'parquet'", + fixture.ctx(), + &format!("CREATE TABLE t (a int, b int, c int) storage_format = '{format}'"), + ) + .await?; + execute_sql( + fixture.ctx(), + &format!("CREATE TABLE temp_t (a int, b int, c int) storage_format = '{format}'"), ) .await?; execute_sql( - ctx.clone(), - "CREATE TABLE temp_t (a int, b int, c int) storage_format = 'parquet'", + fixture.ctx(), + &format!("CREATE AGGREGATING INDEX index AS {index_sql}"), ) .await?; - let index_id = create_index(ctx.clone(), "index", index_sql).await?; - let plan = plan_sql(ctx.clone(), query_sql).await?; + let plan = plan_sql(fixture.ctx(), &query_sql).await?; if !is_index_scan_plan(&plan) { assert!(!is_index_scan, "{}", fuzz_info); // Clear - drop_index(ctx.clone(), "index").await?; - execute_sql(ctx.clone(), "DROP TABLE rt ALL").await?; - execute_sql(ctx.clone(), "DROP TABLE t ALL").await?; - execute_sql(ctx, "DROP TABLE temp_t ALL").await?; + drop_index(fixture.ctx(), "index").await?; + execute_sql(fixture.ctx(), "DROP TABLE rt ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE t ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE temp_t ALL").await?; return Ok(()); } assert!(is_index_scan, "{}", fuzz_info); // Generate data with index - let num_index_blocks = (num_blocks as f64 * index_block_ratio) as usize; - for _ in 0..num_index_blocks { + for _ in 0..num_blocks { execute_sql( - ctx.clone(), + fixture.ctx(), &format!( "INSERT INTO t SELECT * FROM rt LIMIT {}", num_rows_per_block @@ -563,32 +488,28 @@ async fn fuzz( ) .await?; } - refresh_index(ctx.clone(), refresh_sql, index_id).await?; - // Generate data without index - for _ in 0..num_blocks - num_index_blocks { + let num_index_blocks = (num_blocks as f64 * index_block_ratio) as usize; + if num_index_blocks > 0 { execute_sql( - ctx.clone(), - &format!( - "INSERT INTO t SELECT * FROM rt LIMIT {}", - num_rows_per_block - ), + fixture.ctx(), + &format!("REFRESH AGGREGATING INDEX index LIMIT {num_index_blocks}"), ) .await?; } // Copy data into temp table - execute_sql(ctx.clone(), "INSERT INTO temp_t SELECT * FROM t").await?; + execute_sql(fixture.ctx(), "INSERT INTO temp_t SELECT * FROM t").await?; // Get query result let expect: Vec = - execute_sql(ctx.clone(), &query_sql.replace("from t", "from temp_t")) + execute_sql(fixture.ctx(), &query_sql.replace("from t", "from temp_t")) .await? .try_collect() .await?; // Get index scan query result - let actual: Vec = execute_sql(ctx.clone(), query_sql) + let actual: Vec = execute_sql(fixture.ctx(), &query_sql) .await? .try_collect() .await?; @@ -607,10 +528,10 @@ async fn fuzz( if expect.is_empty() { assert!(actual.is_empty()); // Clear - drop_index(ctx.clone(), "index").await?; - execute_sql(ctx.clone(), "DROP TABLE rt ALL").await?; - execute_sql(ctx.clone(), "DROP TABLE t ALL").await?; - execute_sql(ctx, "DROP TABLE temp_t ALL").await?; + drop_index(fixture.ctx(), "index").await?; + execute_sql(fixture.ctx(), "DROP TABLE rt ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE t ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE temp_t ALL").await?; return Ok(()); } @@ -650,10 +571,10 @@ async fn fuzz( ); // Clear - drop_index(ctx.clone(), "index").await?; - execute_sql(ctx.clone(), "DROP TABLE rt ALL").await?; - execute_sql(ctx.clone(), "DROP TABLE t ALL").await?; - execute_sql(ctx, "DROP TABLE temp_t ALL").await?; + drop_index(fixture.ctx(), "index").await?; + execute_sql(fixture.ctx(), "DROP TABLE rt ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE t ALL").await?; + execute_sql(fixture.ctx(), "DROP TABLE temp_t ALL").await?; Ok(()) } @@ -662,8 +583,8 @@ async fn fuzz( struct TestSuite { query: &'static str, index: &'static str, - refresh: &'static str, - is_matched: bool, + + is_index_scan: bool, } /// Generate test suites. @@ -679,282 +600,227 @@ fn get_test_suites() -> Vec { TestSuite { query: "select to_string(c + 1) from t", index: "select c + 1 from t", - refresh: "select c + 1, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select c + 1 from t", index: "select c + 1 from t", - refresh: "select c + 1, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a from t", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a as z from t", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a + 1, to_string(a) from t", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a + 1 as z, to_string(a) from t", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select b from t", index: "select a, b from t", - refresh: "select a, b, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a from t", index: "select b, c from t", - refresh: "select b, c, _block_name from t", - is_matched: false, + is_index_scan: false, }, // query: eval-filter-scan, index: eval-scan TestSuite { query: "select a from t where b > 1", index: "select b, c from t", - refresh: "select b, c, _block_name from t", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select a from t where b > 1", index: "select a, b from t", - refresh: "select a, b, _block_name from t", - is_matched: true, + is_index_scan: true, }, // query: eval-agg-eval-scan, index: eval-scan TestSuite { query: "select sum(a) from t group by b", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select avg(a + 1) from t group by b", index: "select a + 1, b from t", - refresh: "select b, a + 1, _block_name from t", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select avg(a + 1) from t", index: "select a + 1, b from t", - refresh: "select b, a + 1, _block_name from t", - is_matched: true, + is_index_scan: true, }, // query: eval-agg-eval-filter-scan, index: eval-scan TestSuite { query: "select sum(a) from t where a > 1 group by b", index: "select a from t", - refresh: "select a, _block_name from t", - is_matched: false, + is_index_scan: false, }, // query: eval-scan, index: eval-filter-scan TestSuite { query: "select a from t", index: "select a from t where b > 1", - refresh: "select a, _block_name from t where b > 1", - is_matched: false, + is_index_scan: false, }, // query: eval-filter-scan, index: eval-filter-scan TestSuite { query: "select a from t where b > 1", index: "select a, b from t where b > 2", - refresh: "select a, b, _block_name from t where b > 2", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select a from t where b > 1", index: "select a, b from t where b > 0", - refresh: "select a, b, _block_name from t where b > 0", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a from t where b < 5", index: "select a, b from t where b > 0", - refresh: "select a, b, _block_name from t where b > 0", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select a from t where b > 1 and b < 5", index: "select a, b from t where b > 0", - refresh: "select a, b, _block_name from t where b > 0", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select a from t where b > 1 and b < 5", index: "select a, b from t where b > 0 and b < 6", - refresh: "select a, b, _block_name from t where b > 0 and b < 6", - is_matched: true, + is_index_scan: true, }, // query: eval-agg-eval-scan, index: eval-filter-scan TestSuite { query: "select sum(a) from t group by b", index: "select a from t where b > 1", - refresh: "select a, _block_name from t where b > 1", - is_matched: false, + is_index_scan: false, }, // query: eval-agg-eval-filter-scan, index: eval-filter-scan TestSuite { query: "select sum(a) from t where b > 1 group by b", index: "select a from t where b > 1", - refresh: "select a, _block_name from t where b > 1", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select sum(a) from t where b > 1 group by b", index: "select a, b from t where b > 1", - refresh: "select a, b, _block_name from t where b > 1", - is_matched: true, + is_index_scan: true, }, // query: eval-scan, index: eval-agg-eval-scan TestSuite { query: "select b from t", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-filter-scan, index: eval-agg-eval-scan TestSuite { query: "select b from t where c > 1", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-agg-eval-scan, index: eval-agg-eval-scan TestSuite { query: "select sum(a) from t group by b", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select sum(a) from t group by b", index: "select sum(a) from t group by b", - refresh: "select sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select sum(a) + 1 from t group by b", index: "select sum(a) from t group by b", - refresh: "select sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select sum(a) + 1, b + 1 from t group by b", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select sum(a) from t group by c", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, TestSuite { query: "select sum(a) + 1 from t group by b", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: true, + is_index_scan: true, }, // query: eval-agg-eval-filter-scan, index: eval-agg-eval-scan TestSuite { query: "select sum(a) + 1 from t where b > 1 group by b", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select sum(a) + 1 from t where c > 1 group by b", index: "select b, sum(a) from t group by b", - refresh: "select b, sum_state(a), _block_name from t group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-scan, index: eval-agg-eval-filter-scan TestSuite { query: "select b from t", index: "select b, sum(a) from t where a > 1 group by b", - refresh: "select b, sum_state(a), _block_name from t where a > 1 group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-filter-scan, index: eval-agg-eval-filter-scan TestSuite { query: "select b from t where a > 1", index: "select b, sum(a) from t where a > 1 group by b", - refresh: "select b, sum_state(a), _block_name from t where a > 1 group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-agg-eval-scan, index: eval-agg-eval-filter-scan TestSuite { query: "select sum(a) + 1 from t group by b", index: "select b, sum(a) from t where c > 1 group by b", - refresh: "select b, sum_state(a), _block_name from t where c > 1 group by b, _block_name", - is_matched: false, + is_index_scan: false, }, // query: eval-agg-eval-filter-scan, index: eval-agg-eval-filter-scan TestSuite { query: "select sum(a) + 1 from t where c > 1 group by b", index: "select b, sum(a) from t where c > 1 group by b", - refresh: "select b, sum_state(a), _block_name from t where c > 1 group by b, _block_name", - is_matched: true, + is_index_scan: true, }, TestSuite { query: "select sum(a) + 1, b + 2 from t where b > 1 group by b", index: "select b, sum(a) from t where b > 0 group by b", - refresh: "select b, sum_state(a), _block_name from t where b > 0 group by b, _block_name", - is_matched: true, + is_index_scan: true, }, ] } -#[inline(always)] -fn format_fuzz_test_params( - query: &str, - index: &str, - refresh: &str, - is_index_scan: bool, - num_blocks: usize, - num_rows_per_block: usize, - index_block_ratio: f64, -) -> String { - format!( - "Query: {}\nIndex: {}\nRefresh: {}\nIsIndexScan: {}\nNumBlocks: {}\nNumRowsPerBlock: {}\nIndexBlockRatio: {}\n", - query, index, refresh, is_index_scan, num_blocks, num_rows_per_block, index_block_ratio, - ) -} - -#[tokio::test(flavor = "multi_thread")] -async fn test_fuzz() -> Result<()> { +async fn test_fuzz_impl(format: &str) -> Result<()> { let test_suites = get_test_suites(); for suite in test_suites { for num_blocks in [1, 10] { - for num_rows_per_block in [1, 10, 50, 100] { - for index_block_ratio in [0.0, 0.2, 0.5, 0.8, 1.0] { - fuzz( + for num_rows_per_block in [1, 50] { + for index_block_ratio in [0.2, 0.5, 0.8, 1.0] { + fuzz(FuzzParams { num_rows_per_block, num_blocks, index_block_ratio, - suite.query, - suite.index, - suite.refresh, - suite.is_matched, - ) + query_sql: suite.query.to_string(), + index_sql: suite.index.to_string(), + is_index_scan: suite.is_index_scan, + table_format: format.to_string(), + }) .await?; } } diff --git a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs index 5acf31a3f8d9..b16ccda194e4 100644 --- a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs +++ b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs @@ -22,12 +22,14 @@ use common_exception::Result; use common_storages_fuse::FuseTable; use databend_query::test_kits::table_test_fixture::append_sample_data; use databend_query::test_kits::table_test_fixture::check_data_dir; +use databend_query::test_kits::table_test_fixture::execute_command; use databend_query::test_kits::table_test_fixture::execute_query; use databend_query::test_kits::table_test_fixture::TestFixture; use databend_query::test_kits::utils::generate_orphan_files; use databend_query::test_kits::utils::generate_snapshot_with_segments; use databend_query::test_kits::utils::query_count; use enterprise_query::storages::fuse::do_vacuum; +use enterprise_query::storages::fuse::do_vacuum_drop_tables; use enterprise_query::storages::fuse::operations::vacuum::get_snapshot_referenced_files; use enterprise_query::storages::fuse::operations::vacuum::SnapshotReferencedFiles; @@ -457,3 +459,73 @@ async fn test_fuse_do_vacuum() -> Result<()> { Ok(()) } + +#[tokio::test(flavor = "multi_thread")] +async fn test_fuse_do_vacuum_drop_table() -> Result<()> { + let fixture = TestFixture::new().await; + let ctx = fixture.ctx(); + let table_ctx: Arc = ctx.clone(); + table_ctx.get_settings().set_retention_period(0)?; + fixture.create_default_table().await?; + + let number_of_block = 1; + append_sample_data(number_of_block, &fixture).await?; + + let table = fixture.latest_default_table().await?; + + check_data_dir( + &fixture, + "test_fuse_do_vacuum_drop_table: verify generate files", + 1, + 0, + 1, + 1, + 1, + None, + None, + ) + .await?; + + // do gc. + let db = fixture.default_db_name(); + let tbl = fixture.default_table_name(); + let qry = format!("drop table {}.{}", db, tbl); + let ctx = fixture.ctx(); + execute_command(ctx, &qry).await?; + + // verify dry run never delete files + { + do_vacuum_drop_tables(vec![table.clone()], Some(100)).await?; + check_data_dir( + &fixture, + "test_fuse_do_vacuum_drop_table: verify generate files", + 1, + 0, + 1, + 1, + 1, + None, + None, + ) + .await?; + } + + { + do_vacuum_drop_tables(vec![table], None).await?; + + // after vacuum drop tables, verify the files number + check_data_dir( + &fixture, + "test_fuse_do_vacuum_drop_table: verify generate retention files", + 0, + 0, + 0, + 0, + 0, + None, + None, + ) + .await?; + } + Ok(()) +} diff --git a/src/query/expression/src/kernels/filter.rs b/src/query/expression/src/kernels/filter.rs index 6b810f4e4c20..6b962798a9d9 100644 --- a/src/query/expression/src/kernels/filter.rs +++ b/src/query/expression/src/kernels/filter.rs @@ -178,7 +178,7 @@ impl Column { Column::Tuple(fields) } Column::Variant(column) => { - let bytes_per_row = column.data.len() / filter.len().max(1); + let bytes_per_row = column.data().len() / filter.len().max(1); let data_capacity = (filter.len() - filter.unset_bits()) * bytes_per_row; Self::filter_scalar_types::( @@ -324,8 +324,8 @@ impl Column { if selected == values.len() { return values.clone(); } - let data = values.data.as_slice(); - let offsets = values.offsets.as_slice(); + let data = values.data().as_slice(); + let offsets = values.offsets().as_slice(); let mut res_offsets = Vec::with_capacity(selected + 1); res_offsets.push(0); @@ -394,9 +394,6 @@ impl Column { pos += 1; } - StringColumn { - data: res_data.into(), - offsets: res_offsets.into(), - } + StringColumn::new(res_data.into(), res_offsets.into()) } } diff --git a/src/query/expression/src/row/row_converter.rs b/src/query/expression/src/row/row_converter.rs index 5c8dd11c7916..0d6b5a3a5b15 100644 --- a/src/query/expression/src/row/row_converter.rs +++ b/src/query/expression/src/row/row_converter.rs @@ -85,8 +85,8 @@ impl RowConverter { } let rows = builder.build(); - debug_assert_eq!(*rows.offsets.last().unwrap(), rows.data.len() as u64); - debug_assert!(rows.offsets.windows(2).all(|w| w[0] <= w[1])); + debug_assert_eq!(*rows.offsets().last().unwrap(), rows.data().len() as u64); + debug_assert!(rows.offsets().windows(2).all(|w| w[0] <= w[1])); rows } diff --git a/src/query/expression/src/types/bitmap.rs b/src/query/expression/src/types/bitmap.rs index abfc5827a8ce..afe30de74c8b 100644 --- a/src/query/expression/src/types/bitmap.rs +++ b/src/query/expression/src/types/bitmap.rs @@ -1,17 +1,17 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::ops::Range; use crate::property::Domain; @@ -145,7 +145,7 @@ impl ValueType for BitmapType { } fn column_memory_size(col: &Self::Column) -> usize { - col.data.len() + col.offsets.len() * 8 + col.data().len() + col.offsets().len() * 8 } } diff --git a/src/query/expression/src/types/string.rs b/src/query/expression/src/types/string.rs index 15a96ed6318c..6f571119d8ad 100644 --- a/src/query/expression/src/types/string.rs +++ b/src/query/expression/src/types/string.rs @@ -146,7 +146,7 @@ impl ValueType for StringType { } fn column_memory_size(col: &Self::Column) -> usize { - col.data.len() + col.offsets.len() * 8 + col.data().len() + col.offsets().len() * 8 } } @@ -169,15 +169,28 @@ impl ArgType for StringType { #[derive(Clone, PartialEq)] pub struct StringColumn { - pub data: Buffer, - pub offsets: Buffer, + data: Buffer, + offsets: Buffer, } impl StringColumn { + pub fn new(data: Buffer, offsets: Buffer) -> Self { + debug_assert!({ offsets.windows(2).all(|w| w[0] <= w[1]) }); + StringColumn { data, offsets } + } + pub fn len(&self) -> usize { self.offsets.len() - 1 } + pub fn data(&self) -> &Buffer { + &self.data + } + + pub fn offsets(&self) -> &Buffer { + &self.offsets + } + pub fn memory_size(&self) -> usize { let offsets = self.offsets.as_slice(); let len = offsets.len(); @@ -364,10 +377,7 @@ impl StringColumnBuilder { } pub fn build(self) -> StringColumn { - StringColumn { - data: self.data.into(), - offsets: self.offsets.into(), - } + StringColumn::new(self.data.into(), self.offsets.into()) } pub fn build_scalar(self) -> Vec { diff --git a/src/query/expression/src/types/variant.rs b/src/query/expression/src/types/variant.rs index 0052e90ece67..bbb1c483b8c7 100644 --- a/src/query/expression/src/types/variant.rs +++ b/src/query/expression/src/types/variant.rs @@ -155,7 +155,7 @@ impl ValueType for VariantType { } fn column_memory_size(col: &Self::Column) -> usize { - col.data.len() + col.offsets.len() * 8 + col.data().len() + col.offsets().len() * 8 } } diff --git a/src/query/expression/src/utils/display.rs b/src/query/expression/src/utils/display.rs index 1d09a6c7d57a..d26ee13a1386 100755 --- a/src/query/expression/src/utils/display.rs +++ b/src/query/expression/src/utils/display.rs @@ -378,8 +378,11 @@ impl Debug for DecimalColumn { impl Debug for StringColumn { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { f.debug_struct("StringColumn") - .field("data", &format_args!("0x{}", &hex::encode(&*self.data))) - .field("offsets", &self.offsets) + .field( + "data", + &format_args!("0x{}", &hex::encode(self.data().as_slice())), + ) + .field("offsets", &self.offsets()) .finish() } } diff --git a/src/query/expression/src/values.rs b/src/query/expression/src/values.rs index b4b2206fcd91..e6ad6841ddbc 100755 --- a/src/query/expression/src/values.rs +++ b/src/query/expression/src/values.rs @@ -1060,12 +1060,12 @@ impl Column { ), Column::String(col) => { let offsets: Buffer = - col.offsets.iter().map(|offset| *offset as i64).collect(); + col.offsets().iter().map(|offset| *offset as i64).collect(); Box::new( common_arrow::arrow::array::BinaryArray::::try_new( arrow_type, unsafe { OffsetsBuffer::new_unchecked(offsets) }, - col.data.clone(), + col.data().clone(), None, ) .unwrap(), @@ -1129,12 +1129,12 @@ impl Column { } Column::Bitmap(col) => { let offsets: Buffer = - col.offsets.iter().map(|offset| *offset as i64).collect(); + col.offsets().iter().map(|offset| *offset as i64).collect(); Box::new( common_arrow::arrow::array::BinaryArray::::try_new( arrow_type, unsafe { OffsetsBuffer::new_unchecked(offsets) }, - col.data.clone(), + col.data().clone(), None, ) .unwrap(), @@ -1154,12 +1154,12 @@ impl Column { ), Column::Variant(col) => { let offsets: Buffer = - col.offsets.iter().map(|offset| *offset as i64).collect(); + col.offsets().iter().map(|offset| *offset as i64).collect(); Box::new( common_arrow::arrow::array::BinaryArray::::try_new( arrow_type, unsafe { OffsetsBuffer::new_unchecked(offsets) }, - col.data.clone(), + col.data().clone(), None, ) .unwrap(), @@ -1329,10 +1329,7 @@ impl Column { let offsets = arrow_col.offsets().clone().into_inner(); let offsets = unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::String(StringColumn { - data: arrow_col.values().clone(), - offsets, - }) + Column::String(StringColumn::new(arrow_col.values().clone(), offsets)) } // TODO: deprecate it and use LargeBinary instead ArrowDataType::Binary => { @@ -1347,10 +1344,10 @@ impl Column { .map(|x| *x as u64) .collect::>(); - Column::String(StringColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) + Column::String(StringColumn::new( + arrow_col.values().clone(), + offsets.into(), + )) } // TODO: deprecate it and use LargeBinary instead ArrowDataType::Utf8 => { @@ -1365,10 +1362,10 @@ impl Column { .map(|x| *x as u64) .collect::>(); - Column::String(StringColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) + Column::String(StringColumn::new( + arrow_col.values().clone(), + offsets.into(), + )) } // TODO: deprecate it and use LargeBinary instead ArrowDataType::LargeUtf8 => { @@ -1382,10 +1379,10 @@ impl Column { .iter() .map(|x| *x as u64) .collect::>(); - Column::String(StringColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) + Column::String(StringColumn::new( + arrow_col.values().clone(), + offsets.into(), + )) } ArrowType::Timestamp(uint, _) => { @@ -1420,21 +1417,40 @@ impl Column { .values() .clone(), ), - ArrowDataType::Extension(name, _, None) if name == ARROW_EXT_TYPE_VARIANT => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read from arrow: array should be `BinaryArray`"); - let offsets = arrow_col - .offsets() - .buffer() - .iter() - .map(|x| *x as u64) - .collect::>(); - Column::Variant(StringColumn { - data: arrow_col.values().clone(), - offsets: offsets.into(), - }) + ArrowDataType::Extension(name, box ty, None) if name == ARROW_EXT_TYPE_VARIANT => { + match ty { + ArrowDataType::LargeBinary => { + let arrow_col = arrow_col + .as_any() + .downcast_ref::>() + .expect("fail to read from arrow: array should be `BinaryArray`"); + let offsets = arrow_col.offsets().clone().into_inner(); + + let offsets = + unsafe { std::mem::transmute::, Buffer>(offsets) }; + + Column::Variant(StringColumn::new(arrow_col.values().clone(), offsets)) + } + ArrowDataType::Binary => { + let arrow_col = arrow_col + .as_any() + .downcast_ref::>() + .expect("fail to read from arrow: array should be `BinaryArray`"); + let offsets = arrow_col + .offsets() + .buffer() + .iter() + .map(|x| *x as u64) + .collect::>(); + Column::Variant(StringColumn::new( + arrow_col.values().clone(), + offsets.into(), + )) + } + _ => unreachable!( + "fail to read from arrow: array should be `BinaryArray` or `BinaryArray`" + ), + } } ArrowDataType::List(f) => { let array_list = arrow_cast::cast( @@ -1542,18 +1558,39 @@ impl Column { scale: *scale as u8, })) } - ArrowDataType::Extension(name, _, None) if name == ARROW_EXT_TYPE_BITMAP => { - let arrow_col = arrow_col - .as_any() - .downcast_ref::>() - .expect("fail to read from arrow: array should be `BinaryArray`"); - let offsets = arrow_col.offsets().clone().into_inner(); - - let offsets = unsafe { std::mem::transmute::, Buffer>(offsets) }; - Column::Bitmap(StringColumn { - data: arrow_col.values().clone(), - offsets, - }) + ArrowDataType::Extension(name, box ty, None) if name == ARROW_EXT_TYPE_BITMAP => { + match ty { + ArrowDataType::LargeBinary => { + let arrow_col = arrow_col + .as_any() + .downcast_ref::>() + .expect("fail to read from arrow: array should be `BinaryArray`"); + let offsets = arrow_col.offsets().clone().into_inner(); + + let offsets = + unsafe { std::mem::transmute::, Buffer>(offsets) }; + Column::Bitmap(StringColumn::new(arrow_col.values().clone(), offsets)) + } + ArrowDataType::Binary => { + let arrow_col = arrow_col + .as_any() + .downcast_ref::>() + .expect("fail to read from arrow: array should be `BinaryArray`"); + let offsets = arrow_col + .offsets() + .buffer() + .iter() + .map(|x| *x as u64) + .collect::>(); + Column::Bitmap(StringColumn::new( + arrow_col.values().clone(), + offsets.into(), + )) + } + _ => unreachable!( + "fail to read from arrow: array should be `BinaryArray` or `BinaryArray`" + ), + } } ty => unimplemented!("unsupported arrow type {ty:?}"), }; diff --git a/src/query/expression/tests/it/row.rs b/src/query/expression/tests/it/row.rs index efe8233e08b3..dc879936e840 100644 --- a/src/query/expression/tests/it/row.rs +++ b/src/query/expression/tests/it/row.rs @@ -67,9 +67,12 @@ fn test_fixed_width() { let rows = converter.convert_columns(&cols, cols[0].len()); - assert_eq!(rows.offsets, vec![0, 8, 16, 24, 32, 40, 48, 56].into()); assert_eq!( - rows.data, + rows.offsets().clone(), + vec![0, 8, 16, 24, 32, 40, 48, 56].into() + ); + assert_eq!( + rows.data().clone(), vec![ 1, 128, 1, // 1, 191, 166, 102, 102, // @@ -467,12 +470,13 @@ fn fuzz_test() { // arrow_ord does not support LargeBinary converted from Databend String Column::Nullable(c) => match &c.column { Column::String(sc) => { - let offsets = sc.offsets.iter().map(|offset| *offset as i32).collect(); + let offsets = + sc.offsets().iter().map(|offset| *offset as i32).collect(); let array = Box::new( common_arrow::arrow::array::Utf8Array::::try_new( common_arrow::arrow::datatypes::DataType::Utf8, unsafe { OffsetsBuffer::new_unchecked(offsets) }, - sc.data.clone(), + sc.data().clone(), None, ) .unwrap(), diff --git a/src/query/functions/src/scalars/comparison.rs b/src/query/functions/src/scalars/comparison.rs index 4a59f77fad6e..3025d6bb2f3f 100644 --- a/src/query/functions/src/scalars/comparison.rs +++ b/src/query/functions/src/scalars/comparison.rs @@ -46,6 +46,7 @@ use common_expression::FunctionSignature; use common_expression::ScalarRef; use common_expression::SimpleDomainCmp; use common_expression::ValueRef; +use memchr::memchr; use memchr::memmem; use regex::bytes::Regex; @@ -481,7 +482,7 @@ fn register_like(registry: &mut FunctionRegistry) { FunctionDomain::Full }, vectorize_like(|str, pat, _, pattern_type| { - match pattern_type { + match &pattern_type { PatternType::OrdinalStr => str == pat, PatternType::EndOfPercent => { // fast path, can use starts_with @@ -502,7 +503,10 @@ fn register_like(registry: &mut FunctionRegistry) { } } - PatternType::PatternStr => like(str, pat), + PatternType::SimplePattern(simple_pattern) => { + simple_like(str, simple_pattern.0, simple_pattern.1, &simple_pattern.2) + } + PatternType::ComplexPattern => like(str, pat), } }), ); @@ -531,13 +535,13 @@ fn register_like(registry: &mut FunctionRegistry) { } fn vectorize_like( - func: impl Fn(&[u8], &[u8], &mut EvalContext, PatternType) -> bool + Copy, + func: impl Fn(&[u8], &[u8], &mut EvalContext, &PatternType) -> bool + Copy, ) -> impl Fn(ValueRef, ValueRef, &mut EvalContext) -> Value + Copy { move |arg1, arg2, ctx| match (arg1, arg2) { (ValueRef::Scalar(arg1), ValueRef::Scalar(arg2)) => { let pattern_type = check_pattern_type(arg2, false); - Value::Scalar(func(arg1, arg2, ctx, pattern_type)) + Value::Scalar(func(arg1, arg2, ctx, &pattern_type)) } (ValueRef::Column(arg1), ValueRef::Scalar(arg2)) => { let arg1_iter = StringType::iter_column(&arg1); @@ -553,7 +557,7 @@ fn vectorize_like( let mut builder = MutableBitmap::with_capacity(arg1.len()); for arg1 in arg1_iter { - builder.push(func(arg1, arg2, ctx, pattern_type)); + builder.push(func(arg1, arg2, ctx, &pattern_type)); } Value::Column(builder.into()) } @@ -562,7 +566,7 @@ fn vectorize_like( let mut builder = MutableBitmap::with_capacity(arg2.len()); for arg2 in arg2_iter { let pattern_type = check_pattern_type(arg2, false); - builder.push(func(arg1, arg2, ctx, pattern_type)); + builder.push(func(arg1, arg2, ctx, &pattern_type)); } Value::Column(builder.into()) } @@ -572,7 +576,7 @@ fn vectorize_like( let mut builder = MutableBitmap::with_capacity(arg2.len()); for (arg1, arg2) in arg1_iter.zip(arg2_iter) { let pattern_type = check_pattern_type(arg2, false); - builder.push(func(arg1, arg2, ctx, pattern_type)); + builder.push(func(arg1, arg2, ctx, &pattern_type)); } Value::Column(builder.into()) } @@ -628,18 +632,21 @@ fn vectorize_regexp( } } -#[derive(Copy, Clone, Debug, Eq, PartialEq, Hash)] +#[derive(Debug, Eq, PartialEq, Hash)] pub enum PatternType { - // e.g. 'Arrow' + // e.g. 'Arrow'. OrdinalStr, - // e.g. 'A%row' - PatternStr, - // e.g. '%rrow' + // e.g. '%rrow'. StartOfPercent, - // e.g. 'Arrow%' + // e.g. 'Arrow%'. EndOfPercent, - // e.g. '%Arrow%' + // e.g. '%Arrow%'. SurroundByPercent, + // Only includes %, e.g. 'A%r%w'. + // SimplePattern is composed of: (has_start_percent, has_end_percent, segments). + SimplePattern((bool, bool, Vec>)), + // e.g. 'A%row', 'A_row', 'A\\%row'. + ComplexPattern, } #[inline] @@ -669,32 +676,40 @@ pub fn check_pattern_type(pattern: &[u8], is_pruning: bool) -> PatternType { } let mut index = 0; - let start_percent = pattern[0] == b'%'; - if start_percent { + let mut first_non_percent = 0; + let mut percent_num = 0; + let has_start_percent = pattern[0] == b'%'; + let mut has_end_percent = false; + let mut segments = Vec::new(); + let mut simple_pattern = true; + if has_start_percent { if is_pruning { - return PatternType::PatternStr; + return PatternType::ComplexPattern; } index += 1; + first_non_percent += 1; + percent_num += 1; } while index < len { match pattern[index] { - b'_' => return PatternType::PatternStr, + b'_' => return PatternType::ComplexPattern, b'%' => { + percent_num += 1; + if index > first_non_percent { + segments.push(pattern[first_non_percent..index].to_vec()); + } + first_non_percent = index + 1; if index == len - 1 { - return if !start_percent { - PatternType::EndOfPercent - } else { - PatternType::SurroundByPercent - }; + has_end_percent = true; } - return PatternType::PatternStr; } b'\\' => { + simple_pattern = false; if index < len - 1 { index += 1; if !is_pruning && is_like_pattern_escape(pattern[index] as char) { - return PatternType::PatternStr; + return PatternType::ComplexPattern; } } } @@ -703,10 +718,21 @@ pub fn check_pattern_type(pattern: &[u8], is_pruning: bool) -> PatternType { index += 1; } - if start_percent { - PatternType::StartOfPercent - } else { - PatternType::OrdinalStr + match percent_num { + 0 => PatternType::OrdinalStr, + 1 if has_start_percent => PatternType::StartOfPercent, + 1 if has_end_percent => PatternType::EndOfPercent, + 2 if has_start_percent && has_end_percent => PatternType::SurroundByPercent, + _ => { + if simple_pattern { + if first_non_percent < len { + segments.push(pattern[first_non_percent..len].to_vec()); + } + PatternType::SimplePattern((has_start_percent, has_end_percent, segments)) + } else { + PatternType::ComplexPattern + } + } } } @@ -776,3 +802,149 @@ fn like(haystack: &[u8], pattern: &[u8]) -> bool { } true } + +fn find(mut haystack: &[u8], needle: &[u8]) -> Option { + let haystack_len = haystack.len(); + let needle_len = needle.len(); + if needle_len > haystack_len { + return None; + } + let offset = memchr(needle[0], haystack)?; + // # Safety + // The `offset` returned by `memchr` is less than `haystack_len`. + haystack = unsafe { haystack.get_unchecked(offset..) }; + let haystack_len = haystack.len(); + if needle_len > haystack_len { + return None; + } + // Inspired by fast_strstr (https://github.com/RaphaelJ/fast_strstr). + let mut checksum = 0; + for i in 0..needle_len { + // # Safety + // `needle_len` <= haystack_len + unsafe { + checksum += haystack.get_unchecked(i); + checksum -= needle.get_unchecked(i); + } + } + let mut idx = 0; + loop { + // # Safety + // `idx` < `haystack_len` and `idx` + `needle_len` <= `haystack_len`. + unsafe { + if checksum == 0 + && haystack[idx] == needle[0] + && haystack.get_unchecked(idx..(idx + needle_len)) == needle + { + return Some(offset + idx + needle_len); + } + } + if idx + needle_len >= haystack_len { + return None; + } + // # Safety + // `idx` < `haystack_len` and `idx` + `needle_len` < `haystack_len`. + unsafe { + checksum -= haystack.get_unchecked(idx); + checksum += haystack.get_unchecked(idx + needle_len); + } + idx += 1; + } +} + +#[inline] +fn simple_like( + haystack: &[u8], + has_start_percent: bool, + has_end_percent: bool, + segments: &Vec>, +) -> bool { + let haystack_len = haystack.len(); + if haystack_len == 0 { + return false; + } + let segments_len = segments.len(); + debug_assert!(haystack_len > 0); + debug_assert!(segments_len > 1); + let mut haystack_start_idx = 0; + let mut segment_idx = 0; + if !has_start_percent { + let segment = &segments[0]; + let haystack_end = haystack_start_idx + segment.len(); + if haystack_end > haystack_len { + return false; + } + // # Safety + // `haystack_start_idx` = 0, `haystack_len` > 0, `haystack_end` <= `haystack_len`. + if unsafe { haystack.get_unchecked(haystack_start_idx..haystack_end) } != segment { + return false; + } + haystack_start_idx = haystack_end; + segment_idx += 1; + } + while segment_idx < segments_len { + if haystack_start_idx >= haystack_len { + return false; + } + let segment = &segments[segment_idx]; + if segment_idx == segments_len - 1 && !has_end_percent { + if haystack_len - haystack_start_idx < segment.len() { + return false; + } + // # Safety + // `haystack_start_idx` + `segment.len()` <= `haystack_len`. + if unsafe { haystack.get_unchecked((haystack_len - segment.len())..) } != segment { + return false; + } + } else if let Some(offset) = + unsafe { find(haystack.get_unchecked(haystack_start_idx..), segment) } + { + haystack_start_idx += offset; + } else { + return false; + } + segment_idx += 1; + } + true +} + +#[test] +fn test_check_pattern_type() { + let segments = vec![ + "databend".as_bytes().to_vec(), + "cloud".as_bytes().to_vec(), + "data".as_bytes().to_vec(), + "warehouse".as_bytes().to_vec(), + ]; + let test_cases = vec![ + ("databend", PatternType::OrdinalStr), + ("%databend", PatternType::StartOfPercent), + ("databend%", PatternType::EndOfPercent), + ("%databend%", PatternType::SurroundByPercent), + ( + "databend%cloud%data%warehouse", + PatternType::SimplePattern((false, false, segments.clone())), + ), + ( + "%databend%cloud%data%warehouse", + PatternType::SimplePattern((true, false, segments.clone())), + ), + ( + "databend%cloud%data%warehouse%", + PatternType::SimplePattern((false, true, segments.clone())), + ), + ( + "%databend%cloud%data%warehouse%", + PatternType::SimplePattern((true, true, segments)), + ), + ("databend_cloud%data%warehouse", PatternType::ComplexPattern), + ( + "databend\\%cloud%data%warehouse", + PatternType::ComplexPattern, + ), + ("databend%cloud_data%warehouse", PatternType::ComplexPattern), + ]; + for (pattern, pattern_type) in test_cases { + assert_eq!(pattern_type, check_pattern_type(pattern.as_bytes(), false)); + } +} diff --git a/src/query/functions/src/scalars/hash.rs b/src/query/functions/src/scalars/hash.rs index fb0ff2414fdc..5afb95938114 100644 --- a/src/query/functions/src/scalars/hash.rs +++ b/src/query/functions/src/scalars/hash.rs @@ -78,7 +78,7 @@ pub fn register(registry: &mut FunctionRegistry) { "md5", |_, _| FunctionDomain::MayThrow, vectorize_string_to_string( - |col| col.data.len() * 32, + |col| col.data().len() * 32, |val, output, ctx| { // TODO md5 lib doesn't allow encode into buffer... let old_len = output.data.len(); @@ -98,7 +98,7 @@ pub fn register(registry: &mut FunctionRegistry) { "sha", |_, _| FunctionDomain::MayThrow, vectorize_string_to_string( - |col| col.data.len() * 40, + |col| col.data().len() * 40, |val, output, ctx| { let old_len = output.data.len(); output.data.resize(old_len + 40, 0); @@ -120,7 +120,7 @@ pub fn register(registry: &mut FunctionRegistry) { "blake3", |_, _| FunctionDomain::MayThrow, vectorize_string_to_string( - |col| col.data.len() * 64, + |col| col.data().len() * 64, |val, output, ctx| { let old_len = output.data.len(); output.data.resize(old_len + 64, 0); diff --git a/src/query/functions/src/scalars/other.rs b/src/query/functions/src/scalars/other.rs index f6b44460982e..153123296aca 100644 --- a/src/query/functions/src/scalars/other.rs +++ b/src/query/functions/src/scalars/other.rs @@ -229,10 +229,7 @@ pub fn register(registry: &mut FunctionRegistry) { write!(&mut values, "{:x}", value).unwrap(); } - let col = StringColumn { - data: values.into(), - offsets: offsets.into(), - }; + let col = StringColumn::new(values.into(), offsets.into()); Value::Column(col) }, ); diff --git a/src/query/functions/src/scalars/string.rs b/src/query/functions/src/scalars/string.rs index 547580208104..6460af32548d 100644 --- a/src/query/functions/src/scalars/string.rs +++ b/src/query/functions/src/scalars/string.rs @@ -50,7 +50,7 @@ pub fn register(registry: &mut FunctionRegistry) { "upper", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { for (start, end, ch) in val.char_indices() { if ch == '\u{FFFD}' { @@ -73,7 +73,7 @@ pub fn register(registry: &mut FunctionRegistry) { "lower", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { for (start, end, ch) in val.char_indices() { if ch == '\u{FFFD}' { @@ -105,9 +105,9 @@ pub fn register(registry: &mut FunctionRegistry) { ValueRef::Scalar(s) => Value::Scalar(s.len() as u64), ValueRef::Column(c) => { let diffs = c - .offsets + .offsets() .iter() - .zip(c.offsets.iter().skip(1)) + .zip(c.offsets().iter().skip(1)) .map(|(a, b)| b - a) .collect::>(); @@ -302,7 +302,7 @@ pub fn register(registry: &mut FunctionRegistry) { "to_base64", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len() * 4 / 3 + col.len() * 4, + |col| col.data().len() * 4 / 3 + col.len() * 4, |val, output, _| { base64::write::EncoderWriter::new(&mut output.data, &general_purpose::STANDARD) .write_all(val) @@ -316,7 +316,7 @@ pub fn register(registry: &mut FunctionRegistry) { "from_base64", |_, _| FunctionDomain::MayThrow, vectorize_string_to_string( - |col| col.data.len() * 4 / 3 + col.len() * 4, + |col| col.data().len() * 4 / 3 + col.len() * 4, |val, output, ctx| { if let Err(err) = general_purpose::STANDARD.decode_vec(val, &mut output.data) { ctx.set_error(output.len(), err.to_string()); @@ -330,7 +330,7 @@ pub fn register(registry: &mut FunctionRegistry) { "quote", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len() * 2, + |col| col.data().len() * 2, |val, output, _| { for ch in val { match ch { @@ -354,7 +354,7 @@ pub fn register(registry: &mut FunctionRegistry) { "reverse", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { let start = output.data.len(); output.put_slice(val); @@ -384,7 +384,7 @@ pub fn register(registry: &mut FunctionRegistry) { "ltrim", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { let pos = val.iter().position(|ch| *ch != b' ' && *ch != b'\t'); if let Some(idx) = pos { @@ -399,7 +399,7 @@ pub fn register(registry: &mut FunctionRegistry) { "rtrim", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { let pos = val.iter().rev().position(|ch| *ch != b' ' && *ch != b'\t'); if let Some(idx) = pos { @@ -414,7 +414,7 @@ pub fn register(registry: &mut FunctionRegistry) { "trim", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len(), + |col| col.data().len(), |val, output, _| { let start_pos = val.iter().position(|ch| *ch != b' ' && *ch != b'\t'); let end_pos = val.iter().rev().position(|ch| *ch != b' ' && *ch != b'\t'); @@ -430,7 +430,7 @@ pub fn register(registry: &mut FunctionRegistry) { "trim_leading", |_, _, _| FunctionDomain::Full, vectorize_string_to_string_2_arg( - |col, _| col.data.len(), + |col, _| col.data().len(), |val, trim_str, _, output| { let chunk_size = trim_str.len(); let pos = val.chunks(chunk_size).position(|chunk| chunk != trim_str); @@ -446,7 +446,7 @@ pub fn register(registry: &mut FunctionRegistry) { "trim_trailing", |_, _, _| FunctionDomain::Full, vectorize_string_to_string_2_arg( - |col, _| col.data.len(), + |col, _| col.data().len(), |val, trim_str, _, output| { let chunk_size = trim_str.len(); let pos = val.rchunks(chunk_size).position(|chunk| chunk != trim_str); @@ -462,7 +462,7 @@ pub fn register(registry: &mut FunctionRegistry) { "trim_both", |_, _, _| FunctionDomain::Full, vectorize_string_to_string_2_arg( - |col, _| col.data.len(), + |col, _| col.data().len(), |val, trim_str, _, output| { let chunk_size = trim_str.len(); let start_pos = val.chunks(chunk_size).position(|chunk| chunk != trim_str); @@ -490,7 +490,7 @@ pub fn register(registry: &mut FunctionRegistry) { "hex", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| col.data.len() * 2, + |col| col.data().len() * 2, |val, output, _| { let old_len = output.data.len(); let extra_len = val.len() * 2; @@ -554,7 +554,7 @@ pub fn register(registry: &mut FunctionRegistry) { "unhex", |_, _| FunctionDomain::MayThrow, vectorize_string_to_string( - |col| col.data.len() / 2, + |col| col.data().len() / 2, |val, output, ctx| { let old_len = output.data.len(); let extra_len = val.len() / 2; @@ -595,7 +595,7 @@ pub fn register(registry: &mut FunctionRegistry) { "soundex", |_, _| FunctionDomain::Full, vectorize_string_to_string( - |col| usize::max(col.data.len(), 4 * col.len()), + |col| usize::max(col.data().len(), 4 * col.len()), soundex::soundex, ), ); diff --git a/src/query/functions/src/scalars/string_multi_args.rs b/src/query/functions/src/scalars/string_multi_args.rs index 35784c9a6c87..847985b45bad 100644 --- a/src/query/functions/src/scalars/string_multi_args.rs +++ b/src/query/functions/src/scalars/string_multi_args.rs @@ -531,10 +531,7 @@ fn char_fn(args: &[ValueRef], _: &mut EvalContext) -> Value { let offsets = (0..(input_rows + 1) as u64 * args.len() as u64) .step_by(args.len()) .collect::>(); - let result = StringColumn { - data: values.into(), - offsets: offsets.into(), - }; + let result = StringColumn::new(values.into(), offsets.into()); Value::Column(Column::String(result)) } diff --git a/src/query/functions/src/scalars/variant.rs b/src/query/functions/src/scalars/variant.rs index 1d8a58cd095d..a701a455429d 100644 --- a/src/query/functions/src/scalars/variant.rs +++ b/src/query/functions/src/scalars/variant.rs @@ -872,6 +872,22 @@ pub fn register(registry: &mut FunctionRegistry) { }); } + registry.register_passthrough_nullable_1_arg::( + "json_to_string", + |_, _| FunctionDomain::Full, + vectorize_with_builder_1_arg::(|val, output, ctx| { + if let Some(validity) = &ctx.validity { + if !validity.get_bit(output.len()) { + output.commit_row(); + return; + } + } + let s = to_string(val); + output.put_slice(s.as_bytes()); + output.commit_row(); + }), + ); + registry.register_function_factory("json_object", |_, args_type| { Some(Arc::new(Function { signature: FunctionSignature { diff --git a/src/query/functions/tests/it/scalars/testdata/function_list.txt b/src/query/functions/tests/it/scalars/testdata/function_list.txt index ab34b8ed0425..b2319863ffbf 100644 --- a/src/query/functions/tests/it/scalars/testdata/function_list.txt +++ b/src/query/functions/tests/it/scalars/testdata/function_list.txt @@ -1765,6 +1765,8 @@ Functions overloads: 1 json_path_query_array(Variant NULL, String NULL) :: Variant NULL 0 json_path_query_first(Variant, String) :: Variant NULL 1 json_path_query_first(Variant NULL, String NULL) :: Variant NULL +0 json_to_string(Variant) :: String +1 json_to_string(Variant NULL) :: String NULL 0 left(String, UInt64) :: String 1 left(String NULL, UInt64 NULL) :: String NULL 0 length(Variant NULL) :: UInt32 NULL diff --git a/src/query/functions/tests/it/scalars/testdata/variant.txt b/src/query/functions/tests/it/scalars/testdata/variant.txt index f1973545c4c4..0aa9507ace0c 100644 --- a/src/query/functions/tests/it/scalars/testdata/variant.txt +++ b/src/query/functions/tests/it/scalars/testdata/variant.txt @@ -2306,3 +2306,48 @@ evaluation (internal): +--------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +ast : json_to_string(parse_json('true')) +raw expr : json_to_string(parse_json('true')) +checked expr : json_to_string(parse_json("true")) +optimized expr : "true" +output type : String +output domain : {"true"..="true"} +output : 'true' + + +ast : json_to_string(parse_json('123456')) +raw expr : json_to_string(parse_json('123456')) +checked expr : json_to_string(parse_json("123456")) +optimized expr : "123456" +output type : String +output domain : {"123456"..="123456"} +output : '123456' + + +ast : json_to_string(parse_json('"abcd"')) +raw expr : json_to_string(parse_json('"abcd"')) +checked expr : json_to_string(parse_json("\"abcd\"")) +optimized expr : "\"abcd\"" +output type : String +output domain : {"\"abcd\""..="\"abcd\""} +output : '"abcd"' + + +ast : json_to_string(parse_json('[1, 2, 3, 4, 5, 6]')) +raw expr : json_to_string(parse_json('[1, 2, 3, 4, 5, 6]')) +checked expr : json_to_string(parse_json("[1, 2, 3, 4, 5, 6]")) +optimized expr : "[1,2,3,4,5,6]" +output type : String +output domain : {"[1,2,3,4,5,6]"..="[1,2,3,4,5,6]"} +output : '[1,2,3,4,5,6]' + + +ast : json_to_string(parse_json('{"k1":123, "k2":"abc"}')) +raw expr : json_to_string(parse_json('{"k1":123, "k2":"abc"}')) +checked expr : json_to_string(parse_json("{\"k1\":123, \"k2\":\"abc\"}")) +optimized expr : "{\"k1\":123,\"k2\":\"abc\"}" +output type : String +output domain : {"{\"k1\":123,\"k2\":\"abc\"}"..="{\"k1\":123,\"k2\":\"abc\"}"} +output : '{"k1":123,"k2":"abc"}' + + diff --git a/src/query/functions/tests/it/scalars/variant.rs b/src/query/functions/tests/it/scalars/variant.rs index 0174bae9e289..7b6774856310 100644 --- a/src/query/functions/tests/it/scalars/variant.rs +++ b/src/query/functions/tests/it/scalars/variant.rs @@ -41,6 +41,7 @@ fn test_variant() { test_json_object_keep_null(file); test_json_path_query_array(file); test_json_path_query_first(file); + test_json_to_string(file); } fn test_parse_json(file: &mut impl Write) { @@ -679,3 +680,16 @@ fn test_json_path_query_first(file: &mut impl Write) { ), ]); } + +fn test_json_to_string(file: &mut impl Write) { + run_ast(file, "json_to_string(parse_json('true'))", &[]); + run_ast(file, "json_to_string(parse_json('123456'))", &[]); + run_ast(file, "json_to_string(parse_json('\"abcd\"'))", &[]); + run_ast(file, "json_to_string(parse_json('[1, 2, 3, 4, 5, 6]'))", &[ + ]); + run_ast( + file, + "json_to_string(parse_json('{\"k1\":123, \"k2\":\"abc\"}'))", + &[], + ); +} diff --git a/src/query/pipeline/sources/src/input_formats/impls/input_format_parquet.rs b/src/query/pipeline/sources/src/input_formats/impls/input_format_parquet.rs index 20de09b5d44e..5e8e5b853a57 100644 --- a/src/query/pipeline/sources/src/input_formats/impls/input_format_parquet.rs +++ b/src/query/pipeline/sources/src/input_formats/impls/input_format_parquet.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use common_arrow::arrow::array::Array; use common_arrow::arrow::chunk::Chunk as ArrowChunk; use common_arrow::arrow::datatypes::Field; -use common_arrow::arrow::io::parquet::read::infer_schema; use common_arrow::arrow::io::parquet::read::read_columns; use common_arrow::arrow::io::parquet::read::read_metadata_async; use common_arrow::arrow::io::parquet::read::to_deserializer; @@ -45,6 +44,7 @@ use common_expression::TableSchemaRef; use common_meta_app::principal::StageInfo; use common_pipeline_core::Pipeline; use common_settings::Settings; +use common_storage::infer_schema_with_extension; use common_storage::read_parquet_metas_in_parallel; use common_storage::StageFileInfo; use futures::AsyncRead; @@ -80,7 +80,7 @@ impl InputFormatParquet { let path = info.path.clone(); let row_groups = mem::take(&mut file_meta.row_groups); if schema.is_none() { - schema = Some(infer_schema(&file_meta)?); + schema = Some(infer_schema_with_extension(&file_meta)?); } let fields = Arc::new(schema.clone().unwrap().fields); let read_file_meta = Arc::new(FileMeta { fields }); @@ -149,7 +149,7 @@ impl InputFormat for InputFormatParquet { async fn infer_schema(&self, path: &str, op: &Operator) -> Result { let mut reader = op.reader(path).await?; let file_meta = read_metadata_async(&mut reader).await?; - let arrow_schema = infer_schema(&file_meta)?; + let arrow_schema = infer_schema_with_extension(&file_meta)?; Ok(Arc::new(TableSchema::from(&arrow_schema))) } @@ -457,7 +457,7 @@ impl AligningStateTrait for ParquetAligningState { ); let mut cursor = Cursor::new(file_in_memory); let file_meta = read_metadata(&mut cursor)?; - let infer_schema = infer_schema(&file_meta)?; + let infer_schema = infer_schema_with_extension(&file_meta)?; let fields = Arc::new(get_used_fields(&infer_schema.fields, &self.ctx.schema)?); let mut row_batches = Vec::with_capacity(file_meta.row_groups.len()); for row_group in file_meta.row_groups.into_iter() { diff --git a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs index 96cf2c22d764..eacc3c7bb256 100644 --- a/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs +++ b/src/query/pipeline/transforms/src/processors/transforms/sort/rows/common.rs @@ -89,7 +89,7 @@ impl RowConverter for CommonRowConverter { let col = c.remove_nullable(); let col = col.as_variant().unwrap(); let mut builder = - StringColumnBuilder::with_capacity(col.len(), col.data.len()); + StringColumnBuilder::with_capacity(col.len(), col.data().len()); for (i, val) in col.iter().enumerate() { if let Some(validity) = validity { if unsafe { !validity.get_bit_unchecked(i) } { diff --git a/src/query/service/src/auth.rs b/src/query/service/src/auth.rs index 6c3003b6bf20..adced16f1526 100644 --- a/src/query/service/src/auth.rs +++ b/src/query/service/src/auth.rs @@ -91,8 +91,12 @@ impl AuthMgr { _ => return Err(ErrorCode::AuthenticateFailure("wrong auth type")), }, Err(e) => { - if e.code() != ErrorCode::UNKNOWN_USER { - return Err(ErrorCode::AuthenticateFailure(e.message())); + match e.code() { + ErrorCode::UNKNOWN_USER => {} + ErrorCode::META_SERVICE_ERROR => { + return Err(e); + } + _ => return Err(ErrorCode::AuthenticateFailure(e.message())), } let ensure_user = jwt .custom diff --git a/src/query/service/src/catalogs/default/mutable_catalog.rs b/src/query/service/src/catalogs/default/mutable_catalog.rs index 15b62da32ee2..3aa0379c8117 100644 --- a/src/query/service/src/catalogs/default/mutable_catalog.rs +++ b/src/query/service/src/catalogs/default/mutable_catalog.rs @@ -55,6 +55,7 @@ use common_meta_app::schema::IndexMeta; use common_meta_app::schema::ListDatabaseReq; use common_meta_app::schema::ListIndexesReq; use common_meta_app::schema::ListTableLockRevReq; +use common_meta_app::schema::ListTableReq; use common_meta_app::schema::ListVirtualColumnsReq; use common_meta_app::schema::RenameDatabaseReply; use common_meta_app::schema::RenameDatabaseReq; @@ -339,8 +340,27 @@ impl Catalog for MutableCatalog { db_name: &str, filter: Option, ) -> Result>> { - let db = self.get_database(tenant, db_name).await?; - db.list_tables_history(filter).await + if db_name.is_empty() { + let ctx = DatabaseContext { + meta: self.ctx.meta.clone(), + storage_factory: self.ctx.storage_factory.clone(), + tenant: self.tenant.clone(), + }; + let table_infos = ctx + .meta + .get_table_history(ListTableReq::new_with_filter(tenant, db_name, filter)) + .await?; + + let storage = ctx.storage_factory.clone(); + table_infos.iter().try_fold(vec![], |mut acc, item| { + let tbl = storage.get_table(item.as_ref())?; + acc.push(tbl); + Ok(acc) + }) + } else { + let db = self.get_database(tenant, db_name).await?; + db.list_tables_history(filter).await + } } #[async_backtrace::framed] diff --git a/src/query/service/src/databases/system/system_database.rs b/src/query/service/src/databases/system/system_database.rs index c3119e6855d5..758c90bdd8c0 100644 --- a/src/query/service/src/databases/system/system_database.rs +++ b/src/query/service/src/databases/system/system_database.rs @@ -22,6 +22,7 @@ use common_meta_app::schema::DatabaseMeta; use common_meta_app::schema::DatabaseNameIdent; use common_storages_system::BackgroundJobTable; use common_storages_system::BackgroundTaskTable; +use common_storages_system::BacktraceTable; use common_storages_system::BuildOptionsTable; use common_storages_system::CachesTable; use common_storages_system::CatalogsTable; @@ -109,6 +110,7 @@ impl SystemDatabase { QueryProfileTable::create(sys_db_meta.next_table_id()), BackgroundTaskTable::create(sys_db_meta.next_table_id()), BackgroundJobTable::create(sys_db_meta.next_table_id()), + BacktraceTable::create(sys_db_meta.next_table_id()), ]; let disable_tables = Self::disable_system_tables(); diff --git a/src/query/service/src/interpreters/access/privilege_access.rs b/src/query/service/src/interpreters/access/privilege_access.rs index 3f000b349c6a..b1c8c2aaa512 100644 --- a/src/query/service/src/interpreters/access/privilege_access.rs +++ b/src/query/service/src/interpreters/access/privilege_access.rs @@ -364,6 +364,14 @@ impl AccessChecker for PrivilegeAccess { ) .await?; } + Plan::VacuumDropTable(plan) => { + session + .validate_privilege( + &GrantObject::Database(plan.catalog.clone(), plan.database.clone()), + vec![UserPrivilegeType::Super], + ) + .await?; + } Plan::AnalyzeTable(plan) => { session .validate_privilege( diff --git a/src/query/service/src/interpreters/common/mod.rs b/src/query/service/src/interpreters/common/mod.rs index ca177655c221..d752fcb60cfe 100644 --- a/src/query/service/src/interpreters/common/mod.rs +++ b/src/query/service/src/interpreters/common/mod.rs @@ -13,13 +13,9 @@ // limitations under the License. mod grant; -mod stage; mod table; mod util; + pub use grant::validate_grant_object_exists; -pub use stage::try_purge_files; -pub use table::append2table; -pub use table::append2table_without_commit; pub use table::check_referenced_computed_columns; -pub use table::fill_missing_columns; pub use util::check_deduplicate_label; diff --git a/src/query/service/src/interpreters/common/table.rs b/src/query/service/src/interpreters/common/table.rs index b8ee4a948732..b431e583c1dd 100644 --- a/src/query/service/src/interpreters/common/table.rs +++ b/src/query/service/src/interpreters/common/table.rs @@ -14,96 +14,14 @@ use std::sync::Arc; -use common_catalog::table::AppendMode; -use common_catalog::table::Table; use common_catalog::table_context::TableContext; use common_exception::ErrorCode; use common_exception::Result; use common_expression::ComputedExpr; -use common_expression::DataSchemaRef; use common_expression::DataSchemaRefExt; use common_expression::TableSchemaRef; -use common_meta_app::schema::UpsertTableCopiedFileReq; -use common_pipeline_core::Pipeline; use common_sql::parse_computed_expr; -use crate::pipelines::processors::transforms::TransformAddComputedColumns; -use crate::pipelines::processors::TransformResortAddOn; -use crate::sessions::QueryContext; - -pub fn fill_missing_columns( - ctx: Arc, - table: Arc, - source_schema: DataSchemaRef, - pipeline: &mut Pipeline, -) -> Result<()> { - let table_default_schema = &table.schema().remove_computed_fields(); - let table_computed_schema = &table.schema().remove_virtual_computed_fields(); - let default_schema: DataSchemaRef = Arc::new(table_default_schema.into()); - let computed_schema: DataSchemaRef = Arc::new(table_computed_schema.into()); - - // Fill missing default columns and resort the columns. - if source_schema != default_schema { - pipeline.add_transform(|transform_input_port, transform_output_port| { - TransformResortAddOn::try_create( - ctx.clone(), - transform_input_port, - transform_output_port, - source_schema.clone(), - default_schema.clone(), - table.clone(), - ) - })?; - } - - // Fill computed columns. - if default_schema != computed_schema { - pipeline.add_transform(|transform_input_port, transform_output_port| { - TransformAddComputedColumns::try_create( - ctx.clone(), - transform_input_port, - transform_output_port, - default_schema.clone(), - computed_schema.clone(), - ) - })?; - } - - Ok(()) -} - -pub fn append2table( - ctx: Arc, - table: Arc, - source_schema: DataSchemaRef, - main_pipeline: &mut Pipeline, - copied_files: Option, - overwrite: bool, - append_mode: AppendMode, -) -> Result<()> { - fill_missing_columns(ctx.clone(), table.clone(), source_schema, main_pipeline)?; - - table.append_data(ctx.clone(), main_pipeline, append_mode)?; - - table.commit_insertion(ctx, main_pipeline, copied_files, overwrite)?; - - Ok(()) -} - -pub fn append2table_without_commit( - ctx: Arc, - table: Arc, - source_schema: DataSchemaRef, - main_pipeline: &mut Pipeline, - append_mode: AppendMode, -) -> Result<()> { - fill_missing_columns(ctx.clone(), table.clone(), source_schema, main_pipeline)?; - - table.append_data(ctx, main_pipeline, append_mode)?; - - Ok(()) -} - pub fn check_referenced_computed_columns( ctx: Arc, table_schema: TableSchemaRef, diff --git a/src/query/service/src/interpreters/interpreter_copy.rs b/src/query/service/src/interpreters/interpreter_copy.rs index 8e8aba716843..8f3a49d06f71 100644 --- a/src/query/service/src/interpreters/interpreter_copy.rs +++ b/src/query/service/src/interpreters/interpreter_copy.rs @@ -12,27 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; use std::sync::Arc; use std::time::Instant; -use chrono::Utc; -use common_base::runtime::GlobalIORuntime; use common_catalog::plan::StageTableInfo; use common_catalog::table::AppendMode; -use common_catalog::table::Table; use common_exception::ErrorCode; use common_exception::Result; use common_expression::infer_table_schema; use common_expression::BlockThresholds; use common_expression::DataField; -use common_expression::DataSchema; use common_expression::DataSchemaRef; use common_expression::DataSchemaRefExt; -use common_expression::Scalar; use common_meta_app::principal::StageInfo; -use common_meta_app::schema::TableCopiedFileInfo; -use common_meta_app::schema::UpsertTableCopiedFileReq; use common_pipeline_core::Pipeline; use common_sql::executor::table_read_plan::ToReadDataSourcePlan; use common_sql::executor::DistributedCopyIntoTable; @@ -43,19 +35,17 @@ use common_sql::plans::CopyIntoTableMode; use common_sql::plans::CopyIntoTablePlan; use common_storage::StageFileInfo; use common_storage::StageFilesInfo; -use common_storages_fuse::io::Files; use common_storages_stage::StageTable; -use tracing::debug; -use tracing::error; use tracing::info; -use crate::interpreters::common::append2table; -use crate::interpreters::common::append2table_without_commit; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::Interpreter; use crate::interpreters::SelectInterpreter; -use crate::pipelines::processors::transforms::TransformAddConstColumns; -use crate::pipelines::processors::TransformCastSchema; +use crate::pipelines::builders::build_append2table_pipeline; +use crate::pipelines::builders::build_append_data_with_finish_pipeline; +use crate::pipelines::builders::build_upsert_copied_files_to_meta_req; +use crate::pipelines::builders::try_purge_files; +use crate::pipelines::builders::CopyPlanParam; use crate::pipelines::PipelineBuildResult; use crate::schedulers::build_distributed_pipeline; use crate::sessions::QueryContext; @@ -68,11 +58,6 @@ pub struct CopyInterpreter { plan: CopyPlan, } -pub enum PlanParam { - CopyIntoTablePlanOption(CopyIntoTablePlan), - DistributedCopyIntoTable(DistributedCopyIntoTable), -} - impl CopyInterpreter { /// Create a CopyInterpreter with context and [`CopyPlan`]. pub fn try_create(ctx: Arc, plan: CopyPlan) -> Result { @@ -119,8 +104,9 @@ impl CopyInterpreter { Ok((build_res, data_schema)) } + /// Build a pipeline for local copy into stage. #[async_backtrace::framed] - async fn build_copy_into_stage_pipeline( + async fn build_local_copy_into_stage_pipeline( &self, stage: &StageInfo, path: &str, @@ -140,11 +126,11 @@ impl CopyInterpreter { is_select: false, }; let table = StageTable::try_create(stage_table_info)?; - append2table( + build_append2table_pipeline( self.ctx.clone(), + &mut build_res.main_pipeline, table, data_schema, - &mut build_res.main_pipeline, None, false, AppendMode::Normal, @@ -168,23 +154,8 @@ impl CopyInterpreter { info!(status); } - let table_ctx: Arc = ctx.clone(); - let op = StageTable::get_op(stage_info); - match op { - Ok(op) => { - let file_op = Files::create(table_ctx, op); - let files = stage_file_infos - .iter() - .map(|v| v.path.clone()) - .collect::>(); - if let Err(e) = file_op.remove_file_in_batch(&files).await { - error!("Failed to delete file: {:?}, error: {}", files, e); - } - } - Err(e) => { - error!("Failed to get stage table op, error: {}", e); - } - } + try_purge_files(ctx.clone(), stage_info, stage_file_infos).await; + // Status. info!( "end to purge files:{}, elapsed:{}", @@ -199,7 +170,7 @@ impl CopyInterpreter { } #[async_backtrace::framed] - async fn transform_copy_plan_distributed( + async fn try_transform_copy_plan_from_local_to_distributed( &self, plan: &CopyIntoTablePlan, ) -> Result> { @@ -253,7 +224,7 @@ impl CopyInterpreter { } #[async_backtrace::framed] - async fn build_read_stage( + async fn build_read_stage_table_data_pipeline( &self, pipeline: &mut Pipeline, plan: &CopyIntoTablePlan, @@ -290,8 +261,9 @@ impl CopyInterpreter { Ok(()) } + /// Build a pipeline to copy data from local. #[async_backtrace::framed] - async fn build_copy_into_table_pipeline( + async fn build_local_copy_into_table_pipeline( &self, plan: &CopyIntoTablePlan, ) -> Result { @@ -318,7 +290,7 @@ impl CopyInterpreter { if files.is_empty() { return Ok(build_res); } - self.build_read_stage( + self.build_read_stage_table_data_pipeline( &mut build_res.main_pipeline, plan, to_table.get_block_thresholds(), @@ -328,11 +300,11 @@ impl CopyInterpreter { (build_res, plan.required_source_schema.clone(), files) }; - append_data_and_set_finish( + build_append_data_with_finish_pipeline( + ctx, &mut build_res.main_pipeline, source_schema, - PlanParam::CopyIntoTablePlanOption(plan.clone()), - ctx, + CopyPlanParam::CopyIntoTablePlanOption(plan.clone()), to_table, files, start, @@ -341,53 +313,46 @@ impl CopyInterpreter { Ok(build_res) } - fn upsert_copied_files_request( - ctx: Arc, - to_table: Arc, - stage_info: StageInfo, - copied_files: Vec, - force: bool, - ) -> Result> { - let mut copied_file_tree = BTreeMap::new(); - for file in &copied_files { - // Short the etag to 7 bytes for less space in metasrv. - let short_etag = file.etag.clone().map(|mut v| { - v.truncate(7); - v - }); - copied_file_tree.insert(file.path.clone(), TableCopiedFileInfo { - etag: short_etag, - content_length: file.size, - last_modified: Some(file.last_modified), - }); - } - - let expire_hours = ctx.get_settings().get_load_file_metadata_expire_hours()?; + /// Build a pipeline to copy data into table for distributed. + #[async_backtrace::framed] + async fn build_distributed_copy_into_table_pipeline( + &self, + distributed_plan: &DistributedCopyIntoTable, + ) -> Result { + // add exchange plan node to enable distributed + // TODO(leiysky): we reuse the id of exchange here, + // which is not correct. We should generate a new id for insert. + let exchange_plan = PhysicalPlan::Exchange(Exchange { + plan_id: 0, + input: Box::new(PhysicalPlan::DistributedCopyIntoTable(Box::new( + distributed_plan.clone(), + ))), + kind: FragmentKind::Merge, + keys: Vec::new(), + }); + let mut build_res = build_distributed_pipeline(&self.ctx, &exchange_plan, false).await?; - let upsert_copied_files_request = { - if stage_info.copy_options.purge && force { - // if `purge-after-copy` is enabled, and in `force` copy mode, - // we do not need to upsert copied files into meta server - info!( - "[purge] and [force] are both enabled, will not update copied-files set. ({})", - &to_table.get_table_info().desc - ); - None - } else if copied_file_tree.is_empty() { - None - } else { - tracing::debug!("upsert_copied_files_info: {:?}", copied_file_tree); - let expire_at = expire_hours * 60 * 60 + Utc::now().timestamp() as u64; - let req = UpsertTableCopiedFileReq { - file_info: copied_file_tree, - expire_at: Some(expire_at), - fail_if_duplicated: !force, - }; - Some(req) - } - }; + let catalog = self.ctx.get_catalog(&distributed_plan.catalog_name)?; + let to_table = catalog.get_table_by_info(&distributed_plan.table_info)?; + let copied_files = build_upsert_copied_files_to_meta_req( + self.ctx.clone(), + to_table.clone(), + distributed_plan.stage_table_info.stage_info.clone(), + distributed_plan.files.clone(), + distributed_plan.force, + )?; + let mut overwrite_ = false; + if let CopyIntoTableMode::Insert { overwrite } = distributed_plan.write_mode { + overwrite_ = overwrite; + } + to_table.commit_insertion( + self.ctx.clone(), + &mut build_res.main_pipeline, + copied_files, + overwrite_, + )?; - Ok(upsert_copied_files_request) + Ok(build_res) } } @@ -407,265 +372,26 @@ impl Interpreter for CopyInterpreter { match &self.plan { CopyPlan::IntoTable(plan) => { if plan.enable_distributed { - let distributed_plan_op = self.transform_copy_plan_distributed(plan).await?; - if distributed_plan_op.is_none() { - return self.build_copy_into_table_pipeline(plan).await; + let distributed_plan_op = self + .try_transform_copy_plan_from_local_to_distributed(plan) + .await?; + if let Some(distributed_plan) = distributed_plan_op { + self.build_distributed_copy_into_table_pipeline(&distributed_plan) + .await + } else { + self.build_local_copy_into_table_pipeline(plan).await } - let distributed_plan = distributed_plan_op.unwrap(); - - // add exchange plan node to enable distributed - // TODO(leiysky): we reuse the id of exchange here, - // which is not correct. We should generate a new id for insert. - let exchange_plan = PhysicalPlan::Exchange(Exchange { - plan_id: 0, - input: Box::new(PhysicalPlan::DistributedCopyIntoTable(Box::new( - distributed_plan.clone(), - ))), - kind: FragmentKind::Merge, - keys: Vec::new(), - }); - let mut build_res = - build_distributed_pipeline(&self.ctx, &exchange_plan, false).await?; - - let catalog = self.ctx.get_catalog(&distributed_plan.catalog_name)?; - let to_table = catalog.get_table_by_info(&distributed_plan.table_info)?; - let copied_files = CopyInterpreter::upsert_copied_files_request( - self.ctx.clone(), - to_table.clone(), - distributed_plan.stage_table_info.stage_info.clone(), - distributed_plan.files.clone(), - distributed_plan.force, - )?; - let mut overwrite_ = false; - if let CopyIntoTableMode::Insert { overwrite } = plan.write_mode { - overwrite_ = overwrite; - } - to_table.commit_insertion( - self.ctx.clone(), - &mut build_res.main_pipeline, - copied_files, - overwrite_, - )?; - - Ok(build_res) } else { - self.build_copy_into_table_pipeline(plan).await + self.build_local_copy_into_table_pipeline(plan).await } } CopyPlan::IntoStage { stage, from, path, .. - } => self.build_copy_into_stage_pipeline(stage, path, from).await, - CopyPlan::NoFileToCopy => Ok(PipelineBuildResult::create()), - } - } -} - -fn fill_const_columns( - ctx: Arc, - pipeline: &mut Pipeline, - input_schema: DataSchemaRef, - output_schema: DataSchemaRef, - const_values: Vec, -) -> Result<()> { - pipeline.add_transform(|transform_input_port, transform_output_port| { - TransformAddConstColumns::try_create( - ctx.clone(), - transform_input_port, - transform_output_port, - input_schema.clone(), - output_schema.clone(), - const_values.clone(), - ) - })?; - Ok(()) -} - -#[allow(clippy::too_many_arguments)] -pub fn append_data_and_set_finish( - main_pipeline: &mut Pipeline, - source_schema: Arc, - plan_option: PlanParam, - ctx: Arc, - to_table: Arc, - files: Vec, - start: Instant, - use_commit: bool, -) -> Result<()> { - let plan_required_source_schema: DataSchemaRef; - let plan_required_values_schema: DataSchemaRef; - let plan_values_consts: Vec; - let plan_stage_table_info: StageTableInfo; - let plan_force: bool; - let plan_write_mode: CopyIntoTableMode; - let local_id; - match plan_option { - PlanParam::CopyIntoTablePlanOption(plan) => { - plan_required_source_schema = plan.required_source_schema; - plan_required_values_schema = plan.required_values_schema; - plan_values_consts = plan.values_consts; - plan_stage_table_info = plan.stage_table_info; - plan_force = plan.force; - plan_write_mode = plan.write_mode; - local_id = ctx.get_cluster().local_id.clone(); - } - PlanParam::DistributedCopyIntoTable(plan) => { - plan_required_source_schema = plan.required_source_schema; - plan_required_values_schema = plan.required_values_schema; - plan_values_consts = plan.values_consts; - plan_stage_table_info = plan.stage_table_info; - plan_force = plan.force; - plan_write_mode = plan.write_mode; - local_id = plan.local_node_id; - } - } - - debug!("source schema:{:?}", source_schema); - debug!("required source schema:{:?}", plan_required_source_schema); - debug!("required values schema:{:?}", plan_required_values_schema); - - if source_schema != plan_required_source_schema { - // only parquet need cast - let func_ctx = ctx.get_function_context()?; - main_pipeline.add_transform(|transform_input_port, transform_output_port| { - TransformCastSchema::try_create( - transform_input_port, - transform_output_port, - source_schema.clone(), - plan_required_source_schema.clone(), - func_ctx.clone(), - ) - })?; - } - - if !plan_values_consts.is_empty() { - fill_const_columns( - ctx.clone(), - main_pipeline, - source_schema, - plan_required_values_schema.clone(), - plan_values_consts, - )?; - } - - let stage_info_clone = plan_stage_table_info.stage_info; - let write_mode = plan_write_mode; - let mut purge = true; - match write_mode { - CopyIntoTableMode::Insert { overwrite } => { - if use_commit { - append2table( - ctx.clone(), - to_table, - plan_required_values_schema, - main_pipeline, - None, - overwrite, - AppendMode::Copy, - )?; - } else { - append2table_without_commit( - ctx.clone(), - to_table, - plan_required_values_schema, - main_pipeline, - AppendMode::Copy, - )? - } - } - CopyIntoTableMode::Copy => { - if !stage_info_clone.copy_options.purge { - purge = false; - } - - if use_commit { - let copied_files = CopyInterpreter::upsert_copied_files_request( - ctx.clone(), - to_table.clone(), - stage_info_clone.clone(), - files.clone(), - plan_force, - )?; - append2table( - ctx.clone(), - to_table, - plan_required_values_schema, - main_pipeline, - copied_files, - false, - AppendMode::Copy, - )?; - } else { - append2table_without_commit( - ctx.clone(), - to_table, - plan_required_values_schema, - main_pipeline, - AppendMode::Copy, - )? + } => { + self.build_local_copy_into_stage_pipeline(stage, path, from) + .await } + CopyPlan::NoFileToCopy => Ok(PipelineBuildResult::create()), } - CopyIntoTableMode::Replace => {} } - if local_id == ctx.get_cluster().local_id { - main_pipeline.set_on_finished(move |may_error| { - match may_error { - None => { - error!("main node recieve success"); - GlobalIORuntime::instance().block_on(async move { - { - let status = - format!("end of commit, number of copied files:{}", files.len()); - ctx.set_status_info(&status); - info!(status); - } - - // 1. log on_error mode errors. - // todo(ariesdevil): persist errors with query_id - if let Some(error_map) = ctx.get_maximum_error_per_file() { - for (file_name, e) in error_map { - error!( - "copy(on_error={}): file {} encounter error {},", - stage_info_clone.copy_options.on_error, - file_name, - e.to_string() - ); - } - } - - // 2. Try to purge copied files if purge option is true, if error will skip. - // If a file is already copied(status with AlreadyCopied) we will try to purge them. - if purge { - CopyInterpreter::try_purge_files( - ctx.clone(), - &stage_info_clone, - &files, - ) - .await; - } - - // Status. - { - info!("all copy finished, elapsed:{}", start.elapsed().as_secs()); - } - - Ok(()) - })?; - } - Some(error) => { - error!("main node recieve error"); - error!( - "copy failed, elapsed:{}, reason: {}", - start.elapsed().as_secs(), - error - ); - } - } - Ok(()) - }); - } else { - // remote node does nothing. - main_pipeline.set_on_finished(move |_| Ok(())) - } - - Ok(()) } diff --git a/src/query/service/src/interpreters/interpreter_factory.rs b/src/query/service/src/interpreters/interpreter_factory.rs index ac87d5b29ea7..0218e6233f82 100644 --- a/src/query/service/src/interpreters/interpreter_factory.rs +++ b/src/query/service/src/interpreters/interpreter_factory.rs @@ -208,6 +208,9 @@ impl InterpreterFactory { ctx, *vacuum_table.clone(), )?)), + Plan::VacuumDropTable(vacuum_drop_table) => Ok(Arc::new( + VacuumDropTablesInterpreter::try_create(ctx, *vacuum_drop_table.clone())?, + )), Plan::AnalyzeTable(analyze_table) => Ok(Arc::new(AnalyzeTableInterpreter::try_create( ctx, *analyze_table.clone(), diff --git a/src/query/service/src/interpreters/interpreter_index_refresh.rs b/src/query/service/src/interpreters/interpreter_index_refresh.rs index f63826550326..07868252f0b6 100644 --- a/src/query/service/src/interpreters/interpreter_index_refresh.rs +++ b/src/query/service/src/interpreters/interpreter_index_refresh.rs @@ -235,10 +235,9 @@ impl Interpreter for RefreshIndexInterpreter { .await?; if new_read_source.is_none() { - return Err(ErrorCode::IndexAlreadyRefreshed(format!( - "Aggregating Index {} already refreshed", - self.plan.index_name.clone() - ))); + // The partitions are all pruned, we don't need to generate indexes for these partitions (blocks). + let empty_pipeline = PipelineBuildResult::create(); + return Ok(empty_pipeline); } let new_read_source = new_read_source.unwrap(); diff --git a/src/query/service/src/interpreters/interpreter_insert.rs b/src/query/service/src/interpreters/interpreter_insert.rs index 685e29c7ca9e..fd0f7dd86ae4 100644 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ b/src/query/service/src/interpreters/interpreter_insert.rs @@ -49,10 +49,10 @@ use once_cell::sync::Lazy; use parking_lot::Mutex; use parking_lot::RwLock; -use crate::interpreters::common::append2table; use crate::interpreters::common::check_deduplicate_label; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::builders::build_append2table_pipeline; use crate::pipelines::processors::transforms::TransformRuntimeCastSchema; use crate::pipelines::PipelineBuildResult; use crate::pipelines::SourcePipeBuilder; @@ -270,11 +270,11 @@ impl Interpreter for InsertInterpreter { _ => AppendMode::Normal, }; - append2table( + build_append2table_pipeline( self.ctx.clone(), + &mut build_res.main_pipeline, table.clone(), plan.schema(), - &mut build_res.main_pipeline, None, self.plan.overwrite, append_mode, diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 5023f8fc76de..9bd250e5aa81 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -26,12 +26,12 @@ use common_sql::plans::Replace; use common_sql::NameResolutionContext; use crate::interpreters::common::check_deduplicate_label; -use crate::interpreters::fill_missing_columns; use crate::interpreters::interpreter_copy::CopyInterpreter; use crate::interpreters::interpreter_insert::ValueSource; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; +use crate::pipelines::builders::build_fill_missing_columns_pipeline; use crate::pipelines::processors::TransformCastSchema; use crate::pipelines::PipelineBuildResult; use crate::sessions::QueryContext; @@ -76,11 +76,11 @@ impl Interpreter for ReplaceInterpreter { return Ok(pipeline); } - fill_missing_columns( + build_fill_missing_columns_pipeline( self.ctx.clone(), + &mut pipeline.main_pipeline, table.clone(), self.plan.schema(), - &mut pipeline.main_pipeline, )?; let on_conflict_fields = plan.on_conflict_fields.clone(); diff --git a/src/query/service/src/interpreters/interpreter_table_create.rs b/src/query/service/src/interpreters/interpreter_table_create.rs index 0caa9bfbd043..bfae34be847f 100644 --- a/src/query/service/src/interpreters/interpreter_table_create.rs +++ b/src/query/service/src/interpreters/interpreter_table_create.rs @@ -30,12 +30,14 @@ use common_meta_types::MatchSeq; use common_sql::binder::INTERNAL_COLUMN_FACTORY; use common_sql::field_default_value; use common_sql::plans::CreateTablePlan; +use common_storage::DataOperator; use common_storages_fuse::io::MetaReaders; use common_storages_fuse::FUSE_OPT_KEY_BLOCK_IN_MEM_SIZE_THRESHOLD; use common_storages_fuse::FUSE_OPT_KEY_BLOCK_PER_SEGMENT; use common_storages_fuse::FUSE_OPT_KEY_ROW_AVG_DEPTH_THRESHOLD; use common_storages_fuse::FUSE_OPT_KEY_ROW_PER_BLOCK; use common_storages_fuse::FUSE_OPT_KEY_ROW_PER_PAGE; +use common_storages_fuse::FUSE_TBL_LAST_SNAPSHOT_HINT; use common_users::UserApiProvider; use once_cell::sync::Lazy; use storages_common_cache::LoadParams; @@ -46,6 +48,7 @@ use storages_common_table_meta::table::OPT_KEY_DATABASE_ID; use storages_common_table_meta::table::OPT_KEY_ENGINE; use storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; use storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; +use storages_common_table_meta::table::OPT_KEY_STORAGE_PREFIX; use storages_common_table_meta::table::OPT_KEY_TABLE_COMPRESSION; use tracing::error; @@ -204,7 +207,12 @@ impl CreateTableInterpreter { }); } } - catalog.create_table(self.build_request(stat)?).await?; + let req = if let Some(storage_prefix) = self.plan.options.get(OPT_KEY_STORAGE_PREFIX) { + self.build_attach_request(storage_prefix).await + } else { + self.build_request(stat) + }?; + catalog.create_table(req).await?; Ok(PipelineBuildResult::create()) } @@ -273,6 +281,62 @@ impl CreateTableInterpreter { Ok(req) } + + async fn build_attach_request(&self, storage_prefix: &str) -> Result { + // Safe to unwrap in this function, as attach table must have storage params. + let sp = self.plan.storage_params.as_ref().unwrap(); + let operator = DataOperator::try_create(sp).await?; + let operator = operator.operator(); + let reader = MetaReaders::table_snapshot_reader(operator.clone()); + let hint = format!("{}/{}", storage_prefix, FUSE_TBL_LAST_SNAPSHOT_HINT); + let snapshot_loc = operator.read(&hint).await?; + let snapshot_loc = String::from_utf8(snapshot_loc)?; + let info = operator.info(); + let root = info.root(); + let snapshot_loc = snapshot_loc[root.len()..].to_string(); + let mut options = self.plan.options.clone(); + options.insert(OPT_KEY_SNAPSHOT_LOCATION.to_string(), snapshot_loc.clone()); + + let params = LoadParams { + location: snapshot_loc.clone(), + len_hint: None, + ver: TableSnapshot::VERSION, + put_cache: true, + }; + + let snapshot = reader.read(¶ms).await?; + let stat = TableStatistics { + number_of_rows: snapshot.summary.row_count, + data_bytes: snapshot.summary.uncompressed_byte_size, + compressed_data_bytes: snapshot.summary.compressed_byte_size, + index_data_bytes: snapshot.summary.index_size, + number_of_segments: Some(snapshot.segments.len() as u64), + number_of_blocks: Some(snapshot.summary.block_count), + }; + let table_meta = TableMeta { + schema: Arc::new(snapshot.schema.clone()), + engine: self.plan.engine.to_string(), + storage_params: self.plan.storage_params.clone(), + part_prefix: self.plan.part_prefix.clone(), + options, + default_cluster_key: None, + field_comments: self.plan.field_comments.clone(), + drop_on: None, + statistics: stat, + ..Default::default() + }; + let req = CreateTableReq { + if_not_exists: self.plan.if_not_exists, + name_ident: TableNameIdent { + tenant: self.plan.tenant.to_string(), + db_name: self.plan.database.to_string(), + table_name: self.plan.table.to_string(), + }, + table_meta, + }; + + Ok(req) + } } /// Table option keys that can occur in 'create table statement'. diff --git a/src/query/service/src/interpreters/interpreter_table_drop_column.rs b/src/query/service/src/interpreters/interpreter_table_drop_column.rs index 179a174875eb..0b75fb86f5a3 100644 --- a/src/query/service/src/interpreters/interpreter_table_drop_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_drop_column.rs @@ -23,7 +23,7 @@ use common_sql::plans::DropTableColumnPlan; use common_storages_share::save_share_table_info; use common_storages_view::view_table::VIEW_ENGINE; -use crate::interpreters::check_referenced_computed_columns; +use crate::interpreters::common::check_referenced_computed_columns; use crate::interpreters::Interpreter; use crate::pipelines::PipelineBuildResult; use crate::sessions::QueryContext; diff --git a/src/query/service/src/interpreters/interpreter_table_rename_column.rs b/src/query/service/src/interpreters/interpreter_table_rename_column.rs index 5c885a546ca9..2a7759d603b3 100644 --- a/src/query/service/src/interpreters/interpreter_table_rename_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_rename_column.rs @@ -24,7 +24,7 @@ use common_sql::plans::RenameTableColumnPlan; use common_storages_share::save_share_table_info; use common_storages_view::view_table::VIEW_ENGINE; -use crate::interpreters::check_referenced_computed_columns; +use crate::interpreters::common::check_referenced_computed_columns; use crate::interpreters::Interpreter; use crate::pipelines::PipelineBuildResult; use crate::sessions::QueryContext; diff --git a/src/query/service/src/interpreters/interpreter_vacuum_drop_tables.rs b/src/query/service/src/interpreters/interpreter_vacuum_drop_tables.rs new file mode 100644 index 000000000000..0ee989d1c33b --- /dev/null +++ b/src/query/service/src/interpreters/interpreter_vacuum_drop_tables.rs @@ -0,0 +1,117 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::cmp::min; +use std::sync::Arc; + +use common_exception::Result; +use common_expression::types::StringType; +use common_expression::DataBlock; +use common_expression::DataSchemaRef; +use common_expression::FromData; +use common_license::license::Feature::Vacuum; +use common_license::license_manager::get_license_manager; +use common_meta_app::schema::TableInfoFilter; +use common_sql::plans::VacuumDropTablePlan; +use vacuum_handler::get_vacuum_handler; + +use crate::interpreters::Interpreter; +use crate::pipelines::PipelineBuildResult; +use crate::sessions::QueryContext; +use crate::sessions::TableContext; + +const DRY_RUN_LIMIT: usize = 1000; + +#[allow(dead_code)] +pub struct VacuumDropTablesInterpreter { + ctx: Arc, + plan: VacuumDropTablePlan, +} + +impl VacuumDropTablesInterpreter { + pub fn try_create(ctx: Arc, plan: VacuumDropTablePlan) -> Result { + Ok(VacuumDropTablesInterpreter { ctx, plan }) + } +} + +#[async_trait::async_trait] +impl Interpreter for VacuumDropTablesInterpreter { + fn name(&self) -> &str { + "VacuumDropTablesInterpreter" + } + + fn schema(&self) -> DataSchemaRef { + self.plan.schema() + } + + #[async_backtrace::framed] + async fn execute2(&self) -> Result { + let license_manager = get_license_manager(); + license_manager.manager.check_enterprise_enabled( + &self.ctx.get_settings(), + self.ctx.get_tenant(), + Vacuum, + )?; + + let ctx = self.ctx.clone(); + let hours = match self.plan.option.retain_hours { + Some(hours) => hours as i64, + None => ctx.get_settings().get_retention_period()? as i64, + }; + let retention_time = chrono::Utc::now() - chrono::Duration::hours(hours); + let catalog = self.ctx.get_catalog(self.plan.catalog.as_str())?; + // if database if empty, vacuum all tables + let filter = if self.plan.database.is_empty() { + Some(TableInfoFilter::AllDroppedTables(Some(retention_time))) + } else { + Some(TableInfoFilter::Dropped(Some(retention_time))) + }; + + let tenant = self.ctx.get_tenant(); + let tables = catalog + .list_tables_history(&tenant, &self.plan.database, filter) + .await?; + + let handler = get_vacuum_handler(); + let files_opt = handler + .do_vacuum_drop_tables( + tables, + if self.plan.option.dry_run.is_some() { + Some(DRY_RUN_LIMIT) + } else { + None + }, + ) + .await?; + + match files_opt { + None => return Ok(PipelineBuildResult::create()), + Some(purge_files) => { + let len = min(purge_files.len(), DRY_RUN_LIMIT); + let mut tables: Vec> = Vec::with_capacity(len); + let mut files: Vec> = Vec::with_capacity(len); + let purge_files = &purge_files[0..len]; + for file in purge_files.iter() { + tables.push(file.0.to_string().as_bytes().to_vec()); + files.push(file.1.to_string().as_bytes().to_vec()); + } + + PipelineBuildResult::from_blocks(vec![DataBlock::new_from_columns(vec![ + StringType::from_data(tables), + StringType::from_data(files), + ])]) + } + } + } +} diff --git a/src/query/service/src/interpreters/mod.rs b/src/query/service/src/interpreters/mod.rs index a458c072ed34..f4cd18b7aca9 100644 --- a/src/query/service/src/interpreters/mod.rs +++ b/src/query/service/src/interpreters/mod.rs @@ -98,6 +98,7 @@ mod interpreter_user_stage_remove; mod interpreter_user_udf_alter; mod interpreter_user_udf_create; mod interpreter_user_udf_drop; +mod interpreter_vacuum_drop_tables; mod interpreter_view_alter; mod interpreter_view_create; mod interpreter_view_drop; @@ -107,18 +108,12 @@ mod interpreter_virtual_columns_drop; mod interpreter_virtual_columns_generate; pub use access::ManagementModeAccess; -pub use common::append2table; -pub use common::append2table_without_commit; -pub use common::check_referenced_computed_columns; -pub use common::fill_missing_columns; pub use interpreter::Interpreter; pub use interpreter::InterpreterPtr; pub use interpreter_call::CallInterpreter; pub use interpreter_cluster_key_alter::AlterTableClusterKeyInterpreter; pub use interpreter_cluster_key_drop::DropTableClusterKeyInterpreter; pub use interpreter_clustering_history::InterpreterClusteringHistory; -pub use interpreter_copy::append_data_and_set_finish; -pub use interpreter_copy::PlanParam; pub use interpreter_data_mask_create::CreateDataMaskInterpreter; pub use interpreter_data_mask_desc::DescDataMaskInterpreter; pub use interpreter_data_mask_drop::DropDataMaskInterpreter; @@ -185,6 +180,7 @@ pub use interpreter_user_stage_remove::RemoveUserStageInterpreter; pub use interpreter_user_udf_alter::AlterUserUDFInterpreter; pub use interpreter_user_udf_create::CreateUserUDFInterpreter; pub use interpreter_user_udf_drop::DropUserUDFInterpreter; +pub use interpreter_vacuum_drop_tables::VacuumDropTablesInterpreter; pub use interpreter_view_alter::AlterViewInterpreter; pub use interpreter_view_create::CreateViewInterpreter; pub use interpreter_view_drop::DropViewInterpreter; diff --git a/src/query/service/src/pipelines/builders/copy.rs b/src/query/service/src/pipelines/builders/copy.rs new file mode 100644 index 000000000000..41e2e34a2983 --- /dev/null +++ b/src/query/service/src/pipelines/builders/copy.rs @@ -0,0 +1,304 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::BTreeMap; +use std::sync::Arc; +use std::time::Instant; + +use chrono::Utc; +use common_base::runtime::GlobalIORuntime; +use common_catalog::plan::StageTableInfo; +use common_catalog::table::AppendMode; +use common_catalog::table::Table; +use common_catalog::table_context::TableContext; +use common_exception::Result; +use common_expression::DataSchema; +use common_expression::DataSchemaRef; +use common_expression::Scalar; +use common_meta_app::principal::StageInfo; +use common_meta_app::schema::TableCopiedFileInfo; +use common_meta_app::schema::UpsertTableCopiedFileReq; +use common_pipeline_core::Pipeline; +use common_sql::executor::DistributedCopyIntoTable; +use common_sql::plans::CopyIntoTableMode; +use common_sql::plans::CopyIntoTablePlan; +use common_storage::StageFileInfo; +use tracing::debug; +use tracing::error; +use tracing::info; + +use crate::pipelines::builders::build_append2table_pipeline; +use crate::pipelines::builders::build_append2table_without_commit_pipeline; +use crate::pipelines::builders::try_purge_files; +use crate::pipelines::processors::transforms::TransformAddConstColumns; +use crate::pipelines::processors::TransformCastSchema; +use crate::sessions::QueryContext; + +pub enum CopyPlanParam { + CopyIntoTablePlanOption(CopyIntoTablePlan), + DistributedCopyIntoTable(DistributedCopyIntoTable), +} + +#[allow(clippy::too_many_arguments)] +pub fn build_append_data_with_finish_pipeline( + ctx: Arc, + main_pipeline: &mut Pipeline, + source_schema: Arc, + plan_option: CopyPlanParam, + to_table: Arc, + files: Vec, + start: Instant, + use_commit: bool, +) -> Result<()> { + let plan_required_source_schema: DataSchemaRef; + let plan_required_values_schema: DataSchemaRef; + let plan_values_consts: Vec; + let plan_stage_table_info: StageTableInfo; + let plan_force: bool; + let plan_write_mode: CopyIntoTableMode; + let local_id; + match plan_option { + CopyPlanParam::CopyIntoTablePlanOption(plan) => { + plan_required_source_schema = plan.required_source_schema; + plan_required_values_schema = plan.required_values_schema; + plan_values_consts = plan.values_consts; + plan_stage_table_info = plan.stage_table_info; + plan_force = plan.force; + plan_write_mode = plan.write_mode; + local_id = ctx.get_cluster().local_id.clone(); + } + CopyPlanParam::DistributedCopyIntoTable(plan) => { + plan_required_source_schema = plan.required_source_schema; + plan_required_values_schema = plan.required_values_schema; + plan_values_consts = plan.values_consts; + plan_stage_table_info = plan.stage_table_info; + plan_force = plan.force; + plan_write_mode = plan.write_mode; + local_id = plan.local_node_id; + } + } + + debug!("source schema:{:?}", source_schema); + debug!("required source schema:{:?}", plan_required_source_schema); + debug!("required values schema:{:?}", plan_required_values_schema); + + if source_schema != plan_required_source_schema { + // only parquet need cast + let func_ctx = ctx.get_function_context()?; + main_pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformCastSchema::try_create( + transform_input_port, + transform_output_port, + source_schema.clone(), + plan_required_source_schema.clone(), + func_ctx.clone(), + ) + })?; + } + + if !plan_values_consts.is_empty() { + fill_const_columns( + ctx.clone(), + main_pipeline, + source_schema, + plan_required_values_schema.clone(), + plan_values_consts, + )?; + } + + let stage_info_clone = plan_stage_table_info.stage_info; + let write_mode = plan_write_mode; + let mut purge = true; + match write_mode { + CopyIntoTableMode::Insert { overwrite } => { + if use_commit { + build_append2table_pipeline( + ctx.clone(), + main_pipeline, + to_table, + plan_required_values_schema, + None, + overwrite, + AppendMode::Copy, + )?; + } else { + build_append2table_without_commit_pipeline( + ctx.clone(), + main_pipeline, + to_table, + plan_required_values_schema, + AppendMode::Copy, + )? + } + } + CopyIntoTableMode::Copy => { + if !stage_info_clone.copy_options.purge { + purge = false; + } + + if use_commit { + let copied_files = build_upsert_copied_files_to_meta_req( + ctx.clone(), + to_table.clone(), + stage_info_clone.clone(), + files.clone(), + plan_force, + )?; + build_append2table_pipeline( + ctx.clone(), + main_pipeline, + to_table, + plan_required_values_schema, + copied_files, + false, + AppendMode::Copy, + )?; + } else { + build_append2table_without_commit_pipeline( + ctx.clone(), + main_pipeline, + to_table, + plan_required_values_schema, + AppendMode::Copy, + )? + } + } + CopyIntoTableMode::Replace => {} + } + + if local_id == ctx.get_cluster().local_id { + main_pipeline.set_on_finished(move |may_error| { + match may_error { + None => { + GlobalIORuntime::instance().block_on(async move { + { + let status = + format!("end of commit, number of copied files:{}", files.len()); + ctx.set_status_info(&status); + info!(status); + } + + // 1. log on_error mode errors. + // todo(ariesdevil): persist errors with query_id + if let Some(error_map) = ctx.get_maximum_error_per_file() { + for (file_name, e) in error_map { + error!( + "copy(on_error={}): file {} encounter error {},", + stage_info_clone.copy_options.on_error, + file_name, + e.to_string() + ); + } + } + + // 2. Try to purge copied files if purge option is true, if error will skip. + // If a file is already copied(status with AlreadyCopied) we will try to purge them. + if purge { + try_purge_files(ctx.clone(), &stage_info_clone, &files).await; + } + + // Status. + { + info!("all copy finished, elapsed:{}", start.elapsed().as_secs()); + } + + Ok(()) + })?; + } + Some(error) => { + error!( + "copy failed, elapsed:{}, reason: {}", + start.elapsed().as_secs(), + error + ); + } + } + Ok(()) + }); + } else { + // remote node does nothing. + main_pipeline.set_on_finished(move |_| Ok(())) + } + + Ok(()) +} + +pub fn build_upsert_copied_files_to_meta_req( + ctx: Arc, + to_table: Arc, + stage_info: StageInfo, + copied_files: Vec, + force: bool, +) -> Result> { + let mut copied_file_tree = BTreeMap::new(); + for file in &copied_files { + // Short the etag to 7 bytes for less space in metasrv. + let short_etag = file.etag.clone().map(|mut v| { + v.truncate(7); + v + }); + copied_file_tree.insert(file.path.clone(), TableCopiedFileInfo { + etag: short_etag, + content_length: file.size, + last_modified: Some(file.last_modified), + }); + } + + let expire_hours = ctx.get_settings().get_load_file_metadata_expire_hours()?; + + let upsert_copied_files_request = { + if stage_info.copy_options.purge && force { + // if `purge-after-copy` is enabled, and in `force` copy mode, + // we do not need to upsert copied files into meta server + info!( + "[purge] and [force] are both enabled, will not update copied-files set. ({})", + &to_table.get_table_info().desc + ); + None + } else if copied_file_tree.is_empty() { + None + } else { + debug!("upsert_copied_files_info: {:?}", copied_file_tree); + let expire_at = expire_hours * 60 * 60 + Utc::now().timestamp() as u64; + let req = UpsertTableCopiedFileReq { + file_info: copied_file_tree, + expire_at: Some(expire_at), + fail_if_duplicated: !force, + }; + Some(req) + } + }; + + Ok(upsert_copied_files_request) +} + +pub fn fill_const_columns( + ctx: Arc, + pipeline: &mut Pipeline, + input_schema: DataSchemaRef, + output_schema: DataSchemaRef, + const_values: Vec, +) -> Result<()> { + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformAddConstColumns::try_create( + ctx.clone(), + transform_input_port, + transform_output_port, + input_schema.clone(), + output_schema.clone(), + const_values.clone(), + ) + })?; + Ok(()) +} diff --git a/src/query/service/src/pipelines/builders/mod.rs b/src/query/service/src/pipelines/builders/mod.rs new file mode 100644 index 000000000000..5de080cb56ca --- /dev/null +++ b/src/query/service/src/pipelines/builders/mod.rs @@ -0,0 +1,26 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod copy; +mod stage; +mod table; + +pub use copy::build_append_data_with_finish_pipeline; +pub use copy::build_upsert_copied_files_to_meta_req; +pub use copy::fill_const_columns; +pub use copy::CopyPlanParam; +pub use stage::try_purge_files; +pub use table::build_append2table_pipeline; +pub use table::build_append2table_without_commit_pipeline; +pub use table::build_fill_missing_columns_pipeline; diff --git a/src/query/service/src/interpreters/common/stage.rs b/src/query/service/src/pipelines/builders/stage.rs similarity index 100% rename from src/query/service/src/interpreters/common/stage.rs rename to src/query/service/src/pipelines/builders/stage.rs diff --git a/src/query/service/src/pipelines/builders/table.rs b/src/query/service/src/pipelines/builders/table.rs new file mode 100644 index 000000000000..c4a3bd39f42b --- /dev/null +++ b/src/query/service/src/pipelines/builders/table.rs @@ -0,0 +1,99 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_catalog::table::AppendMode; +use common_catalog::table::Table; +use common_exception::Result; +use common_expression::DataSchemaRef; +use common_meta_app::schema::UpsertTableCopiedFileReq; +use common_pipeline_core::Pipeline; + +use crate::pipelines::processors::transforms::TransformAddComputedColumns; +use crate::pipelines::processors::TransformResortAddOn; +use crate::sessions::QueryContext; + +pub fn build_fill_missing_columns_pipeline( + ctx: Arc, + pipeline: &mut Pipeline, + table: Arc, + source_schema: DataSchemaRef, +) -> Result<()> { + let table_default_schema = &table.schema().remove_computed_fields(); + let table_computed_schema = &table.schema().remove_virtual_computed_fields(); + let default_schema: DataSchemaRef = Arc::new(table_default_schema.into()); + let computed_schema: DataSchemaRef = Arc::new(table_computed_schema.into()); + + // Fill missing default columns and resort the columns. + if source_schema != default_schema { + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformResortAddOn::try_create( + ctx.clone(), + transform_input_port, + transform_output_port, + source_schema.clone(), + default_schema.clone(), + table.clone(), + ) + })?; + } + + // Fill computed columns. + if default_schema != computed_schema { + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformAddComputedColumns::try_create( + ctx.clone(), + transform_input_port, + transform_output_port, + default_schema.clone(), + computed_schema.clone(), + ) + })?; + } + + Ok(()) +} + +pub fn build_append2table_pipeline( + ctx: Arc, + main_pipeline: &mut Pipeline, + table: Arc, + source_schema: DataSchemaRef, + copied_files: Option, + overwrite: bool, + append_mode: AppendMode, +) -> Result<()> { + build_fill_missing_columns_pipeline(ctx.clone(), main_pipeline, table.clone(), source_schema)?; + + table.append_data(ctx.clone(), main_pipeline, append_mode)?; + + table.commit_insertion(ctx, main_pipeline, copied_files, overwrite)?; + + Ok(()) +} + +pub fn build_append2table_without_commit_pipeline( + ctx: Arc, + main_pipeline: &mut Pipeline, + table: Arc, + source_schema: DataSchemaRef, + append_mode: AppendMode, +) -> Result<()> { + build_fill_missing_columns_pipeline(ctx.clone(), main_pipeline, table.clone(), source_schema)?; + + table.append_data(ctx, main_pipeline, append_mode)?; + + Ok(()) +} diff --git a/src/query/service/src/pipelines/mod.rs b/src/query/service/src/pipelines/mod.rs index 73b29b231608..249c98d4e631 100644 --- a/src/query/service/src/pipelines/mod.rs +++ b/src/query/service/src/pipelines/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod builders; pub mod executor; pub mod processors; diff --git a/src/query/service/src/pipelines/pipeline_builder.rs b/src/query/service/src/pipelines/pipeline_builder.rs index a84b00c25820..d94de034d0e3 100644 --- a/src/query/service/src/pipelines/pipeline_builder.rs +++ b/src/query/service/src/pipelines/pipeline_builder.rs @@ -91,9 +91,9 @@ use super::processors::transforms::WindowFunctionInfo; use super::processors::TransformExpandGroupingSets; use crate::api::DefaultExchangeInjector; use crate::api::ExchangeInjector; -use crate::interpreters::append_data_and_set_finish; -use crate::interpreters::fill_missing_columns; -use crate::interpreters::PlanParam; +use crate::pipelines::builders::build_append_data_with_finish_pipeline; +use crate::pipelines::builders::build_fill_missing_columns_pipeline; +use crate::pipelines::builders::CopyPlanParam; use crate::pipelines::processors::transforms::build_partition_bucket; use crate::pipelines::processors::transforms::AggregateInjector; use crate::pipelines::processors::transforms::FinalSingleStateAggregator; @@ -225,11 +225,11 @@ impl PipelineBuilder { let start = Instant::now(); stage_table.read_data(table_ctx, &distributed_plan.source, &mut self.main_pipeline)?; // append data - append_data_and_set_finish( + build_append_data_with_finish_pipeline( + ctx, &mut self.main_pipeline, distributed_plan.required_source_schema.clone(), - PlanParam::DistributedCopyIntoTable(distributed_plan.clone()), - ctx, + CopyPlanParam::DistributedCopyIntoTable(distributed_plan.clone()), to_table, distributed_plan.files.clone(), start, @@ -1384,11 +1384,11 @@ impl PipelineBuilder { .get_table_by_info(&insert_select.table_info)?; let source_schema = insert_schema; - fill_missing_columns( + build_fill_missing_columns_pipeline( self.ctx.clone(), + &mut self.main_pipeline, table.clone(), source_schema.clone(), - &mut self.main_pipeline, )?; table.append_data( diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs index 67a537078cfb..9dffd7055041 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs @@ -46,6 +46,7 @@ use crate::pipelines::processors::transforms::aggregator::serde::TransformScatte use crate::pipelines::processors::transforms::group_by::Area; use crate::pipelines::processors::transforms::group_by::ArenaHolder; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; +use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::pipelines::processors::transforms::HashTableCell; use crate::pipelines::processors::transforms::TransformAggregateDeserializer; use crate::pipelines::processors::transforms::TransformAggregateSerializer; @@ -90,50 +91,49 @@ struct HashTableHashScatter, } -impl HashTableHashScatter { - fn scatter( - &self, - mut payload: HashTablePayload, - ) -> Result>> { - let mut buckets = Vec::with_capacity(self.buckets); +fn scatter( + mut payload: HashTablePayload, + buckets: usize, + method: &Method, +) -> Result>> { + let mut buckets = Vec::with_capacity(buckets); - for _ in 0..self.buckets { - buckets.push(self.method.create_hash_table(Arc::new(Bump::new()))?); - } + for _ in 0..buckets.capacity() { + buckets.push(method.create_hash_table(Arc::new(Bump::new()))?); + } - for item in payload.cell.hashtable.iter() { - let mods = StrengthReducedU64::new(self.buckets as u64); - let bucket_index = (item.key().fast_hash() % mods) as usize; + let mods = StrengthReducedU64::new(buckets.len() as u64); + for item in payload.cell.hashtable.iter() { + let bucket_index = (item.key().fast_hash() % mods) as usize; - unsafe { - match buckets[bucket_index].insert_and_entry(item.key()) { - Ok(mut entry) => { - *entry.get_mut() = *item.get(); - } - Err(mut entry) => { - *entry.get_mut() = *item.get(); - } + unsafe { + match buckets[bucket_index].insert_and_entry(item.key()) { + Ok(mut entry) => { + *entry.get_mut() = *item.get(); + } + Err(mut entry) => { + *entry.get_mut() = *item.get(); } } } + } - let mut res = Vec::with_capacity(buckets.len()); - let dropper = payload.cell._dropper.take(); - let arena = std::mem::replace(&mut payload.cell.arena, Area::create()); - payload - .cell - .arena_holders - .push(ArenaHolder::create(Some(arena))); - for bucket_table in buckets { - let mut cell = - HashTableCell::::create(bucket_table, dropper.clone().unwrap()); - cell.arena_holders - .extend(payload.cell.arena_holders.clone()); - res.push(cell); - } + let mut res = Vec::with_capacity(buckets.len()); + let dropper = payload.cell._dropper.take(); + let arena = std::mem::replace(&mut payload.cell.arena, Area::create()); + payload + .cell + .arena_holders + .push(ArenaHolder::create(Some(arena))); - Ok(res) + for bucket_table in buckets { + let mut cell = HashTableCell::::create(bucket_table, dropper.clone().unwrap()); + cell.arena_holders + .extend(payload.cell.arena_holders.clone()); + res.push(cell); } + + Ok(res) } impl FlightScatter @@ -148,22 +148,19 @@ impl FlightScatter AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), AggregateMeta::Spilling(payload) => { - let bucket = payload.bucket; - for hashtable_cell in self.scatter(payload)? { + let method = PartitionedHashMethod::create(self.method.clone()); + for hashtable_cell in scatter(payload, self.buckets, &method)? { blocks.push(match hashtable_cell.hashtable.len() == 0 { true => DataBlock::empty(), false => DataBlock::empty_with_meta( - AggregateMeta::::create_spilling( - bucket, - hashtable_cell, - ), + AggregateMeta::::create_spilling(hashtable_cell), ), }); } } AggregateMeta::HashTable(payload) => { let bucket = payload.bucket; - for hashtable_cell in self.scatter(payload)? { + for hashtable_cell in scatter(payload, self.buckets, &self.method)? { blocks.push(match hashtable_cell.hashtable.len() == 0 { true => DataBlock::empty(), false => DataBlock::empty_with_meta( diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs index 3b98387484c2..aa34a39d0cc3 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs @@ -15,6 +15,7 @@ use std::any::Any; use std::fmt::Debug; use std::fmt::Formatter; +use std::ops::Range; use common_expression::BlockMetaInfo; use common_expression::BlockMetaInfoPtr; @@ -23,6 +24,7 @@ use common_expression::DataBlock; use crate::pipelines::processors::transforms::group_by::ArenaHolder; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; +use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::pipelines::processors::transforms::HashTableCell; pub struct HashTablePayload { @@ -46,13 +48,14 @@ impl SerializedPayload { pub struct SpilledPayload { pub bucket: isize, pub location: String, + pub data_range: Range, pub columns_layout: Vec, } pub enum AggregateMeta { Serialized(SerializedPayload), HashTable(HashTablePayload), - Spilling(HashTablePayload), + Spilling(HashTablePayload, V>), Spilled(SpilledPayload), Partitioned { bucket: isize, data: Vec }, @@ -74,10 +77,12 @@ impl AggregateMeta) -> BlockMetaInfoPtr { + pub fn create_spilling( + cell: HashTableCell, V>, + ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::::Spilling(HashTablePayload { cell, - bucket, + bucket: 0, arena_holder: ArenaHolder::create(None), })) } @@ -85,11 +90,13 @@ impl AggregateMeta, columns_layout: Vec, ) -> BlockMetaInfoPtr { Box::new(AggregateMeta::::Spilled(SpilledPayload { bucket, location, + data_range, columns_layout, })) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs index e479927891e5..923715363203 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/serde_meta.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::ops::Range; use common_expression::BlockMetaInfo; use common_expression::BlockMetaInfoDowncast; @@ -27,6 +28,7 @@ pub struct AggregateSerdeMeta { pub typ: usize, pub bucket: isize, pub location: Option, + pub data_range: Option>, pub columns_layout: Vec, } @@ -36,6 +38,7 @@ impl AggregateSerdeMeta { typ: BUCKET_TYPE, bucket, location: None, + data_range: None, columns_layout: vec![], }) } @@ -43,6 +46,7 @@ impl AggregateSerdeMeta { pub fn create_spilled( bucket: isize, location: String, + data_range: Range, columns_layout: Vec, ) -> BlockMetaInfoPtr { Box::new(AggregateSerdeMeta { @@ -50,6 +54,7 @@ impl AggregateSerdeMeta { bucket, columns_layout, location: Some(location), + data_range: Some(data_range), }) } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs index 056478349e90..89e6346d7498 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs @@ -139,6 +139,7 @@ impl TransformAggregateSerializer { AggregateSerdeMeta::create_spilled( payload.bucket, payload.location, + payload.data_range, payload.columns_layout, ), ))); @@ -163,10 +164,10 @@ impl TransformAggregateSerializer { pub fn serialize_aggregate( method: &Method, params: &Arc, - payload: HashTablePayload, + hashtable: &Method::HashTable, ) -> Result { - let keys_len = payload.cell.hashtable.len(); - let value_size = estimated_key_size(&payload.cell.hashtable); + let keys_len = hashtable.len(); + let value_size = estimated_key_size(hashtable); let funcs = ¶ms.aggregate_functions; let offsets_aggregate_states = ¶ms.offsets_aggregate_states; @@ -178,7 +179,7 @@ pub fn serialize_aggregate( let mut group_key_builder = method.keys_column_builder(keys_len, value_size); - for group_entity in payload.cell.hashtable.iter() { + for group_entity in hashtable.iter() { let place = Into::::into(*group_entity.get()); for (idx, func) in funcs.iter().enumerate() { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs index 01552f270212..8825f565ff5b 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::collections::VecDeque; use std::sync::Arc; use std::time::Instant; @@ -23,6 +24,7 @@ use common_expression::arrow::serialize_column; use common_expression::BlockEntry; use common_expression::BlockMetaInfoDowncast; use common_expression::DataBlock; +use common_hashtable::HashtableLike; use common_pipeline_core::processors::port::InputPort; use common_pipeline_core::processors::port::OutputPort; use common_pipeline_core::processors::processor::Event; @@ -35,6 +37,11 @@ use crate::pipelines::processors::transforms::aggregator::aggregate_meta::Aggreg use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; use crate::pipelines::processors::transforms::aggregator::serde::transform_aggregate_serializer::serialize_aggregate; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; +use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_data_serialize_milliseconds; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_write_bytes; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_write_count; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_write_milliseconds; use crate::pipelines::processors::AggregatorParams; pub struct TransformAggregateSpillWriter { @@ -45,7 +52,7 @@ pub struct TransformAggregateSpillWriter { operator: Operator, location_prefix: String, - output_block: Option, + spilled_blocks: VecDeque, spilling_meta: Option>, spilling_future: Option>>, } @@ -66,7 +73,7 @@ impl TransformAggregateSpillWriter { params, operator, location_prefix, - output_block: None, + spilled_blocks: VecDeque::new(), spilling_meta: None, spilling_future: None, }) @@ -99,9 +106,11 @@ impl Processor for TransformAggregateSpillWriter Processor for TransformAggregateSpillWriter Result<()> { if let Some(spilling_meta) = self.spilling_meta.take() { if let AggregateMeta::Spilling(payload) = spilling_meta { - let (output_block, spilling_future) = spilling_aggregate_payload( + let (spilled_blocks, spilling_future) = spilling_aggregate_payload( self.operator.clone(), &self.method, &self.location_prefix, @@ -148,7 +157,7 @@ impl Processor for TransformAggregateSpillWriter Vec { data_block.columns().to_vec() } -fn serialize_spill_file( - method: &Method, - params: &Arc, - payload: HashTablePayload, -) -> Result<(isize, usize, Vec>)> { - let bucket = payload.bucket; - let data_block = serialize_aggregate(method, params, payload)?; - let columns = get_columns(data_block); - - let mut total_size = 0; - let mut columns_data = Vec::with_capacity(columns.len()); - for column in columns.into_iter() { - let column = column.value.as_column().unwrap(); - let column_data = serialize_column(column); - total_size += column_data.len(); - columns_data.push(column_data); - } - - Ok((bucket, total_size, columns_data)) -} - pub fn spilling_aggregate_payload( operator: Operator, method: &Method, location_prefix: &str, params: &Arc, - payload: HashTablePayload, -) -> Result<(DataBlock, BoxFuture<'static, Result<()>>)> { - let (bucket, total_size, data) = serialize_spill_file(method, params, payload)?; - + mut payload: HashTablePayload, usize>, +) -> Result<(VecDeque, BoxFuture<'static, Result<()>>)> { let unique_name = GlobalUniqName::unique(); let location = format!("{}/{}", location_prefix, unique_name); - let columns_layout = data.iter().map(Vec::len).collect::>(); - let output_data_block = DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(bucket, location.clone(), columns_layout), - ); + + let mut write_size = 0; + let mut write_data = Vec::with_capacity(256); + let mut spilled_blocks = VecDeque::with_capacity(256); + for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { + if inner_table.len() == 0 { + spilled_blocks.push_back(DataBlock::empty()); + continue; + } + + let now = Instant::now(); + let data_block = serialize_aggregate(method, params, inner_table)?; + + let begin = write_size; + let columns = get_columns(data_block); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len()); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + spilled_blocks.push_back(DataBlock::empty_with_meta( + AggregateMeta::::create_spilled( + bucket as isize, + location.clone(), + begin..write_size, + columns_layout, + ), + )); + } Ok(( - output_data_block, + spilled_blocks, Box::pin(async move { let instant = Instant::now(); - // temp code: waiting https://github.com/datafuselabs/opendal/pull/1431 - let mut write_data = Vec::with_capacity(total_size); + let mut write_bytes = 0; + + if !write_data.is_empty() { + let mut writer = operator.writer(&location).await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } - for data in data.into_iter() { - write_data.extend(data); + writer.close().await?; } - operator.write(&location, write_data).await?; + // perf + { + metrics_inc_aggregate_spill_write_count(); + metrics_inc_aggregate_spill_write_bytes(write_bytes as u64); + metrics_inc_aggregate_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + } info!( "Write aggregate spill {} successfully, elapsed: {:?}", diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs index 9700f1dd066d..28bd13bbbb09 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs @@ -91,6 +91,7 @@ where false => AggregateMeta::::create_spilled( meta.bucket, meta.location.unwrap(), + meta.data_range.unwrap(), meta.columns_layout, ), }, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs index 0195d853ca61..d0c99afb24c0 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs @@ -129,6 +129,7 @@ impl TransformGroupBySerializer { AggregateSerdeMeta::create_spilled( payload.bucket, payload.location, + payload.data_range, payload.columns_layout, ), ))); @@ -149,13 +150,13 @@ impl TransformGroupBySerializer { pub fn serialize_group_by( method: &Method, - payload: HashTablePayload, + hashtable: &Method::HashTable<()>, ) -> Result { - let keys_len = payload.cell.hashtable.len(); - let value_size = estimated_key_size(&payload.cell.hashtable); + let keys_len = hashtable.len(); + let value_size = estimated_key_size(hashtable); let mut group_key_builder = method.keys_column_builder(keys_len, value_size); - for group_entity in payload.cell.hashtable.iter() { + for group_entity in hashtable.iter() { group_key_builder.append_value(group_entity.key()); } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs index 24c41b292c9c..65f4fd894bfc 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::collections::VecDeque; use std::sync::Arc; use std::time::Instant; @@ -23,6 +24,7 @@ use common_expression::arrow::serialize_column; use common_expression::BlockEntry; use common_expression::BlockMetaInfoDowncast; use common_expression::DataBlock; +use common_hashtable::HashtableLike; use common_pipeline_core::processors::port::InputPort; use common_pipeline_core::processors::port::OutputPort; use common_pipeline_core::processors::processor::Event; @@ -35,6 +37,11 @@ use crate::pipelines::processors::transforms::aggregator::aggregate_meta::Aggreg use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; use crate::pipelines::processors::transforms::aggregator::serde::transform_group_by_serializer::serialize_group_by; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; +use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_data_serialize_milliseconds; +use crate::pipelines::processors::transforms::metrics::metrics_inc_group_by_spill_write_bytes; +use crate::pipelines::processors::transforms::metrics::metrics_inc_group_by_spill_write_count; +use crate::pipelines::processors::transforms::metrics::metrics_inc_group_by_spill_write_milliseconds; pub struct TransformGroupBySpillWriter { method: Method, @@ -43,7 +50,7 @@ pub struct TransformGroupBySpillWriter { operator: Operator, location_prefix: String, - output_block: Option, + spilled_blocks: VecDeque, spilling_meta: Option>, spilling_future: Option>>, } @@ -62,7 +69,7 @@ impl TransformGroupBySpillWriter { output, operator, location_prefix, - output_block: None, + spilled_blocks: VecDeque::new(), spilling_meta: None, spilling_future: None, }) @@ -95,9 +102,11 @@ impl Processor for TransformGroupBySpillWriter return Ok(Event::Async); } - if let Some(spilled_meta) = self.output_block.take() { - self.output.push_data(Ok(spilled_meta)); - return Ok(Event::NeedConsume); + while let Some(spilled_meta) = self.spilled_blocks.pop_front() { + if !spilled_meta.is_empty() || spilled_meta.get_meta().is_some() { + self.output.push_data(Ok(spilled_meta)); + return Ok(Event::NeedConsume); + } } if self.spilling_meta.is_some() { @@ -136,14 +145,14 @@ impl Processor for TransformGroupBySpillWriter fn process(&mut self) -> Result<()> { if let Some(spilling_meta) = self.spilling_meta.take() { if let AggregateMeta::Spilling(payload) = spilling_meta { - let (output_block, spilling_future) = spilling_group_by_payload( + let (spilled_blocks, spilling_future) = spilling_group_by_payload( self.operator.clone(), &self.method, &self.location_prefix, payload, )?; - self.output_block = Some(output_block); + self.spilled_blocks = spilled_blocks; self.spilling_future = Some(spilling_future); return Ok(()); @@ -171,54 +180,81 @@ fn get_columns(data_block: DataBlock) -> Vec { data_block.columns().to_vec() } -fn serialize_spill_file( - method: &Method, - payload: HashTablePayload, -) -> Result<(isize, usize, Vec>)> { - let bucket = payload.bucket; - let data_block = serialize_group_by(method, payload)?; - let columns = get_columns(data_block); - - let mut total_size = 0; - let mut columns_data = Vec::with_capacity(columns.len()); - for column in columns.into_iter() { - let column = column.value.as_column().unwrap(); - let column_data = serialize_column(column); - total_size += column_data.len(); - columns_data.push(column_data); - } - - Ok((bucket, total_size, columns_data)) -} - pub fn spilling_group_by_payload( operator: Operator, method: &Method, location_prefix: &str, - payload: HashTablePayload, -) -> Result<(DataBlock, BoxFuture<'static, Result<()>>)> { - let (bucket, total_size, data) = serialize_spill_file(method, payload)?; - + mut payload: HashTablePayload, ()>, +) -> Result<(VecDeque, BoxFuture<'static, Result<()>>)> { let unique_name = GlobalUniqName::unique(); let location = format!("{}/{}", location_prefix, unique_name); - let columns_layout = data.iter().map(Vec::len).collect::>(); - let output_data_block = DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(bucket, location.clone(), columns_layout), - ); + + let mut write_size = 0; + let mut write_data = Vec::with_capacity(256); + let mut spilled_blocks = VecDeque::with_capacity(256); + for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { + if inner_table.len() == 0 { + spilled_blocks.push_back(DataBlock::empty()); + continue; + } + + let now = Instant::now(); + let data_block = serialize_group_by(method, inner_table)?; + + let begin = write_size; + let columns = get_columns(data_block); + let mut columns_data = Vec::with_capacity(columns.len()); + let mut columns_layout = Vec::with_capacity(columns.len()); + for column in columns.into_iter() { + let column = column.value.as_column().unwrap(); + let column_data = serialize_column(column); + write_size += column_data.len() as u64; + columns_layout.push(column_data.len()); + columns_data.push(column_data); + } + + // perf + { + metrics_inc_aggregate_spill_data_serialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + + write_data.push(columns_data); + spilled_blocks.push_back(DataBlock::empty_with_meta( + AggregateMeta::::create_spilled( + bucket as isize, + location.clone(), + begin..write_size, + columns_layout, + ), + )); + } Ok(( - output_data_block, + spilled_blocks, Box::pin(async move { let instant = Instant::now(); - // temp code: waiting https://github.com/datafuselabs/opendal/pull/1431 - let mut write_data = Vec::with_capacity(total_size); + let mut write_bytes = 0; + if !write_data.is_empty() { + let mut writer = operator.writer(&location).await?; + for write_bucket_data in write_data.into_iter() { + for data in write_bucket_data.into_iter() { + write_bytes += data.len(); + writer.write(data).await?; + } + } - for data in data.into_iter() { - write_data.extend(data); + writer.close().await?; } - operator.write(&location, write_data).await?; + // perf + { + metrics_inc_group_by_spill_write_count(); + metrics_inc_group_by_spill_write_bytes(write_bytes as u64); + metrics_inc_group_by_spill_write_milliseconds(instant.elapsed().as_millis() as u64); + } info!( "Write aggregate spill {} successfully, elapsed: {:?}", diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_serializer.rs index a13c86f67472..6e46679c389d 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_serializer.rs @@ -136,6 +136,7 @@ impl Processor for TransformScatterAggregateSerializer AggregateSerdeMeta::create_spilled( bucket, payload.location, + payload.data_range, payload.columns_layout, ), ); @@ -239,6 +240,7 @@ where Method: HashMethodBounds DataBlock::empty_with_meta(AggregateSerdeMeta::create_spilled( bucket, payload.location, + payload.data_range, payload.columns_layout, )); @@ -246,7 +248,12 @@ where Method: HashMethodBounds } AggregateMeta::HashTable(payload) => { let bucket = payload.bucket; - let data_block = serialize_aggregate(&self.method, &self.params, payload)?; + let data_block = serialize_aggregate( + &self.method, + &self.params, + &payload.cell.hashtable, + )?; + drop(payload); let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))?; serialize_block(bucket, data_block, &self.ipc_fields, &self.options)? diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_spill_writer.rs index 0c95d0caafcd..5c85d752916f 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_aggregate_spill_writer.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::any::Any; +use std::collections::VecDeque; use std::sync::Arc; use common_exception::Result; @@ -40,7 +41,7 @@ pub struct TransformScatterAggregateSpillWriter { location_prefix: String, params: Arc, input_data_block: Option, - output_data_block: Option, + output_data_block: VecDeque, spilling_futures: Vec>>, } @@ -61,8 +62,8 @@ impl TransformScatterAggregateSpillWriter { operator, location_prefix, input_data_block: None, - output_data_block: None, spilling_futures: vec![], + output_data_block: VecDeque::new(), }) } } @@ -93,7 +94,7 @@ impl Processor for TransformScatterAggregateSpillWrite return Ok(Event::Async); } - if let Some(output_block) = self.output_data_block.take() { + if let Some(output_block) = self.output_data_block.pop_front() { self.output.push_data(Ok(output_block)); return Ok(Event::NeedConsume); } @@ -125,6 +126,7 @@ impl Processor for TransformScatterAggregateSpillWrite { let mut new_blocks = Vec::with_capacity(block_meta.blocks.len()); + let mut max_size = 0; for mut block in block_meta.blocks { let block_meta = block .get_meta() @@ -135,7 +137,7 @@ impl Processor for TransformScatterAggregateSpillWrite .take_meta() .and_then(AggregateMeta::::downcast_from) { - let (output_block, spilling_future) = spilling_aggregate_payload( + let (spilled_blocks, spilling_future) = spilling_aggregate_payload( self.operator.clone(), &self.method, &self.location_prefix, @@ -143,18 +145,36 @@ impl Processor for TransformScatterAggregateSpillWrite payload, )?; - new_blocks.push(output_block); + max_size = std::cmp::max(max_size, spilled_blocks.len()); + new_blocks.push(spilled_blocks); self.spilling_futures.push(Box::pin(spilling_future)); continue; } } - new_blocks.push(block); + max_size = std::cmp::max(max_size, 1); + new_blocks.push(VecDeque::from(vec![block])); } - self.output_data_block = Some(DataBlock::empty_with_meta( - ExchangeShuffleMeta::create(new_blocks), - )); + for _index in 0..max_size { + let mut has_data = false; + let mut buckets_block = Vec::with_capacity(new_blocks.len()); + for bucket_blocks in new_blocks.iter_mut() { + buckets_block.push(match bucket_blocks.pop_front() { + None => DataBlock::empty(), + Some(block) => { + has_data |= !block.is_empty() || block.get_meta().is_some(); + block + } + }); + } + + if has_data { + self.output_data_block.push_back(DataBlock::empty_with_meta( + ExchangeShuffleMeta::create(buckets_block), + )); + } + } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_group_by_serializer.rs index 5d059b8f19c8..6f39eee176fd 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_group_by_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_scatter_group_by_serializer.rs @@ -129,6 +129,7 @@ impl Processor for TransformScatterGroupBySerializer { operator: Operator, location_prefix: String, input_data_block: Option, - output_data_block: Option, + output_data_block: VecDeque, spilling_futures: Vec>>, } @@ -57,7 +58,7 @@ impl TransformScatterGroupBySpillWriter { operator, location_prefix, input_data_block: None, - output_data_block: None, + output_data_block: VecDeque::new(), spilling_futures: vec![], }) } @@ -89,7 +90,7 @@ impl Processor for TransformScatterGroupBySpillWriter< return Ok(Event::Async); } - if let Some(output_block) = self.output_data_block.take() { + if let Some(output_block) = self.output_data_block.pop_front() { self.output.push_data(Ok(output_block)); return Ok(Event::NeedConsume); } @@ -121,6 +122,7 @@ impl Processor for TransformScatterGroupBySpillWriter< { let mut new_blocks = Vec::with_capacity(block_meta.blocks.len()); + let mut max_size = 0; for mut block in block_meta.blocks { let block_meta = block .get_meta() @@ -131,25 +133,43 @@ impl Processor for TransformScatterGroupBySpillWriter< .take_meta() .and_then(AggregateMeta::::downcast_from) { - let (output_block, spilling_future) = spilling_group_by_payload( + let (spilled_blocks, spilling_future) = spilling_group_by_payload( self.operator.clone(), &self.method, &self.location_prefix, payload, )?; - new_blocks.push(output_block); + max_size = std::cmp::max(max_size, spilled_blocks.len()); + new_blocks.push(spilled_blocks); self.spilling_futures.push(spilling_future); continue; } } - new_blocks.push(block); + max_size = std::cmp::max(max_size, 1); + new_blocks.push(VecDeque::from(vec![block])); } - self.output_data_block = Some(DataBlock::empty_with_meta( - ExchangeShuffleMeta::create(new_blocks), - )); + for _index in 0..max_size { + let mut has_data = false; + let mut buckets_block = Vec::with_capacity(new_blocks.len()); + for bucket_blocks in new_blocks.iter_mut() { + buckets_block.push(match bucket_blocks.pop_front() { + None => DataBlock::empty(), + Some(block) => { + has_data |= !block.is_empty() || block.get_meta().is_some(); + block + } + }); + } + + if has_data { + self.output_data_block.push_back(DataBlock::empty_with_meta( + ExchangeShuffleMeta::create(buckets_block), + )); + } + } } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs index da21fd43be1b..2913bb5df017 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs @@ -30,13 +30,16 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; use itertools::Itertools; use opendal::Operator; -use tracing::error; use tracing::info; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SerializedPayload; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SpilledPayload; use crate::pipelines::processors::transforms::group_by::HashMethodBounds; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_data_deserialize_milliseconds; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_read_bytes; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_read_count; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_spill_read_milliseconds; type DeserializingMeta = (AggregateMeta, VecDeque>); @@ -176,14 +179,10 @@ impl Processor AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::Spilled(payload) => { let instant = Instant::now(); - let data = self.operator.read(&payload.location).await?; - - if let Err(cause) = self.operator.delete(&payload.location).await { - error!( - "Cannot delete spill file {}, cause: {:?}", - &payload.location, cause - ); - } + let data = self + .operator + .range_read(&payload.location, payload.data_range.clone()) + .await?; info!( "Read aggregate spill {} successfully, elapsed: {:?}", @@ -199,15 +198,18 @@ impl Processor if let AggregateMeta::Spilled(payload) = meta { let location = payload.location.clone(); let operator = self.operator.clone(); + let data_range = payload.data_range.clone(); read_data.push(common_base::base::tokio::spawn( async_backtrace::frame!(async move { let instant = Instant::now(); - let data = operator.read(&location).await?; - - if let Err(cause) = operator.delete(&location).await { - error!( - "Cannot delete spill file {}, cause: {:?}", - location, cause + let data = operator.range_read(&location, data_range).await?; + + // perf + { + metrics_inc_aggregate_spill_read_count(); + metrics_inc_aggregate_spill_read_bytes(data.len() as u64); + metrics_inc_aggregate_spill_read_milliseconds( + instant.elapsed().as_millis() as u64, ); } @@ -264,11 +266,20 @@ impl TransformSpillReader) -> AggregateMeta { let mut begin = 0; let mut columns = Vec::with_capacity(payload.columns_layout.len()); + + let now = Instant::now(); for column_layout in payload.columns_layout { columns.push(deserialize_column(&data[begin..begin + column_layout]).unwrap()); begin += column_layout; } + // perf + { + metrics_inc_aggregate_spill_data_deserialize_milliseconds( + now.elapsed().as_millis() as u64 + ); + } + AggregateMeta::::Serialized(SerializedPayload { bucket: payload.bucket, data_block: DataBlock::new_from_columns(columns), diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs index 19b856ddaffd..622dacd91560 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs @@ -38,6 +38,9 @@ use crate::pipelines::processors::transforms::aggregator::aggregate_meta::Aggreg use crate::pipelines::processors::transforms::group_by::HashMethodBounds; use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_partial_hashtable_allocated_bytes; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_partial_spill_cell_count; +use crate::pipelines::processors::transforms::metrics::metrics_inc_aggregate_partial_spill_count; use crate::pipelines::processors::transforms::HashTableCell; use crate::pipelines::processors::transforms::PartitionedHashTableDropper; use crate::pipelines::processors::AggregatorParams; @@ -304,20 +307,20 @@ impl AccumulatingTransform for TransformPartialAggrega if matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) { if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) { - let _dropper = v._dropper.clone(); - let cells = PartitionedHashTableDropper::split_cell(v); - let mut blocks = Vec::with_capacity(cells.len()); - for (bucket, cell) in cells.into_iter().enumerate() { - if cell.hashtable.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_spilling( - bucket as isize, - cell, - ), - )); - } + // perf + { + metrics_inc_aggregate_partial_spill_count(); + metrics_inc_aggregate_partial_spill_cell_count(1); + metrics_inc_aggregate_partial_hashtable_allocated_bytes( + v.allocated_bytes() as u64, + ); } + let _dropper = v._dropper.clone(); + let blocks = vec![DataBlock::empty_with_meta( + AggregateMeta::::create_spilling(v), + )]; + let arena = Arc::new(Bump::new()); let method = PartitionedHashMethod::::create(self.method.clone()); let new_hashtable = method.create_hash_table(arena)?; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs index 40b74d90ecb7..634004bb2e55 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs @@ -160,18 +160,9 @@ impl AccumulatingTransform for TransformPartialGroupBy if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) { let _dropper = v._dropper.clone(); - let cells = PartitionedHashTableDropper::split_cell(v); - let mut blocks = Vec::with_capacity(cells.len()); - for (bucket, cell) in cells.into_iter().enumerate() { - if cell.hashtable.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_spilling( - bucket as isize, - cell, - ), - )); - } - } + let blocks = vec![DataBlock::empty_with_meta( + AggregateMeta::::create_spilling(v), + )]; let arena = Arc::new(Bump::new()); let method = PartitionedHashMethod::::create(self.method.clone()); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state_impl.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state_impl.rs index 10428aed8f29..b9fc3db38b71 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state_impl.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_state_impl.rs @@ -60,14 +60,17 @@ use crate::sql::planner::plans::JoinType; impl HashJoinState for JoinHashTable { fn build(&self, input: DataBlock) -> Result<()> { let mut buffer = self.row_space.buffer.write(); + let mut buffer_row_size = self.row_space.buffer_row_size.write(); + *buffer_row_size += input.num_rows(); buffer.push(input); - let buffer_row_size = buffer.iter().fold(0, |acc, x| acc + x.num_rows()); - if buffer_row_size < *self.build_side_block_size_limit { + if *buffer_row_size < *self.build_side_block_size_limit { Ok(()) } else { let data_block = DataBlock::concat(buffer.as_slice())?; buffer.clear(); + *buffer_row_size = 0; drop(buffer); + drop(buffer_row_size); self.add_build_block(data_block) } } @@ -82,7 +85,8 @@ impl HashJoinState for JoinHashTable { | JoinType::Left | JoinType::LeftMark | JoinType::RightMark - | JoinType::Single + | JoinType::LeftSingle + | JoinType::RightSingle | JoinType::Right | JoinType::Full => self.probe_join(input, probe_state), JoinType::Cross => self.probe_cross_join(input, probe_state), @@ -304,7 +308,7 @@ impl HashJoinState for JoinHashTable { let build_keys_iter = $method.build_keys_iter(&keys_state)?; let space_size = match &keys_state { - KeysState::Column(Column::String(col)) => col.offsets.last(), + KeysState::Column(Column::String(col)) => col.offsets().last(), // The function `build_keys_state` of both HashMethodSerializer and HashMethodSingleString // must return `KeysState::Column(Column::String)`. _ => unreachable!(), @@ -504,7 +508,9 @@ impl HashJoinState for JoinHashTable { fn final_scan(&self, task: usize, state: &mut ProbeState) -> Result> { match &self.hash_join_desc.join_type { - JoinType::Right | JoinType::Full => self.right_and_full_outer_scan(task, state), + JoinType::Right | JoinType::RightSingle | JoinType::Full => { + self.right_and_full_outer_scan(task, state) + } JoinType::RightSemi => self.right_semi_outer_scan(task, state), JoinType::RightAnti => self.right_anti_outer_scan(task, state), JoinType::LeftMark => self.left_mark_scan(task, state), @@ -515,7 +521,11 @@ impl HashJoinState for JoinHashTable { fn need_outer_scan(&self) -> bool { matches!( self.hash_join_desc.join_type, - JoinType::Full | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti + JoinType::Full + | JoinType::Right + | JoinType::RightSingle + | JoinType::RightSemi + | JoinType::RightAnti ) } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/join_hash_table.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/join_hash_table.rs index f5698b1ee96b..69fc257c2007 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/join_hash_table.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/join_hash_table.rs @@ -147,12 +147,16 @@ impl JoinHashTable { hash_join_desc: HashJoinDesc, method: HashMethodKind, ) -> Result { - if hash_join_desc.join_type == JoinType::Left - || hash_join_desc.join_type == JoinType::Single - { + if matches!( + hash_join_desc.join_type, + JoinType::Left | JoinType::LeftSingle + ) { build_data_schema = build_schema_wrap_nullable(&build_data_schema); }; - if hash_join_desc.join_type == JoinType::Right { + if matches!( + hash_join_desc.join_type, + JoinType::Right | JoinType::RightSingle + ) { probe_data_schema = probe_schema_wrap_nullable(&probe_data_schema); } if hash_join_desc.join_type == JoinType::Full { @@ -199,7 +203,7 @@ impl JoinHashTable { let mut input = (*input).clone(); if matches!( self.hash_join_desc.join_type, - JoinType::Right | JoinType::Full + JoinType::Right | JoinType::RightSingle | JoinType::Full ) { let nullable_columns = input .columns() @@ -265,7 +269,7 @@ impl JoinHashTable { if self.fast_return()? && matches!( self.hash_join_desc.join_type, - JoinType::Left | JoinType::Single | JoinType::Full | JoinType::LeftAnti + JoinType::Left | JoinType::LeftSingle | JoinType::Full | JoinType::LeftAnti ) { return self.left_fast_return(&input); diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs index b5c7f33136d4..e781ac698d2f 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/left_join.rs @@ -90,7 +90,8 @@ impl JoinHashTable { let mut total_probe_matched = 0; if probe_matched > 0 { total_probe_matched += probe_matched; - if self.hash_join_desc.join_type == JoinType::Single && total_probe_matched > 1 { + if self.hash_join_desc.join_type == JoinType::LeftSingle && total_probe_matched > 1 + { return Err(ErrorCode::Internal( "Scalar subquery can't return more than one row", )); @@ -215,7 +216,7 @@ impl JoinHashTable { if probe_matched > 0 { total_probe_matched += probe_matched; - if self.hash_join_desc.join_type == JoinType::Single + if self.hash_join_desc.join_type == JoinType::LeftSingle && total_probe_matched > 1 { return Err(ErrorCode::Internal( @@ -312,7 +313,8 @@ impl JoinHashTable { let mut total_probe_matched = 0; if probe_matched > 0 { total_probe_matched += probe_matched; - if self.hash_join_desc.join_type == JoinType::Single && total_probe_matched > 1 { + if self.hash_join_desc.join_type == JoinType::LeftSingle && total_probe_matched > 1 + { return Err(ErrorCode::Internal( "Scalar subquery can't return more than one row", )); @@ -473,7 +475,7 @@ impl JoinHashTable { if probe_matched > 0 { total_probe_matched += probe_matched; - if self.hash_join_desc.join_type == JoinType::Single + if self.hash_join_desc.join_type == JoinType::LeftSingle && total_probe_matched > 1 { return Err(ErrorCode::Internal( diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/right_join.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/right_join.rs index 6c3c3b6b479b..991df38766d3 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/right_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_join/right_join.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::iter::TrustedLen; +use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use common_arrow::arrow::bitmap::Bitmap; @@ -21,9 +22,11 @@ use common_exception::ErrorCode; use common_exception::Result; use common_expression::DataBlock; use common_hashtable::HashJoinHashtableLike; +use common_hashtable::RowPtr; use crate::pipelines::processors::transforms::hash_join::ProbeState; use crate::pipelines::processors::JoinHashTable; +use crate::sql::plans::JoinType; impl JoinHashTable { pub(crate) fn probe_right_join<'a, H: HashJoinHashtableLike, IT>( @@ -57,6 +60,16 @@ impl JoinHashTable { .iter() .fold(0, |acc, chunk| acc + chunk.num_rows()); let outer_scan_map = unsafe { &mut *self.outer_scan_map.get() }; + let right_single_scan_map = if self.hash_join_desc.join_type == JoinType::RightSingle { + outer_scan_map + .iter_mut() + .map(|sp| unsafe { + std::mem::transmute::<*mut bool, *mut AtomicBool>(sp.as_mut_ptr()) + }) + .collect::>() + } else { + vec![] + }; for (i, key) in keys_iter.enumerate() { let (mut match_count, mut incomplete_ptr) = self.probe_key( @@ -107,9 +120,17 @@ impl JoinHashTable { let merged_block = self.merge_eq_block(&build_block, &probe_block)?; if self.hash_join_desc.other_predicate.is_none() { result_blocks.push(merged_block); - for row_ptr in local_build_indexes.iter() { - outer_scan_map[row_ptr.chunk_index as usize] - [row_ptr.row_index as usize] = true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + local_build_indexes, + &right_single_scan_map, + None, + )?; + } else { + for row_ptr in local_build_indexes.iter() { + outer_scan_map[row_ptr.chunk_index as usize] + [row_ptr.row_index as usize] = true; + } } } else { let (bm, all_true, all_false) = self.get_other_filters( @@ -119,22 +140,39 @@ impl JoinHashTable { if all_true { result_blocks.push(merged_block); - for row_ptr in local_build_indexes.iter() { - outer_scan_map[row_ptr.chunk_index as usize] - [row_ptr.row_index as usize] = true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + local_build_indexes, + &right_single_scan_map, + None, + )?; + } else { + for row_ptr in local_build_indexes.iter() { + outer_scan_map[row_ptr.chunk_index as usize] + [row_ptr.row_index as usize] = true; + } } } else if !all_false { // Safe to unwrap. let validity = bm.unwrap(); - let mut idx = 0; - while idx < max_block_size { - let valid = unsafe { validity.get_bit_unchecked(idx) }; - if valid { - outer_scan_map - [local_build_indexes[idx].chunk_index as usize] - [local_build_indexes[idx].row_index as usize] = true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + local_build_indexes, + &right_single_scan_map, + Some(&validity), + )?; + } else { + let mut idx = 0; + while idx < max_block_size { + let valid = unsafe { validity.get_bit_unchecked(idx) }; + if valid { + outer_scan_map + [local_build_indexes[idx].chunk_index as usize] + [local_build_indexes[idx].row_index as usize] = + true; + } + idx += 1; } - idx += 1; } let filtered_block = DataBlock::filter_with_bitmap(merged_block, &validity)?; @@ -198,8 +236,17 @@ impl JoinHashTable { if !merged_block.is_empty() { if self.hash_join_desc.other_predicate.is_none() { result_blocks.push(merged_block); - for row_ptr in local_build_indexes.iter().take(matched_num) { - outer_scan_map[row_ptr.chunk_index as usize][row_ptr.row_index as usize] = true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + &local_build_indexes[0..matched_num], + &right_single_scan_map, + None, + )?; + } else { + for row_ptr in local_build_indexes.iter().take(matched_num) { + outer_scan_map[row_ptr.chunk_index as usize][row_ptr.row_index as usize] = + true; + } } } else { let (bm, all_true, all_false) = self.get_other_filters( @@ -209,28 +256,82 @@ impl JoinHashTable { if all_true { result_blocks.push(merged_block); - for row_ptr in local_build_indexes.iter().take(matched_num) { - outer_scan_map[row_ptr.chunk_index as usize][row_ptr.row_index as usize] = - true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + &local_build_indexes[0..matched_num], + &right_single_scan_map, + None, + )?; + } else { + for row_ptr in local_build_indexes.iter().take(matched_num) { + outer_scan_map[row_ptr.chunk_index as usize] + [row_ptr.row_index as usize] = true; + } } } else if !all_false { // Safe to unwrap. let validity = bm.unwrap(); - let mut idx = 0; - while idx < matched_num { - let valid = unsafe { validity.get_bit_unchecked(idx) }; - if valid { - outer_scan_map[local_build_indexes[idx].chunk_index as usize] - [local_build_indexes[idx].row_index as usize] = true; + if self.hash_join_desc.join_type == JoinType::RightSingle { + self.update_right_single_scan_map( + &local_build_indexes[0..matched_num], + &right_single_scan_map, + Some(&validity), + )?; + } else { + let mut idx = 0; + while idx < matched_num { + let valid = unsafe { validity.get_bit_unchecked(idx) }; + if valid { + outer_scan_map[local_build_indexes[idx].chunk_index as usize] + [local_build_indexes[idx].row_index as usize] = true; + } + idx += 1; } - idx += 1; } let filtered_block = DataBlock::filter_with_bitmap(merged_block, &validity)?; result_blocks.push(filtered_block); } } } - Ok(result_blocks) } + + fn update_right_single_scan_map( + &self, + build_indexes: &[RowPtr], + right_single_scan_map: &[*mut AtomicBool], + bitmap: Option<&Bitmap>, + ) -> Result<()> { + let dummy_bitmap = Bitmap::new(); + let (has_bitmap, validity) = match bitmap { + Some(validity) => (true, validity), + None => (false, &dummy_bitmap), + }; + for (idx, row_ptr) in build_indexes.iter().enumerate() { + if has_bitmap && unsafe { !validity.get_bit_unchecked(idx) } { + continue; + } + let old = unsafe { + (*right_single_scan_map[row_ptr.chunk_index as usize] + .add(row_ptr.row_index as usize)) + .load(Ordering::SeqCst) + }; + if old { + return Err(ErrorCode::Internal( + "Scalar subquery can't return more than one row", + )); + } + let res = unsafe { + (*right_single_scan_map[row_ptr.chunk_index as usize] + .add(row_ptr.row_index as usize)) + .compare_exchange_weak(old, true, Ordering::SeqCst, Ordering::SeqCst) + }; + if res.is_err() { + return Err(ErrorCode::Internal( + "Scalar subquery can't return more than one row", + )); + } + } + Ok(()) + } } diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs index 5fa94f4f04fe..4c17d5d043bc 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/probe_state.rs @@ -54,7 +54,7 @@ impl ProbeState { true_validity.extend_constant(max_block_size, true); let true_validity: Bitmap = true_validity.into(); let (row_state, row_state_indexes, probe_unmatched_indexes) = match &join_type { - JoinType::Left | JoinType::Single | JoinType::Full => { + JoinType::Left | JoinType::LeftSingle | JoinType::Full => { if with_conjunct { ( Some(vec![0; max_block_size]), diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs index b9e710547efe..1cf9c7b091d3 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/result_blocks.rs @@ -71,7 +71,7 @@ impl JoinHashTable { } } // Single join is similar to left join, but the result is a single row. - JoinType::Left | JoinType::Single | JoinType::Full => { + JoinType::Left | JoinType::LeftSingle | JoinType::Full => { if self.hash_join_desc.other_predicate.is_none() { self.probe_left_join::<_, _>(hash_table, probe_state, keys_iter, input) } else { @@ -83,7 +83,7 @@ impl JoinHashTable { ) } } - JoinType::Right => { + JoinType::Right | JoinType::RightSingle => { self.probe_right_join::<_, _>(hash_table, probe_state, keys_iter, input) } // Three cases will produce Mark join: diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/row.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/row.rs index bf6253e2873c..ecf2838eb19b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/row.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/row.rs @@ -39,6 +39,7 @@ pub struct RowSpace { pub data_schema: DataSchemaRef, pub chunks: RwLock>, pub buffer: RwLock>, + pub buffer_row_size: RwLock, } impl RowSpace { @@ -48,6 +49,7 @@ impl RowSpace { data_schema, chunks: RwLock::new(vec![]), buffer: RwLock::new(Vec::with_capacity(buffer_size as usize)), + buffer_row_size: RwLock::new(0), }) } diff --git a/src/query/storages/fuse/src/table_functions/fuse_snapshots/table_args.rs b/src/query/service/src/pipelines/processors/transforms/metrics/mod.rs similarity index 61% rename from src/query/storages/fuse/src/table_functions/fuse_snapshots/table_args.rs rename to src/query/service/src/pipelines/processors/transforms/metrics/mod.rs index 96cd8b931c1a..b2a9f6d6b2cf 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_snapshots/table_args.rs +++ b/src/query/service/src/pipelines/processors/transforms/metrics/mod.rs @@ -12,14 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_exception::Result; +mod transform_metrics; -use crate::table_functions::string_value; -use crate::table_functions::TableArgs; - -pub fn parse_func_history_args(table_args: &TableArgs) -> Result<(String, String)> { - let args = table_args.expect_all_positioned("fuse_blocks", Some(2))?; - let db = string_value(&args[0])?; - let tbl = string_value(&args[1])?; - Ok((db, tbl)) -} +pub use transform_metrics::*; diff --git a/src/query/service/src/pipelines/processors/transforms/metrics/transform_metrics.rs b/src/query/service/src/pipelines/processors/transforms/metrics/transform_metrics.rs new file mode 100644 index 000000000000..bb4119308b87 --- /dev/null +++ b/src/query/service/src/pipelines/processors/transforms/metrics/transform_metrics.rs @@ -0,0 +1,86 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use metrics::increment_gauge; + +macro_rules! key { + ($key: literal) => { + concat!("transform_", $key) + }; +} + +pub fn metrics_inc_aggregate_partial_spill_count() { + increment_gauge!(key!("aggregate_partial_spill_count"), 1_f64); +} + +pub fn metrics_inc_aggregate_partial_spill_cell_count(c: u64) { + increment_gauge!(key!("aggregate_partial_spill_cell_count"), c as f64); +} + +pub fn metrics_inc_aggregate_partial_hashtable_allocated_bytes(c: u64) { + increment_gauge!( + key!("aggregate_partial_hashtable_allocated_bytes"), + c as f64 + ); +} + +pub fn metrics_inc_group_by_spill_write_count() { + increment_gauge!(key!("group_by_spill_write_count"), 1_f64); +} + +pub fn metrics_inc_group_by_spill_write_bytes(c: u64) { + increment_gauge!(key!("group_by_spill_write_bytes"), c as f64); +} + +pub fn metrics_inc_group_by_spill_write_milliseconds(c: u64) { + increment_gauge!(key!("group_by_spill_write_milliseconds"), c as f64); +} + +pub fn metrics_inc_aggregate_spill_write_count() { + increment_gauge!(key!("aggregate_spill_write_count"), 1_f64); +} + +pub fn metrics_inc_aggregate_spill_write_bytes(c: u64) { + increment_gauge!(key!("aggregate_spill_write_bytes"), c as f64); +} + +pub fn metrics_inc_aggregate_spill_write_milliseconds(c: u64) { + increment_gauge!(key!("aggregate_spill_write_milliseconds"), c as f64); +} + +pub fn metrics_inc_aggregate_spill_read_count() { + increment_gauge!(key!("aggregate_spill_read_count"), 1_f64); +} + +pub fn metrics_inc_aggregate_spill_read_bytes(c: u64) { + increment_gauge!(key!("aggregate_spill_read_bytes"), c as f64); +} + +pub fn metrics_inc_aggregate_spill_read_milliseconds(c: u64) { + increment_gauge!(key!("aggregate_spill_read_milliseconds"), c as f64); +} + +pub fn metrics_inc_aggregate_spill_data_serialize_milliseconds(c: u64) { + increment_gauge!( + key!("aggregate_spill_data_serialize_milliseconds"), + c as f64 + ); +} + +pub fn metrics_inc_aggregate_spill_data_deserialize_milliseconds(c: u64) { + increment_gauge!( + key!("aggregate_spill_data_deserialize_milliseconds"), + c as f64 + ); +} diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 95b7494eefbd..f40db1ad0662 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -21,6 +21,7 @@ mod transform_hash_join; mod transform_limit; mod window; +mod metrics; mod range_join; mod runtime_filter; mod transform_add_computed_columns; diff --git a/src/query/service/src/pipelines/processors/transforms/transform_hash_join.rs b/src/query/service/src/pipelines/processors/transforms/transform_hash_join.rs index 49d329dadadc..65a46d48a287 100644 --- a/src/query/service/src/pipelines/processors/transforms/transform_hash_join.rs +++ b/src/query/service/src/pipelines/processors/transforms/transform_hash_join.rs @@ -221,14 +221,18 @@ impl Processor for TransformHashJoinProbe { if self.join_state.fast_return()? { match self.join_state.join_type() { JoinType::Inner - | JoinType::Right | JoinType::Cross + | JoinType::Right + | JoinType::RightSingle | JoinType::RightAnti | JoinType::RightSemi | JoinType::LeftSemi => { self.step = HashJoinStep::Finished; } - JoinType::Left | JoinType::Full | JoinType::Single | JoinType::LeftAnti => { + JoinType::Left + | JoinType::Full + | JoinType::LeftSingle + | JoinType::LeftAnti => { self.step = HashJoinStep::Probe; } _ => { diff --git a/src/query/service/src/procedures/systems/clustering_information.rs b/src/query/service/src/procedures/systems/clustering_information.rs index 58886988c002..62364a0c417b 100644 --- a/src/query/service/src/procedures/systems/clustering_information.rs +++ b/src/query/service/src/procedures/systems/clustering_information.rs @@ -46,6 +46,7 @@ impl OneBlockProcedure for ClusteringInformationProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert_eq!(args.len(), 2); let database_name = args[0].clone(); let table_name = args[1].clone(); let tenant_id = ctx.get_tenant(); diff --git a/src/query/service/src/procedures/systems/execute_job.rs b/src/query/service/src/procedures/systems/execute_job.rs index 840227365471..2bd629623fd7 100644 --- a/src/query/service/src/procedures/systems/execute_job.rs +++ b/src/query/service/src/procedures/systems/execute_job.rs @@ -48,6 +48,7 @@ impl OneBlockProcedure for ExecuteJobProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert_eq!(args.len(), 1); let name = args[0].clone(); let background_handler = get_background_service_handler(); diff --git a/src/query/service/src/procedures/systems/fuse_block.rs b/src/query/service/src/procedures/systems/fuse_block.rs index 2ac3fb37a67b..49ae68106edc 100644 --- a/src/query/service/src/procedures/systems/fuse_block.rs +++ b/src/query/service/src/procedures/systems/fuse_block.rs @@ -46,6 +46,7 @@ impl OneBlockProcedure for FuseBlockProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert!(args.len() >= 2); let database_name = args[0].clone(); let table_name = args[1].clone(); let snapshot_id = if args.len() > 2 { diff --git a/src/query/service/src/procedures/systems/fuse_segment.rs b/src/query/service/src/procedures/systems/fuse_segment.rs index c36f373bd654..a47318110b4d 100644 --- a/src/query/service/src/procedures/systems/fuse_segment.rs +++ b/src/query/service/src/procedures/systems/fuse_segment.rs @@ -46,9 +46,15 @@ impl OneBlockProcedure for FuseSegmentProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert!(args.len() >= 2); let database_name = args[0].clone(); let table_name = args[1].clone(); - let snapshot_id = args[2].clone(); + let snapshot_id = if args.len() > 2 { + Some(args[2].clone()) + } else { + None + }; + let tenant_id = ctx.get_tenant(); let tbl = ctx .get_catalog(&ctx.get_current_catalog())? @@ -61,7 +67,7 @@ impl OneBlockProcedure for FuseSegmentProcedure { let tbl = FuseTable::try_from_table(tbl.as_ref())?; - Ok(FuseSegment::new(ctx, tbl, snapshot_id) + Ok(FuseSegment::new(ctx, tbl, snapshot_id, None) .get_segments() .await?) } diff --git a/src/query/service/src/procedures/systems/fuse_snapshot.rs b/src/query/service/src/procedures/systems/fuse_snapshot.rs index a7239da532b2..63b3eab0e6d2 100644 --- a/src/query/service/src/procedures/systems/fuse_snapshot.rs +++ b/src/query/service/src/procedures/systems/fuse_snapshot.rs @@ -47,6 +47,7 @@ impl OneBlockProcedure for FuseSnapshotProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert_eq!(args.len(), 2); let database_name = args[0].clone(); let table_name = args[1].clone(); let tenant_id = ctx.get_tenant(); diff --git a/src/query/service/src/procedures/systems/search_tables.rs b/src/query/service/src/procedures/systems/search_tables.rs index b704f2d29dea..5337f73b738e 100644 --- a/src/query/service/src/procedures/systems/search_tables.rs +++ b/src/query/service/src/procedures/systems/search_tables.rs @@ -52,6 +52,7 @@ impl OneBlockProcedure for SearchTablesProcedure { #[async_backtrace::framed] async fn all_data(&self, ctx: Arc, args: Vec) -> Result { + assert_eq!(args.len(), 1); let query = format!( "SELECT * FROM system.tables WHERE name like '%{}%' ORDER BY database, name", args[0] diff --git a/src/query/service/src/servers/http/clickhouse_handler.rs b/src/query/service/src/servers/http/clickhouse_handler.rs index 6247e183c762..97bebc3ab2f2 100644 --- a/src/query/service/src/servers/http/clickhouse_handler.rs +++ b/src/query/service/src/servers/http/clickhouse_handler.rs @@ -54,6 +54,7 @@ use tracing::info; use crate::interpreters::InterpreterFactory; use crate::interpreters::InterpreterPtr; +use crate::servers::http::middleware::sanitize_request_headers; use crate::servers::http::v1::HttpQueryContext; use crate::sessions::short_sql; use crate::sessions::QueryContext; @@ -61,7 +62,7 @@ use crate::sessions::SessionType; use crate::sessions::TableContext; // accept all clickhouse params, so they do not go to settings. -#[derive(Serialize, Deserialize)] +#[derive(Serialize, Deserialize, Debug)] pub struct StatementHandlerParams { query: Option, #[allow(unused)] @@ -267,6 +268,11 @@ pub async fn clickhouse_handler_post( Query(params): Query, headers: &HeaderMap, ) -> PoemResult { + info!( + "new clickhouse handler request: headers={:?}, params={:?}", + sanitize_request_headers(headers), + params, + ); let session = ctx.get_session(SessionType::ClickHouseHttpHandler); if let Some(db) = ¶ms.database { session.set_current_database(db.clone()); diff --git a/src/query/service/src/servers/http/middleware.rs b/src/query/service/src/servers/http/middleware.rs index bf61a3c020ed..90e2bbbcad7a 100644 --- a/src/query/service/src/servers/http/middleware.rs +++ b/src/query/service/src/servers/http/middleware.rs @@ -34,7 +34,7 @@ use poem::Middleware; use poem::Request; use poem::Response; use tracing::error; -use tracing::info; +use tracing::warn; use super::v1::HttpQueryContext; use crate::auth::AuthMgr; @@ -190,26 +190,42 @@ impl Endpoint for HTTPSessionEndpoint { #[async_backtrace::framed] async fn call(&self, mut req: Request) -> PoemResult { - // method, url, version, header - info!( - "http session endpoint: got new request: {} {} {:?}", - req.method(), - req.uri(), - sanitize_request_headers(req.headers()), - ); + let method = req.method().clone(); + let uri = req.uri().clone(); + let headers = req.headers().clone(); + let res = match self.auth(&req).await { Ok(ctx) => { req.extensions_mut().insert(ctx); self.ep.call(req).await } - Err(err) => Err(PoemError::from_string( - err.message(), - StatusCode::UNAUTHORIZED, - )), + Err(err) => match err.code() { + ErrorCode::AUTHENTICATE_FAILURE => { + warn!( + "http auth failure: {method} {uri}, headers={:?}, error={}", + sanitize_request_headers(&headers), + err + ); + Err(PoemError::from_string( + err.message(), + StatusCode::UNAUTHORIZED, + )) + } + _ => { + error!( + "http request err: {method} {uri}, headers={:?}, error={}", + sanitize_request_headers(&headers), + err + ); + Err(PoemError::from_string( + err.message(), + StatusCode::INTERNAL_SERVER_ERROR, + )) + } + }, }; match res { Err(err) => { - error!("http request error: {}", err); let body = Body::from_json(serde_json::json!({ "error": { "code": err.status().as_str(), @@ -224,7 +240,7 @@ impl Endpoint for HTTPSessionEndpoint { } } -fn sanitize_request_headers(headers: &HeaderMap) -> HashMap { +pub fn sanitize_request_headers(headers: &HeaderMap) -> HashMap { let sensitive_headers = vec!["authorization", "x-clickhouse-key", "cookie"]; headers .iter() diff --git a/src/query/service/src/servers/http/v1/http_query_handlers.rs b/src/query/service/src/servers/http/v1/http_query_handlers.rs index bb0d8e5640ff..8d729aedda33 100644 --- a/src/query/service/src/servers/http/v1/http_query_handlers.rs +++ b/src/query/service/src/servers/http/v1/http_query_handlers.rs @@ -204,6 +204,7 @@ async fn query_final_handler( _ctx: &HttpQueryContext, Path(query_id): Path, ) -> PoemResult { + info!("final http query: {}", query_id); let http_query_manager = HttpQueryManager::instance(); match http_query_manager.remove_query(&query_id).await { Some(query) => { @@ -226,6 +227,7 @@ async fn query_cancel_handler( _ctx: &HttpQueryContext, Path(query_id): Path, ) -> impl IntoResponse { + info!("kill http query: {}", query_id); let http_query_manager = HttpQueryManager::instance(); match http_query_manager.get_query(&query_id).await { Some(query) => { @@ -278,7 +280,7 @@ pub(crate) async fn query_handler( ctx: &HttpQueryContext, Json(req): Json, ) -> PoemResult { - info!("receive http query: {:?}", req); + info!("new http query request: {:?}", req); let http_query_manager = HttpQueryManager::instance(); let sql = req.sql.clone(); diff --git a/src/query/service/src/servers/http/v1/load.rs b/src/query/service/src/servers/http/v1/load.rs index 42cfcc56be49..5f544427d908 100644 --- a/src/query/service/src/servers/http/v1/load.rs +++ b/src/query/service/src/servers/http/v1/load.rs @@ -40,9 +40,11 @@ use poem::Request; use serde::Deserialize; use serde::Serialize; use tokio::sync::mpsc::Sender; +use tracing::info; use super::HttpQueryContext; use crate::interpreters::InterpreterFactory; +use crate::servers::http::middleware::sanitize_request_headers; use crate::sessions::QueryContext; use crate::sessions::SessionType; use crate::sessions::TableContext; @@ -89,6 +91,10 @@ pub async fn streaming_load( req: &Request, mut multipart: Multipart, ) -> PoemResult> { + info!( + "new streaming load request:, headers={:?}", + sanitize_request_headers(req.headers()), + ); let session = ctx.get_session(SessionType::HTTPStreamingLoad); let context = session .create_query_context() diff --git a/src/query/service/src/servers/http/v1/query/page_manager.rs b/src/query/service/src/servers/http/v1/query/page_manager.rs index 95020bbdb4dc..1d7a0d9c63b7 100644 --- a/src/query/service/src/servers/http/v1/query/page_manager.rs +++ b/src/query/service/src/servers/http/v1/query/page_manager.rs @@ -22,6 +22,7 @@ use common_expression::DataBlock; use common_expression::DataSchemaRef; use common_io::prelude::FormatSettings; use serde_json::Value as JsonValue; +use tracing::debug; use tracing::info; use crate::servers::http::v1::json_block::block_to_json_value; @@ -163,7 +164,7 @@ impl PageManager { let d = *t - now; match tokio::time::timeout(d, self.block_receiver.recv()).await { Ok(Some(block)) => { - info!( + debug!( "http query {} got new block with {} rows", &self.query_id, block.num_rows() diff --git a/src/query/service/src/table_functions/list_stage/table_args.rs b/src/query/service/src/table_functions/list_stage/table_args.rs index ef5e01f025f2..a00ba4aba79f 100644 --- a/src/query/service/src/table_functions/list_stage/table_args.rs +++ b/src/query/service/src/table_functions/list_stage/table_args.rs @@ -26,7 +26,7 @@ pub(crate) struct ListStageArgsParsed { impl ListStageArgsParsed { pub fn parse(table_args: &TableArgs) -> Result { - let args = table_args.expect_all_named("infer_schema")?; + let args = table_args.expect_all_named("list_stage")?; let mut location = None; let mut files_info = StageFilesInfo { diff --git a/src/query/service/src/test_kits/table_test_fixture.rs b/src/query/service/src/test_kits/table_test_fixture.rs index 58855f099581..d0ef7579c8b1 100644 --- a/src/query/service/src/test_kits/table_test_fixture.rs +++ b/src/query/service/src/test_kits/table_test_fixture.rs @@ -63,11 +63,11 @@ use tempfile::TempDir; use uuid::Uuid; use walkdir::WalkDir; -use crate::interpreters::fill_missing_columns; use crate::interpreters::CreateTableInterpreter; use crate::interpreters::DeleteInterpreter; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterFactory; +use crate::pipelines::builders::build_fill_missing_columns_pipeline; use crate::pipelines::executor::ExecutorSettings; use crate::pipelines::executor::PipelineCompleteExecutor; use crate::pipelines::PipelineBuildResult; @@ -532,11 +532,11 @@ impl TestFixture { )?; let data_schema: DataSchemaRef = Arc::new(source_schema.into()); - fill_missing_columns( + build_fill_missing_columns_pipeline( self.ctx.clone(), + &mut build_res.main_pipeline, table.clone(), data_schema, - &mut build_res.main_pipeline, )?; table.append_data( diff --git a/src/query/service/tests/it/sql/planner/optimizer/agg_index_query_rewrite.rs b/src/query/service/tests/it/sql/planner/optimizer/agg_index_query_rewrite.rs index 9bb88123e3cf..87af1ec5d9fa 100644 --- a/src/query/service/tests/it/sql/planner/optimizer/agg_index_query_rewrite.rs +++ b/src/query/service/tests/it/sql/planner/optimizer/agg_index_query_rewrite.rs @@ -44,6 +44,7 @@ use databend_query::interpreters::Interpreter; use databend_query::test_kits::TestFixture; use parking_lot::RwLock; use storages_common_table_meta::table::OPT_KEY_DATABASE_ID; +use storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; #[derive(Default)] struct TestSuite { @@ -56,7 +57,7 @@ struct TestSuite { rewritten_predicates: Vec<&'static str>, } -fn create_table_plan(fixture: &TestFixture) -> CreateTablePlan { +fn create_table_plan(fixture: &TestFixture, format: &str) -> CreateTablePlan { CreateTablePlan { if_not_exists: false, tenant: fixture.default_tenant(), @@ -74,6 +75,7 @@ fn create_table_plan(fixture: &TestFixture) -> CreateTablePlan { options: [ // database id is required for FUSE (OPT_KEY_DATABASE_ID.to_owned(), "1".to_owned()), + (OPT_KEY_STORAGE_FORMAT.to_owned(), format.to_owned()), ] .into(), field_comments: vec![], @@ -374,9 +376,14 @@ fn get_test_suites() -> Vec { #[tokio::test(flavor = "multi_thread")] async fn test_query_rewrite() -> Result<()> { + test_query_rewrite_impl("parquet").await?; + test_query_rewrite_impl("native").await +} + +async fn test_query_rewrite_impl(format: &str) -> Result<()> { let fixture = TestFixture::new().await; let ctx = fixture.ctx(); - let create_table_plan = create_table_plan(&fixture); + let create_table_plan = create_table_plan(&fixture, format); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; interpreter.execute(ctx.clone()).await?; diff --git a/src/query/service/tests/it/storages/testdata/columns_table.txt b/src/query/service/tests/it/storages/testdata/columns_table.txt index f2678c31734d..1821b8828f76 100644 --- a/src/query/service/tests/it/storages/testdata/columns_table.txt +++ b/src/query/service/tests/it/storages/testdata/columns_table.txt @@ -247,6 +247,7 @@ DB.Table: 'system'.'columns', Table: columns-table_id:1, ver:0, Engine: SystemCo | 'sql_user' | 'system' | 'query_log' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | | 'sql_user_privileges' | 'system' | 'query_log' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | | 'sql_user_quota' | 'system' | 'query_log' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | +| 'stack' | 'system' | 'backtrace' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | | 'stack_trace' | 'system' | 'query_log' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | | 'stage_params' | 'system' | 'stages' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | | 'stage_type' | 'system' | 'stages' | 'String' | 'VARCHAR' | '' | '' | 'NO' | '' | diff --git a/src/query/sql/src/executor/physical_join/hash_join.rs b/src/query/sql/src/executor/physical_join/hash_join.rs index dcde7ecef2f5..3f65f3d3d208 100644 --- a/src/query/sql/src/executor/physical_join/hash_join.rs +++ b/src/query/sql/src/executor/physical_join/hash_join.rs @@ -104,7 +104,7 @@ impl PhysicalPlanBuilder { }; let probe_schema = match join.join_type { - JoinType::Right | JoinType::Full => { + JoinType::Right | JoinType::RightSingle | JoinType::Full => { let probe_schema = probe_side.output_schema()?; // Wrap nullable type for columns in probe side. let probe_schema = DataSchemaRefExt::create( diff --git a/src/query/sql/src/executor/physical_plan.rs b/src/query/sql/src/executor/physical_plan.rs index 04f843556005..7bfd42f7f948 100644 --- a/src/query/sql/src/executor/physical_plan.rs +++ b/src/query/sql/src/executor/physical_plan.rs @@ -500,7 +500,7 @@ impl HashJoin { pub fn output_schema(&self) -> Result { let mut fields = self.probe.output_schema()?.fields().clone(); match self.join_type { - JoinType::Left | JoinType::Single => { + JoinType::Left | JoinType::LeftSingle => { for field in self.build.output_schema()?.fields() { fields.push(DataField::new( field.name().as_str(), @@ -508,7 +508,7 @@ impl HashJoin { )); } } - JoinType::Right => { + JoinType::Right | JoinType::RightSingle => { fields.clear(); for field in self.probe.output_schema()?.fields() { fields.push(DataField::new( diff --git a/src/query/sql/src/planner/binder/binder.rs b/src/query/sql/src/planner/binder/binder.rs index fd098c2ebc9e..53489a101c9a 100644 --- a/src/query/sql/src/planner/binder/binder.rs +++ b/src/query/sql/src/planner/binder/binder.rs @@ -249,6 +249,7 @@ impl<'a> Binder { Statement::ShowDropTables(stmt) => { self.bind_show_drop_tables(bind_context, stmt).await? } + Statement::AttachTable(stmt) => self.bind_attach_table(stmt).await?, Statement::CreateTable(stmt) => self.bind_create_table(stmt).await?, Statement::DropTable(stmt) => self.bind_drop_table(stmt).await?, Statement::UndropTable(stmt) => self.bind_undrop_table(stmt).await?, @@ -257,6 +258,7 @@ impl<'a> Binder { Statement::TruncateTable(stmt) => self.bind_truncate_table(stmt).await?, Statement::OptimizeTable(stmt) => self.bind_optimize_table(bind_context, stmt).await?, Statement::VacuumTable(stmt) => self.bind_vacuum_table(bind_context, stmt).await?, + Statement::VacuumDropTable(stmt) => self.bind_vacuum_drop_table(bind_context, stmt).await?, Statement::AnalyzeTable(stmt) => self.bind_analyze_table(stmt).await?, Statement::ExistsTable(stmt) => self.bind_exists_table(stmt).await?, diff --git a/src/query/sql/src/planner/binder/ddl/table.rs b/src/query/sql/src/planner/binder/ddl/table.rs index 19e4eba72be7..b2951962b776 100644 --- a/src/query/sql/src/planner/binder/ddl/table.rs +++ b/src/query/sql/src/planner/binder/ddl/table.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use common_ast::ast::AlterTableAction; use common_ast::ast::AlterTableStmt; use common_ast::ast::AnalyzeTableStmt; +use common_ast::ast::AttachTableStmt; use common_ast::ast::ColumnDefinition; use common_ast::ast::ColumnExpr; use common_ast::ast::CompactTarget; @@ -44,6 +45,7 @@ use common_ast::ast::TableReference; use common_ast::ast::TruncateTableStmt; use common_ast::ast::UndropTableStmt; use common_ast::ast::UriLocation; +use common_ast::ast::VacuumDropTableStmt; use common_ast::ast::VacuumTableStmt; use common_ast::parser::parse_sql; use common_ast::parser::tokenize_sql; @@ -70,6 +72,7 @@ use common_storages_view::view_table::VIEW_ENGINE; use storages_common_table_meta::table::is_reserved_opt_key; use storages_common_table_meta::table::OPT_KEY_DATABASE_ID; use storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; +use storages_common_table_meta::table::OPT_KEY_STORAGE_PREFIX; use storages_common_table_meta::table::OPT_KEY_TABLE_COMPRESSION; use tracing::debug; use tracing::error; @@ -108,6 +111,7 @@ use crate::plans::SetOptionsPlan; use crate::plans::ShowCreateTablePlan; use crate::plans::TruncateTablePlan; use crate::plans::UndropTablePlan; +use crate::plans::VacuumDropTablePlan; use crate::plans::VacuumTableOption; use crate::plans::VacuumTablePlan; use crate::BindContext; @@ -576,6 +580,67 @@ impl Binder { Ok(Plan::CreateTable(Box::new(plan))) } + #[async_backtrace::framed] + pub(in crate::planner::binder) async fn bind_attach_table( + &mut self, + stmt: &AttachTableStmt, + ) -> Result { + let (catalog, database, table) = + self.normalize_object_identifier_triple(&stmt.catalog, &stmt.database, &stmt.table); + + let mut path = stmt.uri_location.path.clone(); + // First, to make it easy for users to use, path = "/testbucket/admin/data/1/2" and path = "/testbucket/admin/data/1/2/" are both legal + // So we need to remove the last "/" + if path.ends_with('/') { + path.pop(); + } + // Then, split the path into two parts, the first part is the root, and the second part is the storage_prefix + // For example, path = "/testbucket/admin/data/1/2", then root = "/testbucket/admin/data/", storage_prefix = "1/2" + // root is used by OpenDAL operator, storage_prefix is used to specify the storage location of the table + // Note that the root must end with "/", and the storage_prefix must not start or end with "/" + let mut parts = path.split('/').collect::>(); + if parts.len() < 2 { + return Err(ErrorCode::BadArguments(format!( + "Invalid path: {}", + stmt.uri_location + ))); + } + let storage_prefix = parts.split_off(parts.len() - 2).join("/"); + let root = format!("{}/", parts.join("/")); + let mut options = BTreeMap::new(); + options.insert(OPT_KEY_STORAGE_PREFIX.to_string(), storage_prefix); + + let mut uri = stmt.uri_location.clone(); + uri.path = root; + let (sp, _) = parse_uri_location(&mut uri)?; + + // create a temporary op to check if params is correct + DataOperator::try_create(&sp).await?; + + // Path ends with "/" means it's a directory. + let part_prefix = if uri.path.ends_with('/') { + uri.part_prefix.clone() + } else { + "".to_string() + }; + + Ok(Plan::CreateTable(Box::new(CreateTablePlan { + if_not_exists: false, + tenant: self.ctx.get_tenant(), + catalog, + database, + table, + options, + engine: Engine::Fuse, + cluster_key: None, + as_select: None, + schema: Arc::new(TableSchema::default()), + field_comments: vec![], + storage_params: Some(sp), + part_prefix, + }))) + } + #[async_backtrace::framed] pub(in crate::planner::binder) async fn bind_drop_table( &mut self, @@ -946,6 +1011,51 @@ impl Binder { }))) } + #[async_backtrace::framed] + pub(in crate::planner::binder) async fn bind_vacuum_drop_table( + &mut self, + _bind_context: &mut BindContext, + stmt: &VacuumDropTableStmt, + ) -> Result { + let VacuumDropTableStmt { + catalog, + database, + option, + } = stmt; + + let catalog = catalog + .as_ref() + .map(|ident| normalize_identifier(ident, &self.name_resolution_ctx).name) + .unwrap_or_else(|| self.ctx.get_current_catalog()); + let database = database + .as_ref() + .map(|ident| normalize_identifier(ident, &self.name_resolution_ctx).name) + .unwrap_or_else(|| "".to_string()); + + let option = { + let retain_hours = match option.retain_hours { + Some(Expr::Literal { + lit: Literal::UInt64(uint), + .. + }) => Some(uint as usize), + Some(_) => { + return Err(ErrorCode::IllegalDataType("Unsupported hour type")); + } + _ => None, + }; + + VacuumTableOption { + retain_hours, + dry_run: option.dry_run, + } + }; + Ok(Plan::VacuumDropTable(Box::new(VacuumDropTablePlan { + catalog, + database, + option, + }))) + } + #[async_backtrace::framed] pub(in crate::planner::binder) async fn bind_analyze_table( &mut self, diff --git a/src/query/sql/src/planner/format/display_plan.rs b/src/query/sql/src/planner/format/display_plan.rs index ba73a21425df..3f2164f5fdfa 100644 --- a/src/query/sql/src/planner/format/display_plan.rs +++ b/src/query/sql/src/planner/format/display_plan.rs @@ -92,6 +92,7 @@ impl Plan { Plan::TruncateTable(truncate_table) => Ok(format!("{:?}", truncate_table)), Plan::OptimizeTable(optimize_table) => Ok(format!("{:?}", optimize_table)), Plan::VacuumTable(vacuum_table) => Ok(format!("{:?}", vacuum_table)), + Plan::VacuumDropTable(vacuum_drop_table) => Ok(format!("{:?}", vacuum_drop_table)), Plan::AnalyzeTable(analyze_table) => Ok(format!("{:?}", analyze_table)), Plan::ExistsTable(exists_table) => Ok(format!("{:?}", exists_table)), diff --git a/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs b/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs index 6166746de2d9..6c3b0a0fbb6f 100644 --- a/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs +++ b/src/query/sql/src/planner/optimizer/heuristic/decorrelate.rs @@ -263,7 +263,7 @@ impl SubqueryRewriter { left_conditions, right_conditions, non_equi_conditions: vec![], - join_type: JoinType::Single, + join_type: JoinType::LeftSingle, marker_index: None, from_correlated_subquery: true, contain_runtime_filter: false, diff --git a/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs b/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs index a1151cb2f5d9..e9c550986d3a 100644 --- a/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs +++ b/src/query/sql/src/planner/optimizer/heuristic/subquery_rewriter.rs @@ -339,7 +339,7 @@ impl SubqueryRewriter { left_conditions: vec![], right_conditions: vec![], non_equi_conditions: vec![], - join_type: JoinType::Single, + join_type: JoinType::LeftSingle, marker_index: None, from_correlated_subquery: false, contain_runtime_filter: false, diff --git a/src/query/sql/src/planner/optimizer/rule/rewrite/agg_index/query_rewrite.rs b/src/query/sql/src/planner/optimizer/rule/rewrite/agg_index/query_rewrite.rs index 1b30e513bee4..2aaff06f0b19 100644 --- a/src/query/sql/src/planner/optimizer/rule/rewrite/agg_index/query_rewrite.rs +++ b/src/query/sql/src/planner/optimizer/rule/rewrite/agg_index/query_rewrite.rs @@ -712,6 +712,10 @@ fn collect_information_impl<'a>( collect_information_impl(s_expr.child(0)?, info) } RelOperator::Scan(scan) => { + if let Some(prewhere) = &scan.prewhere { + debug_assert!(info.predicates.is_none()); + info.predicates.replace(&prewhere.predicates); + } info.table_index = scan.table_index; // Finish the recursion. Ok(()) diff --git a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs index 6a13aeb6ac5e..89e8440b027d 100644 --- a/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs +++ b/src/query/sql/src/planner/optimizer/rule/rewrite/rule_commute_join.rs @@ -77,6 +77,8 @@ impl Rule for RuleCommuteJoin { | JoinType::Cross | JoinType::Left | JoinType::Right + | JoinType::LeftSingle + | JoinType::RightSingle | JoinType::LeftSemi | JoinType::RightSemi | JoinType::LeftAnti @@ -86,7 +88,7 @@ impl Rule for RuleCommuteJoin { } else if left_card == right_card { matches!( join.join_type, - JoinType::Right | JoinType::RightSemi | JoinType::RightAnti + JoinType::Right | JoinType::RightSingle | JoinType::RightSemi | JoinType::RightAnti ) } else { false diff --git a/src/query/sql/src/planner/optimizer/rule/transform/rule_commute_join_base_table.rs b/src/query/sql/src/planner/optimizer/rule/transform/rule_commute_join_base_table.rs index 2469748cec9e..2fdc23d3ef79 100644 --- a/src/query/sql/src/planner/optimizer/rule/transform/rule_commute_join_base_table.rs +++ b/src/query/sql/src/planner/optimizer/rule/transform/rule_commute_join_base_table.rs @@ -75,11 +75,13 @@ impl Rule for RuleCommuteJoinBaseTable { | JoinType::Cross | JoinType::Left | JoinType::Right + | JoinType::LeftSingle + | JoinType::RightSingle | JoinType::LeftSemi | JoinType::RightSemi | JoinType::LeftAnti - | JoinType::LeftMark - | JoinType::RightAnti => { + | JoinType::RightAnti + | JoinType::LeftMark => { // Swap the join conditions side (join.left_conditions, join.right_conditions) = (join.right_conditions, join.left_conditions); diff --git a/src/query/sql/src/planner/plans/ddl/table.rs b/src/query/sql/src/planner/plans/ddl/table.rs index ba1246dddcd6..8b94c633dc8d 100644 --- a/src/query/sql/src/planner/plans/ddl/table.rs +++ b/src/query/sql/src/planner/plans/ddl/table.rs @@ -105,7 +105,35 @@ pub struct VacuumTablePlan { impl VacuumTablePlan { pub fn schema(&self) -> DataSchemaRef { - Arc::new(DataSchema::empty()) + if self.option.dry_run.is_some() { + Arc::new(DataSchema::new(vec![DataField::new( + "Files", + DataType::String, + )])) + } else { + Arc::new(DataSchema::empty()) + } + } +} + +/// Vacuum drop table +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct VacuumDropTablePlan { + pub catalog: String, + pub database: String, + pub option: VacuumTableOption, +} + +impl VacuumDropTablePlan { + pub fn schema(&self) -> DataSchemaRef { + if self.option.dry_run.is_some() { + Arc::new(DataSchema::new(vec![ + DataField::new("Table", DataType::String), + DataField::new("File", DataType::String), + ])) + } else { + Arc::new(DataSchema::empty()) + } } } diff --git a/src/query/sql/src/planner/plans/join.rs b/src/query/sql/src/planner/plans/join.rs index 39e953cd6fd9..72f0645c4971 100644 --- a/src/query/sql/src/planner/plans/join.rs +++ b/src/query/sql/src/planner/plans/join.rs @@ -59,7 +59,8 @@ pub enum JoinType { /// Right Mark Join use subquery as build side, it's executed by streaming. RightMark, /// Single Join is a special kind of join that is used to process correlated scalar subquery. - Single, + LeftSingle, + RightSingle, } impl JoinType { @@ -67,6 +68,8 @@ impl JoinType { match self { JoinType::Left => JoinType::Right, JoinType::Right => JoinType::Left, + JoinType::LeftSingle => JoinType::RightSingle, + JoinType::RightSingle => JoinType::LeftSingle, JoinType::LeftSemi => JoinType::RightSemi, JoinType::RightSemi => JoinType::LeftSemi, JoinType::LeftAnti => JoinType::RightAnti, @@ -122,8 +125,11 @@ impl Display for JoinType { JoinType::RightMark => { write!(f, "RIGHT MARK") } - JoinType::Single => { - write!(f, "SINGLE") + JoinType::LeftSingle => { + write!(f, "LEFT SINGLE") + } + JoinType::RightSingle => { + write!(f, "RIGHT SINGLE") } } } @@ -437,10 +443,13 @@ impl Operator for Join { + f64::max(right_cardinality, inner_join_cardinality) - inner_join_cardinality } - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark | JoinType::Single => { + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark | JoinType::LeftSingle => { left_cardinality } - JoinType::RightSemi | JoinType::RightAnti | JoinType::RightMark => right_cardinality, + JoinType::RightSemi + | JoinType::RightAnti + | JoinType::RightMark + | JoinType::RightSingle => right_cardinality, }; // Derive column statistics let column_stats = if cardinality == 0.0 { diff --git a/src/query/sql/src/planner/plans/plan.rs b/src/query/sql/src/planner/plans/plan.rs index 2d90a3ad6337..c6a5280247a6 100644 --- a/src/query/sql/src/planner/plans/plan.rs +++ b/src/query/sql/src/planner/plans/plan.rs @@ -34,6 +34,7 @@ use super::DropShareEndpointPlan; use super::ModifyTableColumnPlan; use super::RenameTableColumnPlan; use super::SetOptionsPlan; +use super::VacuumDropTablePlan; use super::VacuumTablePlan; use crate::optimizer::SExpr; use crate::plans::copy::CopyPlan; @@ -177,6 +178,7 @@ pub enum Plan { TruncateTable(Box), OptimizeTable(Box), VacuumTable(Box), + VacuumDropTable(Box), AnalyzeTable(Box), ExistsTable(Box), SetOptions(Box), @@ -324,6 +326,7 @@ impl Display for Plan { Plan::TruncateTable(_) => write!(f, "TruncateTable"), Plan::OptimizeTable(_) => write!(f, "OptimizeTable"), Plan::VacuumTable(_) => write!(f, "VacuumTable"), + Plan::VacuumDropTable(_) => write!(f, "VacuumDropTable"), Plan::AnalyzeTable(_) => write!(f, "AnalyzeTable"), Plan::ExistsTable(_) => write!(f, "ExistsTable"), Plan::CreateView(_) => write!(f, "CreateView"), @@ -419,6 +422,7 @@ impl Plan { Plan::ShowCreateTable(plan) => plan.schema(), Plan::DescribeTable(plan) => plan.schema(), Plan::VacuumTable(plan) => plan.schema(), + Plan::VacuumDropTable(plan) => plan.schema(), Plan::ExistsTable(plan) => plan.schema(), Plan::ShowRoles(plan) => plan.schema(), Plan::ShowGrants(plan) => plan.schema(), @@ -465,6 +469,7 @@ impl Plan { | Plan::ShowGrants(_) | Plan::Presign(_) | Plan::VacuumTable(_) + | Plan::VacuumDropTable(_) | Plan::DescDatamaskPolicy(_) ) } diff --git a/src/query/storages/common/table-meta/src/table/table_keys.rs b/src/query/storages/common/table-meta/src/table/table_keys.rs index a3b8f7c9faff..b35bdaf8cbdc 100644 --- a/src/query/storages/common/table-meta/src/table/table_keys.rs +++ b/src/query/storages/common/table-meta/src/table/table_keys.rs @@ -16,6 +16,7 @@ use std::collections::HashSet; use once_cell::sync::Lazy; pub const OPT_KEY_DATABASE_ID: &str = "database_id"; +pub const OPT_KEY_STORAGE_PREFIX: &str = "storage_prefix"; pub const OPT_KEY_SNAPSHOT_LOCATION: &str = "snapshot_location"; pub const OPT_KEY_STORAGE_FORMAT: &str = "storage_format"; pub const OPT_KEY_TABLE_COMPRESSION: &str = "compression"; diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index df40f57a04b9..de4f408025b1 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -64,6 +64,7 @@ use storages_common_table_meta::table::OPT_KEY_DATABASE_ID; use storages_common_table_meta::table::OPT_KEY_LEGACY_SNAPSHOT_LOC; use storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; use storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; +use storages_common_table_meta::table::OPT_KEY_STORAGE_PREFIX; use storages_common_table_meta::table::OPT_KEY_TABLE_COMPRESSION; use tracing::error; use tracing::warn; @@ -200,6 +201,13 @@ impl FuseTable { } pub fn parse_storage_prefix(table_info: &TableInfo) -> Result { + // if OPT_KE_STORAGE_PREFIX is specified, use it as storage prefix + if let Some(prefix) = table_info.options().get(OPT_KEY_STORAGE_PREFIX) { + return Ok(prefix.clone()); + } + + // otherwise, use database id and table id as storage prefix + let table_id = table_info.ident.table_id; let db_id = table_info .options() diff --git a/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader.rs b/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader.rs index 96338df8df46..8d909c9b5907 100644 --- a/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader.rs +++ b/src/query/storages/fuse/src/io/read/agg_index/agg_index_reader.rs @@ -12,12 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::io::BufReader; use std::io::Cursor; +use std::io::Seek; use std::sync::Arc; -use common_arrow::arrow::io::parquet::read::infer_schema; -use common_arrow::arrow::io::parquet::read::read_metadata; -use common_arrow::arrow::io::parquet::read::FileReader; +use common_arrow::arrow::chunk::Chunk; +use common_arrow::arrow::io::parquet::read as pread; +use common_arrow::arrow::io::parquet::write::to_parquet_schema; +use common_arrow::native::read as nread; use common_catalog::plan::AggIndexInfo; use common_catalog::plan::AggIndexMeta; use common_catalog::table_context::TableContext; @@ -36,12 +39,15 @@ use common_functions::BUILTIN_FUNCTIONS; use opendal::Operator; use tracing::debug; +use crate::FuseStorageFormat; + #[derive(Clone)] pub struct AggIndexReader { index_id: u64, // TODO: use `BlockReader` to support partitially reading. dal: Operator, + storage_format: FuseStorageFormat, func_ctx: FunctionContext, schema: DataSchema, @@ -59,6 +65,7 @@ impl AggIndexReader { ctx: Arc, dal: Operator, agg: &AggIndexInfo, + storage_format: FuseStorageFormat, ) -> Result { let func_ctx = ctx.get_function_context()?; let selection = agg @@ -77,6 +84,7 @@ impl AggIndexReader { filter, actual_table_field_len: agg.actual_table_field_len, is_agg: agg.is_agg, + storage_format, }) } @@ -114,15 +122,11 @@ impl AggIndexReader { } pub fn deserialize(&self, data: &[u8]) -> Result { - let mut reader = Cursor::new(data); - let metadata = read_metadata(&mut reader)?; - let schema = infer_schema(&metadata)?; - let reader = FileReader::new(reader, metadata.row_groups, schema, None, None, None); - let chunks = reader.collect::>>()?; - debug_assert_eq!(chunks.len(), 1); - // 1. Deserialize to `DataBlock` - let block = DataBlock::from_arrow_chunk(&chunks[0], &self.schema)?; + let block = match self.storage_format { + FuseStorageFormat::Parquet => self.deserialize_parquet(data)?, + FuseStorageFormat::Native => self.deserialize_native(data)?, + }; let evaluator = Evaluator::new(&block, &self.func_ctx, &BUILTIN_FUNCTIONS); // 2. Filter the block if there is a filter. @@ -164,4 +168,50 @@ impl AggIndexReader { Some(AggIndexMeta::create(self.is_agg)), )) } + + fn deserialize_parquet(&self, data: &[u8]) -> Result { + let mut reader = Cursor::new(data); + let metadata = pread::read_metadata(&mut reader)?; + let schema = pread::infer_schema(&metadata)?; + let reader = pread::FileReader::new(reader, metadata.row_groups, schema, None, None, None); + let chunks = reader.collect::>>()?; + debug_assert_eq!(chunks.len(), 1); + DataBlock::from_arrow_chunk(&chunks[0], &self.schema) + } + + fn deserialize_native(&self, data: &[u8]) -> Result { + let mut reader = Cursor::new(data); + let schema = nread::reader::infer_schema(&mut reader)?; + let mut metas = nread::reader::read_meta(&mut reader)?; + let schema_descriptor = to_parquet_schema(&schema)?; + let mut leaves = schema_descriptor.columns().to_vec(); + + let mut arrays = vec![]; + for field in schema.fields.iter() { + let n = pread::n_columns(&field.data_type); + let curr_metas = metas.drain(..n).collect::>(); + let curr_leaves = leaves.drain(..n).collect::>(); + let mut pages = Vec::with_capacity(n); + let mut readers = Vec::with_capacity(n); + for curr_meta in curr_metas.iter() { + pages.push(curr_meta.pages.clone()); + let mut reader = Cursor::new(data); + reader.seek(std::io::SeekFrom::Start(curr_meta.offset))?; + let buffer_size = curr_meta.total_len() as usize; + let reader = BufReader::with_capacity(buffer_size, reader); + readers.push(reader); + } + let is_nested = !nread::reader::is_primitive(field.data_type()); + let array = nread::batch_read::batch_read_array( + readers, + curr_leaves, + field.clone(), + is_nested, + pages, + )?; + arrays.push(array); + } + let chunk = Chunk::new(arrays); + DataBlock::from_arrow_chunk(&chunk, &self.schema) + } } diff --git a/src/query/storages/fuse/src/lib.rs b/src/query/storages/fuse/src/lib.rs index ba8f3f79a3d1..45e04d37aa5f 100644 --- a/src/query/storages/fuse/src/lib.rs +++ b/src/query/storages/fuse/src/lib.rs @@ -42,8 +42,6 @@ pub use fuse_table::FuseTable; pub use io::MergeIOReadResult; pub use pruning::SegmentLocation; -pub use crate::metrics::metrics_reset; - mod sessions { pub use common_catalog::table_context::TableContext; } diff --git a/src/query/storages/fuse/src/metrics/fuse_metrics.rs b/src/query/storages/fuse/src/metrics/fuse_metrics.rs index 35712f9604c2..f29b7f020f72 100644 --- a/src/query/storages/fuse/src/metrics/fuse_metrics.rs +++ b/src/query/storages/fuse/src/metrics/fuse_metrics.rs @@ -13,7 +13,6 @@ // limitations under the License. use metrics::counter; -use metrics::gauge; use metrics::increment_gauge; macro_rules! key { @@ -186,54 +185,3 @@ pub fn metrics_inc_deletion_block_range_pruned_whole_block_nums(c: u64) { c as f64 ); } - -pub fn metrics_reset() { - let c = 0 as f64; - - // IO metrics. - gauge!(key!("remote_io_seeks"), c); - gauge!(key!("remote_io_seeks_after_merged"), c); - gauge!(key!("remote_io_read_bytes"), c); - gauge!(key!("remote_io_read_bytes_after_merged"), c); - gauge!(key!("remote_io_read_parts"), c); - gauge!(key!("remote_io_read_milliseconds"), c); - gauge!(key!("remote_io_deserialize_milliseconds"), c); - - // Block metrics. - gauge!(key!("block_write_nums"), c); - gauge!(key!("block_write_bytes"), c); - gauge!(key!("block_write_milliseconds"), c); - gauge!(key!("block_index_write_nums"), c); - gauge!(key!("block_index_write_bytes"), c); - gauge!(key!("block_index_write_milliseconds"), c); - gauge!(key!("block_index_read_nums"), c); - gauge!(key!("block_index_read_bytes"), c); - gauge!(key!("block_index_read_milliseconds"), c); - - // Compact metrics. - gauge!(key!("compact_block_read_nums"), c); - gauge!(key!("compact_block_read_bytes"), c); - gauge!(key!("compact_block_read_milliseconds"), c); - - // Pruning metrics. - gauge!(key!("pruning_prewhere_nums"), c); - gauge!(key!("pruning_milliseconds"), c); - - gauge!(key!("segments_range_pruning_before"), c); - gauge!(key!("segments_range_pruning_after"), c); - gauge!(key!("blocks_range_pruning_before"), c); - gauge!(key!("blocks_range_pruning_after"), c); - gauge!(key!("blocks_bloom_pruning_before"), c); - gauge!(key!("blocks_bloom_pruning_after"), c); - gauge!(key!("bytes_segment_range_pruning_before"), c); - gauge!(key!("bytes_segment_range_pruning_after"), c); - gauge!(key!("bytes_block_bloom_pruning_before"), c); - gauge!(key!("bytes_block_bloom_pruning_after"), c); - gauge!(key!("bytes_block_range_pruning_before"), c); - gauge!(key!("bytes_block_range_pruning_after"), c); - gauge!(key!("deletion_block_range_pruned_nums"), c); - gauge!(key!("deletion_block_range_pruned_whole_block_nums"), c); - - // segment metrics - gauge!(key!("deletion_segment_range_pruned_whole_segment_nums"), c); -} diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 1f67eea15c01..d969a9895005 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -275,8 +275,9 @@ impl FuseTable { Ok(v) => v, }; + // root snapshot cannot ignore storage not find error. let referenced_locations = self - .get_block_locations(ctx.clone(), &root_snapshot.segments, put_cache) + .get_block_locations(ctx.clone(), &root_snapshot.segments, put_cache, false) .await?; let snapshot_lite = Arc::new(SnapshotLiteExtended { format_version: ver, @@ -305,7 +306,10 @@ impl FuseTable { // Purge segments&blocks by chunk size let segment_locations = Vec::from_iter(segments_to_be_purged); for chunk in segment_locations.chunks(chunk_size) { - let locations = self.get_block_locations(ctx.clone(), chunk, false).await?; + // since we are purging files, the ErrorCode::STORAGE_NOT_FOUND error can be safely ignored. + let locations = self + .get_block_locations(ctx.clone(), chunk, false, true) + .await?; for loc in &locations.block_location { if locations_referenced_by_root.block_location.contains(loc) { @@ -343,7 +347,10 @@ impl FuseTable { let mut count = 0; let segment_locations = Vec::from_iter(segments_to_be_purged); for chunk in segment_locations.chunks(chunk_size) { - let locations = self.get_block_locations(ctx.clone(), chunk, false).await?; + // since we are purging files, the ErrorCode::STORAGE_NOT_FOUND error can be safely ignored. + let locations = self + .get_block_locations(ctx.clone(), chunk, false, true) + .await?; let mut blocks_to_be_purged = HashSet::new(); for loc in &locations.block_location { @@ -557,30 +564,34 @@ impl FuseTable { ctx: Arc, segment_locations: &[Location], put_cache: bool, + ignore_err: bool, ) -> Result { let mut blocks = HashSet::new(); let mut blooms = HashSet::new(); let fuse_segments = SegmentsIO::create(ctx.clone(), self.operator.clone(), self.schema()); - let results = fuse_segments - .read_segments::(segment_locations, put_cache) - .await?; - for (idx, location_tuple) in results.into_iter().enumerate() { - let location_tuple = match location_tuple { - Err(e) if e.code() == ErrorCode::STORAGE_NOT_FOUND => { - let location = &segment_locations[idx]; - // concurrent gc: someone else has already collected this segment, ignore it - warn!( - "concurrent gc: segment of location {} already collected. table: {}, ident {}", - location.0, self.table_info.desc, self.table_info.ident, - ); - continue; - } - Err(e) => return Err(e), - Ok(v) => v, - }; - blocks.extend(location_tuple.block_location.into_iter()); - blooms.extend(location_tuple.bloom_location.into_iter()); + let chunk_size = ctx.get_settings().get_max_storage_io_requests()? as usize; + for chunk in segment_locations.chunks(chunk_size) { + let results = fuse_segments + .read_segments::(chunk, put_cache) + .await?; + for (idx, location_tuple) in results.into_iter().enumerate() { + let location_tuple = match location_tuple { + Err(e) if e.code() == ErrorCode::STORAGE_NOT_FOUND && ignore_err => { + let location = &segment_locations[idx]; + // concurrent gc: someone else has already collected this segment, ignore it + warn!( + "concurrent gc: segment of location {} already collected. table: {}, ident {}", + location.0, self.table_info.desc, self.table_info.ident, + ); + continue; + } + Err(e) => return Err(e), + Ok(v) => v, + }; + blocks.extend(location_tuple.block_location.into_iter()); + blooms.extend(location_tuple.bloom_location.into_iter()); + } } Ok(LocationTuple { diff --git a/src/query/storages/fuse/src/operations/read/fuse_source.rs b/src/query/storages/fuse/src/operations/read/fuse_source.rs index 3a786ee85eb1..91b8daf3f3e4 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_source.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_source.rs @@ -37,6 +37,7 @@ use crate::operations::read::native_data_source_reader::ReadNativeDataSource; use crate::operations::read::parquet_data_source_deserializer::DeserializeDataTransform; use crate::operations::read::parquet_data_source_reader::ReadParquetDataSource; +#[allow(clippy::too_many_arguments)] pub fn build_fuse_native_source_pipeline( ctx: Arc, pipeline: &mut Pipeline, @@ -45,6 +46,7 @@ pub fn build_fuse_native_source_pipeline( plan: &DataSourcePlan, topk: Option, mut max_io_requests: usize, + index_reader: Arc>, ) -> Result<()> { (max_threads, max_io_requests) = adjust_threads_and_request(true, max_threads, max_io_requests, plan); @@ -75,6 +77,7 @@ pub fn build_fuse_native_source_pipeline( output, block_reader.clone(), partitions.clone(), + index_reader.clone(), )?, ); } @@ -98,6 +101,7 @@ pub fn build_fuse_native_source_pipeline( output, block_reader.clone(), partitions.clone(), + index_reader.clone(), )?, ); } @@ -114,6 +118,7 @@ pub fn build_fuse_native_source_pipeline( topk.clone(), transform_input, transform_output, + index_reader.clone(), ) })?; diff --git a/src/query/storages/fuse/src/operations/read/mod.rs b/src/query/storages/fuse/src/operations/read/mod.rs index 593ec5289bdb..dc3b58606b4e 100644 --- a/src/query/storages/fuse/src/operations/read/mod.rs +++ b/src/query/storages/fuse/src/operations/read/mod.rs @@ -25,5 +25,3 @@ mod parquet_rows_fetcher; pub use fuse_rows_fetcher::build_row_fetcher_pipeline; pub use fuse_source::build_fuse_parquet_source_pipeline; -pub use parquet_data_source::DataSource; -pub use parquet_data_source::DataSourceMeta; diff --git a/src/query/storages/fuse/src/operations/read/native_data_source.rs b/src/query/storages/fuse/src/operations/read/native_data_source.rs index 53077de27cc6..13c9bcf80fca 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source.rs @@ -26,16 +26,21 @@ use serde::Serializer; use crate::io::NativeReaderExt; +pub enum DataSource { + AggIndex(Vec), + Normal(DataChunks), +} + pub type DataChunks = BTreeMap>>>; pub struct NativeDataSourceMeta { pub part: Vec, - pub chunks: Vec, + pub data: Vec, } impl NativeDataSourceMeta { - pub fn create(part: Vec, chunks: Vec) -> BlockMetaInfoPtr { - Box::new(NativeDataSourceMeta { part, chunks }) + pub fn create(part: Vec, data: Vec) -> BlockMetaInfoPtr { + Box::new(NativeDataSourceMeta { part, data }) } } diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs index 16d963394cf0..3ebb9e378dd9 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_deserializer.rs @@ -62,11 +62,12 @@ use common_pipeline_core::processors::Processor; use common_storage::ColumnNode; use super::fuse_source::fill_internal_column_meta; +use super::native_data_source::DataSource; use crate::fuse_part::FusePartInfo; +use crate::io::AggIndexReader; use crate::io::BlockReader; use crate::io::NativeReaderExt; use crate::metrics::metrics_inc_pruning_prewhere_nums; -use crate::operations::read::native_data_source::DataChunks; use crate::operations::read::native_data_source::NativeDataSourceMeta; pub struct NativeDeserializeDataTransform { @@ -79,7 +80,7 @@ pub struct NativeDeserializeDataTransform { output: Arc, output_data: Option, parts: VecDeque, - chunks: VecDeque, + chunks: VecDeque, prewhere_columns: Vec, prewhere_schema: DataSchema, @@ -105,6 +106,8 @@ pub struct NativeDeserializeDataTransform { array_iters: BTreeMap>, // The Page numbers of each ArrayIter can skip. array_skip_pages: BTreeMap, + + index_reader: Arc>, } impl NativeDeserializeDataTransform { @@ -115,6 +118,7 @@ impl NativeDeserializeDataTransform { top_k: Option, input: Arc, output: Arc, + index_reader: Arc>, ) -> Result { let scan_progress = ctx.get_scan_progress(); @@ -230,6 +234,8 @@ impl NativeDeserializeDataTransform { array_iters: BTreeMap::new(), array_skip_pages: BTreeMap::new(), offset_in_part: 0, + + index_reader, }, ))) } @@ -551,7 +557,7 @@ impl Processor for NativeDeserializeDataTransform { if let Some(block_meta) = data_block.take_meta() { if let Some(source_meta) = NativeDataSourceMeta::downcast_from(block_meta) { self.parts = VecDeque::from(source_meta.part); - self.chunks = VecDeque::from(source_meta.chunks); + self.chunks = VecDeque::from(source_meta.data); return Ok(Event::Sync); } } @@ -571,6 +577,16 @@ impl Processor for NativeDeserializeDataTransform { fn process(&mut self) -> Result<()> { if let Some(chunks) = self.chunks.front_mut() { + let chunks = match chunks { + DataSource::AggIndex(data) => { + let agg_index_reader = self.index_reader.as_ref().as_ref().unwrap(); + let block = agg_index_reader.deserialize(data)?; + self.output_data = Some(block); + return self.finish_process(); + } + DataSource::Normal(data) => data, + }; + // this means it's empty projection if chunks.is_empty() && !self.inited { return self.finish_process_with_empty_block(); diff --git a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs index d1e031765fe3..db11dcaa1233 100644 --- a/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/native_data_source_reader.rs @@ -19,6 +19,7 @@ use common_base::base::tokio; use common_catalog::plan::PartInfoPtr; use common_catalog::plan::StealablePartitions; use common_catalog::table_context::TableContext; +use common_exception::ErrorCode; use common_exception::Result; use common_expression::DataBlock; use common_pipeline_core::processors::port::OutputPort; @@ -28,9 +29,12 @@ use common_pipeline_core::processors::Processor; use common_pipeline_sources::SyncSource; use common_pipeline_sources::SyncSourcer; +use super::native_data_source::DataSource; +use crate::io::AggIndexReader; use crate::io::BlockReader; -use crate::operations::read::native_data_source::DataChunks; +use crate::io::TableMetaLocationGenerator; use crate::operations::read::native_data_source::NativeDataSourceMeta; +use crate::FusePartInfo; pub struct ReadNativeDataSource { id: usize, @@ -39,8 +43,10 @@ pub struct ReadNativeDataSource { block_reader: Arc, output: Arc, - output_data: Option<(Vec, Vec)>, + output_data: Option<(Vec, Vec)>, partitions: StealablePartitions, + + index_reader: Arc>, } impl ReadNativeDataSource { @@ -50,6 +56,7 @@ impl ReadNativeDataSource { output: Arc, block_reader: Arc, partitions: StealablePartitions, + index_reader: Arc>, ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; SyncSourcer::create(ctx.clone(), output.clone(), ReadNativeDataSource:: { @@ -60,6 +67,7 @@ impl ReadNativeDataSource { finished: false, output_data: None, partitions, + index_reader, }) } } @@ -71,6 +79,7 @@ impl ReadNativeDataSource { output: Arc, block_reader: Arc, partitions: StealablePartitions, + index_reader: Arc>, ) -> Result { let batch_size = ctx.get_settings().get_storage_fetch_part_num()? as usize; Ok(ProcessorPtr::create(Box::new(ReadNativeDataSource::< @@ -83,6 +92,7 @@ impl ReadNativeDataSource { finished: false, output_data: None, partitions, + index_reader, }))) } } @@ -93,11 +103,30 @@ impl SyncSource for ReadNativeDataSource { fn generate(&mut self) -> Result> { match self.partitions.steal_one(self.id) { None => Ok(None), - Some(part) => Ok(Some(DataBlock::empty_with_meta( - NativeDataSourceMeta::create(vec![part.clone()], vec![ - self.block_reader.sync_read_native_columns_data(part)?, - ]), - ))), + Some(part) => { + if let Some(index_reader) = self.index_reader.as_ref() { + let fuse_part = FusePartInfo::from_part(&part)?; + let loc = + TableMetaLocationGenerator::gen_agg_index_location_from_block_location( + &fuse_part.location, + index_reader.index_id(), + ); + if let Some(data) = index_reader.sync_read_data(&loc) { + // Read from aggregating index. + return Ok(Some(DataBlock::empty_with_meta( + NativeDataSourceMeta::create(vec![part.clone()], vec![ + DataSource::AggIndex(data), + ]), + ))); + } + } + + Ok(Some(DataBlock::empty_with_meta( + NativeDataSourceMeta::create(vec![part.clone()], vec![DataSource::Normal( + self.block_reader.sync_read_native_columns_data(part)?, + )]), + ))) + } } } } @@ -144,10 +173,26 @@ impl Processor for ReadNativeDataSource { for part in &parts { let part = part.clone(); let block_reader = self.block_reader.clone(); + let index_reader = self.index_reader.clone(); chunks.push(async move { let handler = tokio::spawn(async_backtrace::location!().frame(async move { - block_reader.async_read_native_columns_data(part).await + let fuse_part = FusePartInfo::from_part(&part)?; + if let Some(index_reader) = index_reader.as_ref() { + let loc = + TableMetaLocationGenerator::gen_agg_index_location_from_block_location( + &fuse_part.location, + index_reader.index_id(), + ); + if let Some(data) = index_reader.read_data(&loc).await { + // Read from aggregating index. + return Ok::<_, ErrorCode>(DataSource::AggIndex(data)); + } + } + + Ok(DataSource::Normal( + block_reader.async_read_native_columns_data(part).await?, + )) })); handler.await.unwrap() }); diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source.rs index 213c66f22231..560021c7ac0b 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source.rs @@ -31,20 +31,20 @@ pub enum DataSource { } pub struct DataSourceMeta { - pub part: Vec, + pub parts: Vec, pub data: Vec, } impl DataSourceMeta { pub fn create(part: Vec, data: Vec) -> BlockMetaInfoPtr { - Box::new(DataSourceMeta { part, data }) + Box::new(DataSourceMeta { parts: part, data }) } } impl Debug for DataSourceMeta { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { f.debug_struct("DataSourceMeta") - .field("parts", &self.part) + .field("parts", &self.parts) .finish() } } diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs index 29f0bfbc2154..f2de2d210534 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_deserializer.rs @@ -30,7 +30,7 @@ use common_pipeline_core::processors::processor::ProcessorPtr; use common_pipeline_core::processors::Processor; use super::fuse_source::fill_internal_column_meta; -use super::DataSource; +use super::parquet_data_source::DataSource; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; @@ -117,7 +117,7 @@ impl Processor for DeserializeDataTransform { let mut data_block = self.input.pull_data().unwrap()?; if let Some(source_meta) = data_block.take_meta() { if let Some(source_meta) = DataSourceMeta::downcast_from(source_meta) { - self.parts = source_meta.part; + self.parts = source_meta.parts; self.chunks = source_meta.data; return Ok(Event::Sync); } diff --git a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs index 48cb4ae145fa..959258df7264 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_data_source_reader.rs @@ -29,7 +29,7 @@ use common_pipeline_core::processors::Processor; use common_pipeline_sources::SyncSource; use common_pipeline_sources::SyncSourcer; -use super::DataSource; +use super::parquet_data_source::DataSource; use crate::fuse_part::FusePartInfo; use crate::io::AggIndexReader; use crate::io::BlockReader; diff --git a/src/query/storages/fuse/src/operations/read_data.rs b/src/query/storages/fuse/src/operations/read_data.rs index b54b460c389e..fcff61b3eec5 100644 --- a/src/query/storages/fuse/src/operations/read_data.rs +++ b/src/query/storages/fuse/src/operations/read_data.rs @@ -196,7 +196,14 @@ impl FuseTable { plan.push_downs .as_ref() .and_then(|p| p.agg_index.as_ref()) - .map(|agg| AggIndexReader::try_create(ctx.clone(), self.operator.clone(), agg)) + .map(|agg| { + AggIndexReader::try_create( + ctx.clone(), + self.operator.clone(), + agg, + self.storage_format, + ) + }) .transpose()?, ); @@ -239,6 +246,7 @@ impl FuseTable { plan, top_k, max_io_requests, + index_reader, ), FuseStorageFormat::Parquet => build_fuse_parquet_source_pipeline( ctx, diff --git a/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information_table.rs b/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information_table.rs index 39a4816631b4..7c73b50dd901 100644 --- a/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information_table.rs +++ b/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information_table.rs @@ -27,13 +27,13 @@ use common_meta_app::schema::TableInfo; use common_meta_app::schema::TableMeta; use super::clustering_information::ClusteringInformation; -use super::table_args::parse_func_table_args; use crate::pipelines::processors::port::OutputPort; use crate::pipelines::processors::processor::ProcessorPtr; use crate::pipelines::processors::AsyncSource; use crate::pipelines::processors::AsyncSourcer; use crate::pipelines::Pipeline; use crate::sessions::TableContext; +use crate::table_functions::parse_db_tb_args; use crate::table_functions::string_literal; use crate::table_functions::TableArgs; use crate::table_functions::TableFunction; @@ -55,7 +55,8 @@ impl ClusteringInformationTable { table_id: u64, table_args: TableArgs, ) -> Result> { - let (arg_database_name, arg_table_name) = parse_func_table_args(&table_args)?; + let (arg_database_name, arg_table_name) = + parse_db_tb_args(&table_args, FUSE_FUNC_CLUSTERING)?; let engine = FUSE_FUNC_CLUSTERING.to_owned(); diff --git a/src/query/storages/fuse/src/table_functions/clustering_information/mod.rs b/src/query/storages/fuse/src/table_functions/clustering_information/mod.rs index ba2fe2112235..0f1413d1790a 100644 --- a/src/query/storages/fuse/src/table_functions/clustering_information/mod.rs +++ b/src/query/storages/fuse/src/table_functions/clustering_information/mod.rs @@ -15,8 +15,6 @@ #[allow(clippy::module_inception)] mod clustering_information; mod clustering_information_table; -mod table_args; pub use clustering_information::ClusteringInformation; pub use clustering_information_table::ClusteringInformationTable; -pub use table_args::*; diff --git a/src/query/storages/fuse/src/table_functions/clustering_information/table_args.rs b/src/query/storages/fuse/src/table_functions/clustering_information/table_args.rs deleted file mode 100644 index 6fa9634fb349..000000000000 --- a/src/query/storages/fuse/src/table_functions/clustering_information/table_args.rs +++ /dev/null @@ -1,36 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use common_exception::Result; -use common_expression::Expr; - -// use common_sql::ExpressionParser; -use crate::table_functions::string_value; -use crate::table_functions::TableArgs; - -pub fn parse_func_table_args(table_args: &TableArgs) -> Result<(String, String)> { - let args = table_args.expect_all_positioned("clustering_information", Some(2))?; - let db = string_value(&args[0])?; - let tbl = string_value(&args[1])?; - Ok((db, tbl)) -} - -pub fn unwrap_tuple(expr: &Expr) -> Option> { - match expr { - Expr::FunctionCall { function, args, .. } if function.signature.name == "tuple" => { - Some(args.clone()) - } - _ => None, - } -} diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs index 2d47cab25329..6cb4f2ecaa22 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs @@ -67,29 +67,30 @@ impl<'a> FuseBlock<'a> { #[async_backtrace::framed] pub async fn get_blocks(&self) -> Result { let tbl = self.table; + let snapshot_id = self.snapshot_id.clone(); let maybe_snapshot = tbl.read_table_snapshot().await?; if let Some(snapshot) = maybe_snapshot { - if self.snapshot_id.is_none() { - return self.to_block(snapshot).await; - } + if let Some(snapshot_id) = snapshot_id { + // prepare the stream of snapshot + let snapshot_version = tbl.snapshot_format_version(None).await?; + let snapshot_location = tbl + .meta_location_generator + .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; + let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); + let mut snapshot_stream = reader.snapshot_history( + snapshot_location, + snapshot_version, + tbl.meta_location_generator().clone(), + ); - // prepare the stream of snapshot - let snapshot_version = tbl.snapshot_format_version(None).await?; - let snapshot_location = tbl - .meta_location_generator - .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; - let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); - let mut snapshot_stream = reader.snapshot_history( - snapshot_location, - snapshot_version, - tbl.meta_location_generator().clone(), - ); - - // find the element by snapshot_id in stream - while let Some((snapshot, _)) = snapshot_stream.try_next().await? { - if snapshot.snapshot_id.simple().to_string() == self.snapshot_id.clone().unwrap() { - return self.to_block(snapshot).await; + // find the element by snapshot_id in stream + while let Some((snapshot, _)) = snapshot_stream.try_next().await? { + if snapshot.snapshot_id.simple().to_string() == snapshot_id { + return self.to_block(snapshot).await; + } } + } else { + return self.to_block(snapshot).await; } } @@ -101,7 +102,7 @@ impl<'a> FuseBlock<'a> { #[async_backtrace::framed] async fn to_block(&self, snapshot: Arc) -> Result { let limit = self.limit.unwrap_or(usize::MAX); - let len = std::cmp::min(snapshot.segments.len(), limit); + let len = std::cmp::min(snapshot.summary.block_count as usize, limit); let snapshot_id = snapshot.snapshot_id.simple().to_string().into_bytes(); let timestamp = snapshot.timestamp.unwrap_or_default().timestamp_micros(); @@ -118,26 +119,53 @@ impl<'a> FuseBlock<'a> { self.table.operator.clone(), self.table.schema(), ); - let segments = segments_io - .read_segments::>(&snapshot.segments[..len], true) - .await?; - for segment in segments { - let segment = segment?; - segment.blocks.iter().for_each(|block| { - let block = block.as_ref(); - block_location.put_slice(block.location.0.as_bytes()); - block_location.commit_row(); - block_size.push(NumberScalar::UInt64(block.block_size)); - file_size.push(NumberScalar::UInt64(block.file_size)); - row_count.push(NumberScalar::UInt64(block.row_count)); - bloom_filter_location.push( - block - .bloom_filter_index_location - .as_ref() - .map(|s| s.0.as_bytes().to_vec()), - ); - bloom_filter_size.push(NumberScalar::UInt64(block.bloom_filter_index_size)); - }); + + let mut row_num = 0; + let mut end_flag = false; + let chunk_size = std::cmp::min( + self.ctx.get_settings().get_max_storage_io_requests()? as usize, + len, + ); + for chunk in snapshot.segments.chunks(chunk_size) { + let segments = segments_io + .read_segments::>(chunk, true) + .await?; + for segment in segments { + let segment = segment?; + + let block_count = segment.summary.block_count as usize; + let take_num = if row_num + block_count >= len { + end_flag = true; + len - row_num + } else { + row_num += block_count; + block_count + }; + + segment.blocks.iter().take(take_num).for_each(|block| { + let block = block.as_ref(); + block_location.put_slice(block.location.0.as_bytes()); + block_location.commit_row(); + block_size.push(NumberScalar::UInt64(block.block_size)); + file_size.push(NumberScalar::UInt64(block.file_size)); + row_count.push(NumberScalar::UInt64(block.row_count)); + bloom_filter_location.push( + block + .bloom_filter_index_location + .as_ref() + .map(|s| s.0.as_bytes().to_vec()), + ); + bloom_filter_size.push(NumberScalar::UInt64(block.bloom_filter_index_size)); + }); + + if end_flag { + break; + } + } + + if end_flag { + break; + } } Ok(DataBlock::new( diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block_table.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block_table.rs index cef963311d37..b95d052afad5 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block_table.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block_table.rs @@ -26,13 +26,13 @@ use common_meta_app::schema::TableIdent; use common_meta_app::schema::TableInfo; use common_meta_app::schema::TableMeta; -use super::table_args::parse_func_table_args; use crate::pipelines::processors::port::OutputPort; use crate::pipelines::processors::processor::ProcessorPtr; use crate::pipelines::processors::AsyncSource; use crate::pipelines::processors::AsyncSourcer; use crate::pipelines::Pipeline; use crate::sessions::TableContext; +use crate::table_functions::parse_db_tb_ssid_args; use crate::table_functions::string_literal; use crate::table_functions::FuseBlock; use crate::table_functions::TableArgs; @@ -57,7 +57,7 @@ impl FuseBlockTable { table_args: TableArgs, ) -> Result> { let (arg_database_name, arg_table_name, arg_snapshot_id) = - parse_func_table_args(&table_args)?; + parse_db_tb_ssid_args(&table_args, FUSE_FUNC_BLOCK)?; let engine = FUSE_FUNC_BLOCK.to_owned(); @@ -106,10 +106,8 @@ impl Table for FuseBlockTable { let mut args = Vec::new(); args.push(string_literal(self.arg_database_name.as_str())); args.push(string_literal(self.arg_table_name.as_str())); - if self.arg_snapshot_id.is_some() { - args.push(string_literal( - self.arg_snapshot_id.clone().unwrap().as_str(), - )); + if let Some(arg_snapshot_id) = &self.arg_snapshot_id { + args.push(string_literal(arg_snapshot_id)); } Some(TableArgs::new_positioned(args)) } diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/mod.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/mod.rs index 9fd8b60f66fd..5096beb99af3 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/mod.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_blocks/mod.rs @@ -14,7 +14,6 @@ mod fuse_block; mod fuse_block_table; -mod table_args; pub use fuse_block::FuseBlock; pub use fuse_block_table::FuseBlockTable; diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/table_args.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/table_args.rs deleted file mode 100644 index e6657752bf6b..000000000000 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/table_args.rs +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use common_exception::ErrorCode; -use common_exception::Result; - -use crate::table_functions::string_value; -use crate::table_functions::TableArgs; - -pub(crate) fn parse_func_table_args( - table_args: &TableArgs, -) -> Result<(String, String, Option)> { - let args = table_args.expect_all_positioned("fuse_blocks", None)?; - match args.len() { - 3 => { - let db = string_value(&args[0])?; - let tbl = string_value(&args[1])?; - let snapshot_id = string_value(&args[2])?; - Ok((db, tbl, Some(snapshot_id))) - } - 2 => { - let db = string_value(&args[0])?; - let tbl = string_value(&args[1])?; - Ok((db, tbl, None)) - } - _ => Err(ErrorCode::BadArguments(format!( - "expecting , and (as string literals), but got {:?}", - args - ))), - } -} diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs index 28c5d1cd2487..22ba3e868c9f 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs @@ -16,9 +16,9 @@ use std::sync::Arc; use common_catalog::table::Table; use common_exception::Result; -use common_expression::types::number::UInt64Type; use common_expression::types::NumberDataType; use common_expression::types::StringType; +use common_expression::types::UInt64Type; use common_expression::DataBlock; use common_expression::FromData; use common_expression::TableDataType; @@ -38,40 +38,51 @@ use crate::FuseTable; pub struct FuseSegment<'a> { pub ctx: Arc, pub table: &'a FuseTable, - pub snapshot_id: String, + pub snapshot_id: Option, + pub limit: Option, } impl<'a> FuseSegment<'a> { - pub fn new(ctx: Arc, table: &'a FuseTable, snapshot_id: String) -> Self { + pub fn new( + ctx: Arc, + table: &'a FuseTable, + snapshot_id: Option, + limit: Option, + ) -> Self { Self { ctx, table, snapshot_id, + limit, } } #[async_backtrace::framed] pub async fn get_segments(&self) -> Result { let tbl = self.table; + let snapshot_id = self.snapshot_id.clone(); let maybe_snapshot = tbl.read_table_snapshot().await?; if let Some(snapshot) = maybe_snapshot { - // prepare the stream of snapshot - let snapshot_version = tbl.snapshot_format_version(None).await?; - let snapshot_location = tbl - .meta_location_generator - .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; - let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); - let mut snapshot_stream = reader.snapshot_history( - snapshot_location, - snapshot_version, - tbl.meta_location_generator().clone(), - ); - // find the element by snapshot_id in stream - while let Some((snapshot, _)) = snapshot_stream.try_next().await? { - if snapshot.snapshot_id.simple().to_string() == self.snapshot_id { - return self.to_block(&snapshot.segments).await; + if let Some(snapshot_id) = snapshot_id { + // prepare the stream of snapshot + let snapshot_version = tbl.snapshot_format_version(None).await?; + let snapshot_location = tbl + .meta_location_generator + .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; + let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); + let mut snapshot_stream = reader.snapshot_history( + snapshot_location, + snapshot_version, + tbl.meta_location_generator().clone(), + ); + while let Some((snapshot, _)) = snapshot_stream.try_next().await? { + if snapshot.snapshot_id.simple().to_string() == snapshot_id { + return self.to_block(&snapshot.segments).await; + } } + } else { + return self.to_block(&snapshot.segments).await; } } @@ -82,7 +93,9 @@ impl<'a> FuseSegment<'a> { #[async_backtrace::framed] async fn to_block(&self, segment_locations: &[Location]) -> Result { - let len = segment_locations.len(); + let limit = self.limit.unwrap_or(usize::MAX); + let len = std::cmp::min(segment_locations.len(), limit); + let mut format_versions: Vec = Vec::with_capacity(len); let mut block_count: Vec = Vec::with_capacity(len); let mut row_count: Vec = Vec::with_capacity(len); @@ -95,18 +108,40 @@ impl<'a> FuseSegment<'a> { self.table.operator.clone(), self.table.schema(), ); - let segments = segments_io - .read_segments::>(segment_locations, true) - .await?; - for (idx, segment) in segments.into_iter().enumerate() { - let segment = segment?; - format_versions.push(segment_locations[idx].1); - block_count.push(segment.summary.block_count); - row_count.push(segment.summary.row_count); - compressed.push(segment.summary.compressed_byte_size); - uncompressed.push(segment.summary.uncompressed_byte_size); - file_location.push(segment_locations[idx].0.clone().into_bytes()); + + let mut row_num = 0; + let mut end_flag = false; + let chunk_size = std::cmp::min( + self.ctx.get_settings().get_max_storage_io_requests()? as usize, + len, + ); + for chunk in segment_locations.chunks(chunk_size) { + let segments = segments_io + .read_segments::>(chunk, true) + .await?; + + let take_num = if row_num + chunk_size >= len { + end_flag = true; + len - row_num + } else { + row_num += chunk_size; + chunk_size + }; + for (idx, segment) in segments.into_iter().take(take_num).enumerate() { + let segment = segment?; + format_versions.push(segment_locations[idx].1); + block_count.push(segment.summary.block_count); + row_count.push(segment.summary.row_count); + compressed.push(segment.summary.compressed_byte_size); + uncompressed.push(segment.summary.uncompressed_byte_size); + file_location.push(segment_locations[idx].0.clone().into_bytes()); + } + + if end_flag { + break; + } } + Ok(DataBlock::new_from_columns(vec![ StringType::from_data(file_location), UInt64Type::from_data(format_versions), diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment_table.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment_table.rs index 086aa28b8ebd..1676246dafdd 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment_table.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment_table.rs @@ -27,13 +27,13 @@ use common_meta_app::schema::TableInfo; use common_meta_app::schema::TableMeta; use super::fuse_segment::FuseSegment; -use super::table_args::parse_func_history_args; use crate::pipelines::processors::port::OutputPort; use crate::pipelines::processors::processor::ProcessorPtr; use crate::pipelines::processors::AsyncSource; use crate::pipelines::processors::AsyncSourcer; use crate::pipelines::Pipeline; use crate::sessions::TableContext; +use crate::table_functions::parse_db_tb_ssid_args; use crate::table_functions::string_literal; use crate::table_functions::TableArgs; use crate::table_functions::TableFunction; @@ -46,7 +46,7 @@ pub struct FuseSegmentTable { table_info: TableInfo, arg_database_name: String, arg_table_name: String, - arg_snapshot_id: String, + arg_snapshot_id: Option, } impl FuseSegmentTable { @@ -57,7 +57,7 @@ impl FuseSegmentTable { table_args: TableArgs, ) -> Result> { let (arg_database_name, arg_table_name, arg_snapshot_id) = - parse_func_history_args(&table_args)?; + parse_db_tb_ssid_args(&table_args, FUSE_FUNC_SEGMENT)?; let engine = FUSE_FUNC_SEGMENT.to_owned(); @@ -103,17 +103,19 @@ impl Table for FuseSegmentTable { } fn table_args(&self) -> Option { - Some(TableArgs::new_positioned(vec![ - string_literal(self.arg_database_name.as_str()), - string_literal(self.arg_table_name.as_str()), - string_literal(self.arg_snapshot_id.as_str()), - ])) + let mut args = Vec::new(); + args.push(string_literal(self.arg_database_name.as_str())); + args.push(string_literal(self.arg_table_name.as_str())); + if let Some(arg_snapshot_id) = &self.arg_snapshot_id { + args.push(string_literal(arg_snapshot_id)); + } + Some(TableArgs::new_positioned(args)) } fn read_data( &self, ctx: Arc, - _: &DataSourcePlan, + plan: &DataSourcePlan, pipeline: &mut Pipeline, ) -> Result<()> { pipeline.add_source( @@ -124,6 +126,7 @@ impl Table for FuseSegmentTable { self.arg_database_name.to_owned(), self.arg_table_name.to_owned(), self.arg_snapshot_id.to_owned(), + plan.push_downs.as_ref().and_then(|extras| extras.limit), ) }, 1, @@ -138,7 +141,8 @@ struct FuseSegmentSource { ctx: Arc, arg_database_name: String, arg_table_name: String, - arg_snapshot_id: String, + arg_snapshot_id: Option, + limit: Option, } impl FuseSegmentSource { @@ -147,7 +151,8 @@ impl FuseSegmentSource { output: Arc, arg_database_name: String, arg_table_name: String, - arg_snapshot_id: String, + arg_snapshot_id: Option, + limit: Option, ) -> Result { AsyncSourcer::create(ctx.clone(), output, FuseSegmentSource { ctx, @@ -155,6 +160,7 @@ impl FuseSegmentSource { arg_table_name, arg_database_name, arg_snapshot_id, + limit, }) } } @@ -184,9 +190,14 @@ impl AsyncSource for FuseSegmentSource { let tbl = FuseTable::try_from_table(tbl.as_ref())?; Ok(Some( - FuseSegment::new(self.ctx.clone(), tbl, self.arg_snapshot_id.clone()) - .get_segments() - .await?, + FuseSegment::new( + self.ctx.clone(), + tbl, + self.arg_snapshot_id.clone(), + self.limit, + ) + .get_segments() + .await?, )) } } diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/mod.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/mod.rs index 6530098eaefe..98809ce02033 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/mod.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/mod.rs @@ -14,8 +14,6 @@ mod fuse_segment; mod fuse_segment_table; -mod table_args; pub use fuse_segment::FuseSegment; pub use fuse_segment_table::FuseSegmentTable; -pub use table_args::parse_func_history_args; diff --git a/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot_table.rs b/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot_table.rs index f227dadf4ff0..fb5b7757ea28 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot_table.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot_table.rs @@ -30,10 +30,10 @@ use common_pipeline_sources::AsyncSource; use common_pipeline_sources::AsyncSourcer; use super::fuse_snapshot::FuseSnapshot; -use super::table_args::parse_func_history_args; use crate::pipelines::processors::port::OutputPort; use crate::pipelines::Pipeline; use crate::sessions::TableContext; +use crate::table_functions::parse_db_tb_args; use crate::table_functions::string_literal; use crate::table_functions::TableArgs; use crate::table_functions::TableFunction; @@ -55,7 +55,8 @@ impl FuseSnapshotTable { table_id: u64, table_args: TableArgs, ) -> Result> { - let (arg_database_name, arg_table_name) = parse_func_history_args(&table_args)?; + let (arg_database_name, arg_table_name) = + parse_db_tb_args(&table_args, FUSE_FUNC_SNAPSHOT)?; let engine = FUSE_FUNC_SNAPSHOT.to_owned(); diff --git a/src/query/storages/fuse/src/table_functions/fuse_snapshots/mod.rs b/src/query/storages/fuse/src/table_functions/fuse_snapshots/mod.rs index 4ad9f5cc83ac..7f14ccd2d053 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_snapshots/mod.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_snapshots/mod.rs @@ -14,8 +14,6 @@ mod fuse_snapshot; mod fuse_snapshot_table; -mod table_args; pub use fuse_snapshot::FuseSnapshot; pub use fuse_snapshot_table::FuseSnapshotTable; -pub use table_args::parse_func_history_args; diff --git a/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic_table.rs b/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic_table.rs index 07cc49017eaa..2eef9ac2919c 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic_table.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic_table.rs @@ -33,7 +33,7 @@ use super::fuse_statistic::FuseStatistic; use crate::pipelines::processors::port::OutputPort; use crate::pipelines::Pipeline; use crate::sessions::TableContext; -use crate::table_functions::fuse_snapshots::parse_func_history_args; +use crate::table_functions::parse_db_tb_args; use crate::table_functions::string_literal; use crate::table_functions::TableArgs; use crate::table_functions::TableFunction; @@ -55,7 +55,8 @@ impl FuseStatisticTable { table_id: u64, table_args: TableArgs, ) -> Result> { - let (arg_database_name, arg_table_name) = parse_func_history_args(&table_args)?; + let (arg_database_name, arg_table_name) = + parse_db_tb_args(&table_args, FUSE_FUNC_STATISTICS)?; let engine = FUSE_FUNC_STATISTICS.to_owned(); diff --git a/src/query/storages/fuse/src/table_functions/mod.rs b/src/query/storages/fuse/src/table_functions/mod.rs index a1d3f400fbae..4d0452d6d19a 100644 --- a/src/query/storages/fuse/src/table_functions/mod.rs +++ b/src/query/storages/fuse/src/table_functions/mod.rs @@ -19,7 +19,6 @@ mod fuse_snapshots; mod fuse_statistics; mod table_args; -pub use clustering_information::unwrap_tuple; pub use clustering_information::ClusteringInformation; pub use clustering_information::ClusteringInformationTable; use common_catalog::table_args::TableArgs; @@ -31,5 +30,4 @@ pub use fuse_segments::FuseSegmentTable; pub use fuse_snapshots::FuseSnapshot; pub use fuse_snapshots::FuseSnapshotTable; pub use fuse_statistics::FuseStatisticTable; -pub use table_args::string_literal; -pub use table_args::string_value; +pub use table_args::*; diff --git a/src/query/storages/fuse/src/table_functions/table_args.rs b/src/query/storages/fuse/src/table_functions/table_args.rs index 12d193a49163..4ba8bb27cac3 100644 --- a/src/query/storages/fuse/src/table_functions/table_args.rs +++ b/src/query/storages/fuse/src/table_functions/table_args.rs @@ -14,8 +14,11 @@ use common_exception::ErrorCode; use common_exception::Result; +use common_expression::Expr; use common_expression::Scalar; +use crate::table_functions::TableArgs; + pub fn string_value(value: &Scalar) -> Result { match value { Scalar::String(val) => String::from_utf8(val.clone()) @@ -27,3 +30,43 @@ pub fn string_value(value: &Scalar) -> Result { pub fn string_literal(val: &str) -> Scalar { Scalar::String(val.as_bytes().to_vec()) } + +pub fn parse_db_tb_args(table_args: &TableArgs, func_name: &str) -> Result<(String, String)> { + let args = table_args.expect_all_positioned(func_name, Some(2))?; + let db = string_value(&args[0])?; + let tbl = string_value(&args[1])?; + Ok((db, tbl)) +} + +pub fn parse_db_tb_ssid_args( + table_args: &TableArgs, + func_name: &str, +) -> Result<(String, String, Option)> { + let args = table_args.expect_all_positioned(func_name, None)?; + match args.len() { + 3 => { + let db = string_value(&args[0])?; + let tbl = string_value(&args[1])?; + let snapshot_id = string_value(&args[2])?; + Ok((db, tbl, Some(snapshot_id))) + } + 2 => { + let db = string_value(&args[0])?; + let tbl = string_value(&args[1])?; + Ok((db, tbl, None)) + } + _ => Err(ErrorCode::BadArguments(format!( + "expecting , and (as string literals), but got {:?}", + args + ))), + } +} + +pub fn unwrap_tuple(expr: &Expr) -> Option> { + match expr { + Expr::FunctionCall { function, args, .. } if function.signature.name == "tuple" => { + Some(args.clone()) + } + _ => None, + } +} diff --git a/src/query/storages/parquet/src/parquet_table/blocking.rs b/src/query/storages/parquet/src/parquet_table/blocking.rs index 0c118d9f1461..f7bc9d5c64da 100644 --- a/src/query/storages/parquet/src/parquet_table/blocking.rs +++ b/src/query/storages/parquet/src/parquet_table/blocking.rs @@ -22,6 +22,7 @@ use common_catalog::table::Table; use common_exception::ErrorCode; use common_exception::Result; use common_meta_app::principal::StageInfo; +use common_storage::infer_schema_with_extension; use common_storage::StageFileInfo; use common_storage::StageFilesInfo; use opendal::Operator; @@ -56,6 +57,7 @@ impl ParquetTable { stage_info, files_info, files_to_read, + schema_from: first_file, })) } @@ -71,7 +73,7 @@ impl ParquetTable { ErrorCode::Internal(format!("Read parquet file '{}''s meta error: {}", path, e)) })?; - let arrow_schema = pread::infer_schema(&first_meta)?; + let arrow_schema = infer_schema_with_extension(&first_meta)?; let schema_descr = first_meta.schema_descr; Ok((arrow_schema, schema_descr)) } diff --git a/src/query/storages/parquet/src/parquet_table/non_blocking.rs b/src/query/storages/parquet/src/parquet_table/non_blocking.rs index 90bc23cc5249..d6b9f3994c02 100644 --- a/src/query/storages/parquet/src/parquet_table/non_blocking.rs +++ b/src/query/storages/parquet/src/parquet_table/non_blocking.rs @@ -22,6 +22,7 @@ use common_catalog::table::Table; use common_exception::ErrorCode; use common_exception::Result; use common_meta_app::principal::StageInfo; +use common_storage::infer_schema_with_extension; use common_storage::init_stage_operator; use common_storage::StageFileInfo; use common_storage::StageFilesInfo; @@ -67,6 +68,7 @@ impl ParquetTable { stage_info, files_info, files_to_read, + schema_from: first_file, })) } @@ -82,8 +84,7 @@ impl ParquetTable { let first_meta = pread::read_metadata_async(&mut reader).await.map_err(|e| { ErrorCode::Internal(format!("Read parquet file '{}''s meta error: {}", path, e)) })?; - - let arrow_schema = pread::infer_schema(&first_meta)?; + let arrow_schema = infer_schema_with_extension(&first_meta)?; let schema_descr = first_meta.schema_descr; Ok((arrow_schema, schema_descr)) } diff --git a/src/query/storages/parquet/src/parquet_table/partition.rs b/src/query/storages/parquet/src/parquet_table/partition.rs index ca4c71f22679..577156c84b87 100644 --- a/src/query/storages/parquet/src/parquet_table/partition.rs +++ b/src/query/storages/parquet/src/parquet_table/partition.rs @@ -114,6 +114,8 @@ impl ParquetTable { Ok(PartitionPruner { schema, + schema_descr: self.schema_descr.clone(), + schema_from: self.schema_from.clone(), row_group_pruner, page_pruners, columns_to_read, diff --git a/src/query/storages/parquet/src/parquet_table/table.rs b/src/query/storages/parquet/src/parquet_table/table.rs index 57404746d741..d43aebc2acb2 100644 --- a/src/query/storages/parquet/src/parquet_table/table.rs +++ b/src/query/storages/parquet/src/parquet_table/table.rs @@ -54,6 +54,7 @@ pub struct ParquetTable { pub(super) arrow_schema: ArrowSchema, pub(super) schema_descr: SchemaDescriptor, pub(super) files_to_read: Option>, + pub(super) schema_from: String, } impl ParquetTable { @@ -69,6 +70,7 @@ impl ParquetTable { files_info: info.files_info.clone(), files_to_read: info.files_to_read.clone(), schema_descr: info.schema_descr.clone(), + schema_from: info.schema_from.clone(), })) } } @@ -108,6 +110,7 @@ impl Table for ParquetTable { schema_descr: self.schema_descr.clone(), files_info: self.files_info.clone(), files_to_read: self.files_to_read.clone(), + schema_from: self.schema_from.clone(), }) } diff --git a/src/query/storages/parquet/src/pruning.rs b/src/query/storages/parquet/src/pruning.rs index ca52cbf7497d..a6a0f7d247de 100644 --- a/src/query/storages/parquet/src/pruning.rs +++ b/src/query/storages/parquet/src/pruning.rs @@ -27,6 +27,7 @@ use common_arrow::arrow::io::parquet::read::indexes::FieldPageStatistics; use common_arrow::parquet::indexes::Interval; use common_arrow::parquet::metadata::FileMetaData; use common_arrow::parquet::metadata::RowGroupMetaData; +use common_arrow::parquet::metadata::SchemaDescriptor; use common_arrow::parquet::read::read_pages_locations; use common_catalog::plan::PartStatistics; use common_catalog::plan::Partitions; @@ -55,6 +56,8 @@ use crate::statistics::BatchStatistics; pub struct PartitionPruner { /// Table schema. pub schema: TableSchemaRef, + pub schema_descr: SchemaDescriptor, + pub schema_from: String, /// Pruner to prune row groups. pub row_group_pruner: Option>, /// Pruners to prune pages. @@ -73,6 +76,21 @@ pub struct PartitionPruner { pub parquet_fast_read_bytes: usize, } +fn check_parquet_schema( + expect: &SchemaDescriptor, + actual: &SchemaDescriptor, + path: &str, + schema_from: &str, +) -> Result<()> { + if expect.fields() != actual.fields() || expect.columns() != actual.columns() { + return Err(ErrorCode::BadBytes(format!( + "infer schema from '{}', but get diff schema in file '{}'. Expected schema: {:?}, actual: {:?}", + schema_from, path, expect, actual + ))); + } + Ok(()) +} + impl PartitionPruner { #[async_backtrace::framed] pub fn read_and_prune_file_meta( @@ -81,6 +99,12 @@ impl PartitionPruner { file_meta: FileMetaData, operator: Operator, ) -> Result<(PartStatistics, Vec)> { + check_parquet_schema( + &self.schema_descr, + file_meta.schema(), + path, + &self.schema_from, + )?; let mut stats = PartStatistics::default(); let mut partitions = vec![]; diff --git a/src/query/storages/stage/Cargo.toml b/src/query/storages/stage/Cargo.toml index 32df8a9b9800..e05d78ae0297 100644 --- a/src/query/storages/stage/Cargo.toml +++ b/src/query/storages/stage/Cargo.toml @@ -20,6 +20,7 @@ common-formats = { path = "../../formats" } common-meta-app = { path = "../../../meta/app" } common-pipeline-core = { path = "../../pipeline/core" } common-pipeline-sources = { path = "../../pipeline/sources" } +common-pipeline-transforms = { path = "../../pipeline/transforms" } common-storage = { path = "../../../common/storage" } async-backtrace = { workspace = true } @@ -27,7 +28,9 @@ async-trait = { version = "0.1.57", package = "async-trait-fn" } dashmap = "5.4.0" opendal = { workspace = true } parking_lot = "0.12.1" +serde = { workspace = true } tracing = "0.1.36" +typetag = "0.2.6" uuid = { version = "1.1.2", features = ["serde", "v4"] } [build-dependencies] diff --git a/src/query/storages/stage/src/lib.rs b/src/query/storages/stage/src/lib.rs index 4cfa44387dc2..a532a745aa0c 100644 --- a/src/query/storages/stage/src/lib.rs +++ b/src/query/storages/stage/src/lib.rs @@ -14,7 +14,8 @@ #![allow(clippy::uninlined_format_args)] +mod parquet_file; +mod row_based_file; mod stage_table; -mod stage_table_sink; pub use stage_table::StageTable; diff --git a/src/query/storages/stage/src/parquet_file/block_batch.rs b/src/query/storages/stage/src/parquet_file/block_batch.rs new file mode 100644 index 000000000000..f1154f2b85a4 --- /dev/null +++ b/src/query/storages/stage/src/parquet_file/block_batch.rs @@ -0,0 +1,64 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use common_expression::BlockMetaInfo; +use common_expression::DataBlock; + +#[derive(Debug)] +pub struct BlockBatch { + pub blocks: Vec, +} + +impl BlockBatch { + pub fn create_block(blocks: Vec) -> DataBlock { + DataBlock::empty_with_meta(Box::new(BlockBatch { blocks })) + } +} + +impl Clone for BlockBatch { + fn clone(&self) -> Self { + unreachable!("Buffers should not be cloned") + } +} + +impl serde::Serialize for BlockBatch { + fn serialize(&self, _: S) -> Result + where S: serde::Serializer { + unreachable!("Buffers should not be serialized") + } +} + +impl<'de> serde::Deserialize<'de> for BlockBatch { + fn deserialize(_: D) -> Result + where D: serde::Deserializer<'de> { + unreachable!("Buffers should not be deserialized") + } +} + +#[typetag::serde(name = "unload_block_batch")] +impl BlockMetaInfo for BlockBatch { + fn as_any(&self) -> &dyn Any { + self + } + + fn equals(&self, _info: &Box) -> bool { + unreachable!("Buffers should not be compared") + } + + fn clone_self(&self) -> Box { + unreachable!("Buffers should not be cloned") + } +} diff --git a/src/query/storages/stage/src/parquet_file/limit_file_size_processor.rs b/src/query/storages/stage/src/parquet_file/limit_file_size_processor.rs new file mode 100644 index 000000000000..b02ed9f44a17 --- /dev/null +++ b/src/query/storages/stage/src/parquet_file/limit_file_size_processor.rs @@ -0,0 +1,133 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::sync::Arc; + +use common_exception::Result; +use common_expression::DataBlock; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; + +use super::block_batch::BlockBatch; + +pub(super) struct LimitFileSizeProcessor { + input: Arc, + output: Arc, + + threshold: usize, + + input_data: Option, + output_data: Option, + + // since we only output one BlockBatch each time, the remaining blocks is kept here. + // remember to flush it when input is finished + blocks: Vec, +} + +impl LimitFileSizeProcessor { + pub(super) fn try_create( + input: Arc, + output: Arc, + threshold: usize, + ) -> Result { + let p = Self { + input, + output, + threshold, + input_data: None, + output_data: None, + blocks: Vec::new(), + }; + Ok(ProcessorPtr::create(Box::new(p))) + } +} + +impl Processor for LimitFileSizeProcessor { + fn name(&self) -> String { + String::from("ResizeProcessor") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + self.input.finish(); + Ok(Event::Finished) + } else if !self.output.can_push() { + self.input.set_not_need_data(); + Ok(Event::NeedConsume) + } else { + match self.output_data.take() { + Some(data) => { + self.output.push_data(Ok(data)); + Ok(Event::NeedConsume) + } + None => { + if self.input_data.is_some() { + Ok(Event::Sync) + } else if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + Ok(Event::Sync) + } else if self.input.is_finished() { + if self.blocks.is_empty() { + self.output.finish(); + Ok(Event::Finished) + } else { + // flush the remaining blocks + let blocks = std::mem::take(&mut self.blocks); + self.output.push_data(Ok(BlockBatch::create_block(blocks))); + Ok(Event::NeedConsume) + } + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + } + } + } + + fn process(&mut self) -> Result<()> { + assert!(self.input_data.is_some()); + assert!(self.output_data.is_none()); + // slicing has overhead, we do not do it for now. + let block = self.input_data.take().unwrap(); + let mut blocks = std::mem::take(&mut self.blocks); + + blocks.push(block); + let mut break_point = blocks.len(); + let mut size = 0; + for (i, b) in blocks.iter().enumerate() { + size += b.memory_size(); + if size > self.threshold { + break_point = i; + break; + } + } + if break_point == blocks.len() { + self.blocks = blocks; + } else { + let remain = blocks.split_off(break_point + 1); + self.output_data = Some(BlockBatch::create_block(blocks)); + self.blocks = remain; + } + Ok(()) + } +} diff --git a/src/query/storages/stage/src/parquet_file/mod.rs b/src/query/storages/stage/src/parquet_file/mod.rs new file mode 100644 index 000000000000..402ba4cffdd9 --- /dev/null +++ b/src/query/storages/stage/src/parquet_file/mod.rs @@ -0,0 +1,19 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod block_batch; +mod limit_file_size_processor; +mod pipeline; +mod sink_processor; +pub(crate) use pipeline::append_data_to_parquet_files; diff --git a/src/query/storages/stage/src/parquet_file/pipeline.rs b/src/query/storages/stage/src/parquet_file/pipeline.rs new file mode 100644 index 000000000000..ae2f31ad2f56 --- /dev/null +++ b/src/query/storages/stage/src/parquet_file/pipeline.rs @@ -0,0 +1,65 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_catalog::plan::StageTableInfo; +use common_catalog::table_context::TableContext; +use common_exception::Result; +use common_formats::FileFormatOptionsExt; +use common_pipeline_core::Pipeline; +use opendal::Operator; + +use crate::parquet_file::limit_file_size_processor::LimitFileSizeProcessor; +use crate::parquet_file::sink_processor::ParquetFileSink; + +// LimitFileSizeProcessor * 1: slice/group block to batches (as a block meta) that are suitable as a file. +// ParquetFileSink * N: simply serialize blocks in each meta to a whole file and write out. +#[allow(clippy::too_many_arguments)] +pub(crate) fn append_data_to_parquet_files( + pipeline: &mut Pipeline, + ctx: Arc, + table_info: StageTableInfo, + op: Operator, + max_file_size: usize, + max_threads: usize, + uuid: String, + group_id: &std::sync::atomic::AtomicUsize, +) -> Result<()> { + pipeline.try_resize(1)?; + pipeline.add_transform(|input, output| { + LimitFileSizeProcessor::try_create(input, output, max_file_size) + })?; + if max_file_size != usize::MAX { + pipeline.try_resize(max_threads)?; + } + pipeline.add_sink(|input| { + let gid = group_id.fetch_add(1, std::sync::atomic::Ordering::Relaxed); + let mut options_ext = + FileFormatOptionsExt::create_from_settings(&ctx.get_settings(), false)?; + let output_format = options_ext.get_output_format( + table_info.schema(), + table_info.stage_info.file_format_params.clone(), + )?; + ParquetFileSink::try_create( + input, + table_info.clone(), + output_format, + op.clone(), + uuid.clone(), + gid, + ) + })?; + Ok(()) +} diff --git a/src/query/storages/stage/src/parquet_file/sink_processor.rs b/src/query/storages/stage/src/parquet_file/sink_processor.rs new file mode 100644 index 000000000000..1923da9a01a7 --- /dev/null +++ b/src/query/storages/stage/src/parquet_file/sink_processor.rs @@ -0,0 +1,123 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::mem; +use std::sync::Arc; + +use async_trait::async_trait; +use common_catalog::plan::StageTableInfo; +use common_exception::Result; +use common_expression::BlockMetaInfoDowncast; +use common_expression::DataBlock; +use common_formats::output_format::OutputFormat; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; +use opendal::Operator; + +use crate::parquet_file::block_batch::BlockBatch; +use crate::stage_table::unload_path; + +pub struct ParquetFileSink { + input: Arc, + table_info: StageTableInfo, + output_format: Box, + + input_data: Option, + output_data: Vec, + + data_accessor: Operator, + + uuid: String, + group_id: usize, + batch_id: usize, +} + +impl ParquetFileSink { + #[allow(clippy::too_many_arguments)] + pub fn try_create( + input: Arc, + table_info: StageTableInfo, + output_format: Box, + data_accessor: Operator, + uuid: String, + group_id: usize, + ) -> Result { + Ok(ProcessorPtr::create(Box::new(ParquetFileSink { + input, + table_info, + output_format, + input_data: None, + output_data: vec![], + data_accessor, + uuid, + group_id, + batch_id: 0, + }))) + } +} + +#[async_trait] +impl Processor for ParquetFileSink { + fn name(&self) -> String { + "ParquetFileSink".to_string() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if !self.output_data.is_empty() { + self.input.set_not_need_data(); + Ok(Event::Async) + } else if self.input_data.is_some() { + self.input.set_not_need_data(); + Ok(Event::Sync) + } else if self.input.is_finished() { + self.input.set_not_need_data(); + Ok(Event::Finished) + } else if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + self.input.set_not_need_data(); + Ok(Event::Sync) + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + + fn process(&mut self) -> Result<()> { + let block = self.input_data.take().unwrap(); + let block_meta = block.get_owned_meta().unwrap(); + let blocks = BlockBatch::downcast_from(block_meta).unwrap(); + for b in blocks.blocks { + self.output_format.serialize_block(&b)?; + } + self.output_data = self.output_format.finalize()?; + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + assert!(!self.output_data.is_empty()); + let path = unload_path(&self.table_info, &self.uuid, self.group_id, self.batch_id); + let data = mem::take(&mut self.output_data); + self.data_accessor.write(&path, data).await?; + self.batch_id += 1; + Ok(()) + } +} diff --git a/src/query/storages/stage/src/row_based_file/buffers.rs b/src/query/storages/stage/src/row_based_file/buffers.rs new file mode 100644 index 000000000000..6303590005f7 --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/buffers.rs @@ -0,0 +1,64 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use common_expression::BlockMetaInfo; +use common_expression::DataBlock; + +#[derive(Debug)] +pub struct FileOutputBuffers { + pub buffers: Vec>, +} + +impl FileOutputBuffers { + pub fn create_block(buffers: Vec>) -> DataBlock { + DataBlock::empty_with_meta(Box::new(FileOutputBuffers { buffers })) + } +} + +impl Clone for FileOutputBuffers { + fn clone(&self) -> Self { + unreachable!("Buffers should not be cloned") + } +} + +impl serde::Serialize for FileOutputBuffers { + fn serialize(&self, _: S) -> Result + where S: serde::Serializer { + unreachable!("FileOutputBuffers should not be serialized") + } +} + +impl<'de> serde::Deserialize<'de> for FileOutputBuffers { + fn deserialize(_: D) -> Result + where D: serde::Deserializer<'de> { + unreachable!("FileOutputBuffers should not be deserialized") + } +} + +#[typetag::serde(name = "unload_buffers")] +impl BlockMetaInfo for FileOutputBuffers { + fn as_any(&self) -> &dyn Any { + self + } + + fn equals(&self, _info: &Box) -> bool { + unreachable!("FileOutputBuffers should not be compared") + } + + fn clone_self(&self) -> Box { + unreachable!("FileOutputBuffers should not be cloned") + } +} diff --git a/src/query/storages/stage/src/row_based_file/limit_file_size_processor.rs b/src/query/storages/stage/src/row_based_file/limit_file_size_processor.rs new file mode 100644 index 000000000000..353df80ee0fe --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/limit_file_size_processor.rs @@ -0,0 +1,140 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::mem; +use std::sync::Arc; + +use common_exception::Result; +use common_expression::BlockMetaInfoDowncast; +use common_expression::DataBlock; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; + +use crate::row_based_file::buffers::FileOutputBuffers; + +pub(super) struct LimitFileSizeProcessor { + input: Arc, + output: Arc, + threshold: usize, + + input_data: Option, + output_data: Option, + buffers: Vec>, +} + +impl LimitFileSizeProcessor { + pub(super) fn try_create( + input: Arc, + output: Arc, + threshold: usize, + ) -> Result { + let p = Self { + input, + output, + threshold, + input_data: None, + output_data: None, + buffers: Vec::new(), + }; + Ok(ProcessorPtr::create(Box::new(p))) + } +} + +impl Processor for LimitFileSizeProcessor { + fn name(&self) -> String { + String::from("ResizeProcessor") + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> common_exception::Result { + if self.output.is_finished() { + self.input.finish(); + Ok(Event::Finished) + } else if !self.output.can_push() { + self.input.set_not_need_data(); + Ok(Event::NeedConsume) + } else { + match self.output_data.take() { + Some(data) => { + self.output.push_data(Ok(data)); + Ok(Event::NeedConsume) + } + None => { + if self.input_data.is_some() { + Ok(Event::Sync) + } else if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + Ok(Event::Sync) + } else if self.input.is_finished() { + if self.buffers.is_empty() { + self.output.finish(); + Ok(Event::Finished) + } else { + let buffers = std::mem::take(&mut self.buffers); + self.output + .push_data(Ok(FileOutputBuffers::create_block(buffers))); + Ok(Event::NeedConsume) + } + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + } + } + } + + fn process(&mut self) -> common_exception::Result<()> { + assert!(self.output_data.is_none()); + assert!(self.input_data.is_some()); + + let block = self.input_data.take().unwrap(); + let block_meta = block.get_owned_meta().unwrap(); + let buffers = FileOutputBuffers::downcast_from(block_meta).unwrap(); + let buffers = buffers.buffers; + + self.buffers.extend(buffers.into_iter()); + + let mut size = 0; + let mut buffers = mem::take(&mut self.buffers); + let break_idx = buffers + .iter() + .enumerate() + .find_map(|(idx, b)| { + size += b.len(); + if size >= self.threshold { + Some(idx) + } else { + None + } + }) + .unwrap_or(buffers.len()); + if break_idx == buffers.len() { + self.buffers = buffers; + Ok(()) + } else { + let remain = buffers.split_off(break_idx + 1); + self.output_data = Some(FileOutputBuffers::create_block(buffers)); + self.buffers = remain; + Ok(()) + } + } +} diff --git a/src/query/storages/stage/src/row_based_file/mod.rs b/src/query/storages/stage/src/row_based_file/mod.rs new file mode 100644 index 000000000000..53f40c4bb10d --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/mod.rs @@ -0,0 +1,20 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod buffers; +pub(crate) mod limit_file_size_processor; +mod pipeline; +mod serialize_processor; +pub(crate) mod sink_processor; +pub(crate) use pipeline::append_data_to_row_based_files; diff --git a/src/query/storages/stage/src/row_based_file/pipeline.rs b/src/query/storages/stage/src/row_based_file/pipeline.rs new file mode 100644 index 000000000000..a72dfd4b2a10 --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/pipeline.rs @@ -0,0 +1,77 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_catalog::plan::StageTableInfo; +use common_catalog::table_context::TableContext; +use common_exception::Result; +use common_formats::FileFormatOptionsExt; +use common_pipeline_core::Pipeline; +use opendal::Operator; + +use crate::row_based_file::limit_file_size_processor::LimitFileSizeProcessor; +use crate::row_based_file::serialize_processor::SerializeProcessor; +use crate::row_based_file::sink_processor::RowBasedFileSink; + +// SerializeProcessor * N: serialize each data block to many small byte buffers. +// LimitFileSizeProcessor * 1: group small byte buffers to batches (as a block meta) that are large enough as a file. +// RowBasedFileSink * N: simply concat small byte buffers to a whole and write out. +#[allow(clippy::too_many_arguments)] +pub(crate) fn append_data_to_row_based_files( + pipeline: &mut Pipeline, + ctx: Arc, + table_info: StageTableInfo, + op: Operator, + max_file_size: usize, + max_threads: usize, + uuid: String, + group_id: &std::sync::atomic::AtomicUsize, +) -> Result<()> { + let mut options_ext = FileFormatOptionsExt::create_from_settings(&ctx.get_settings(), false)?; + let output_format = options_ext.get_output_format( + table_info.schema(), + table_info.stage_info.file_format_params.clone(), + )?; + let prefix = output_format.serialize_prefix()?; + + pipeline.add_transform(|input, output| { + let mut options_ext = + FileFormatOptionsExt::create_from_settings(&ctx.get_settings(), false)?; + let output_format = options_ext.get_output_format( + table_info.schema(), + table_info.stage_info.file_format_params.clone(), + )?; + SerializeProcessor::try_create(input, output, output_format) + })?; + pipeline.try_resize(1)?; + pipeline.add_transform(|input, output| { + LimitFileSizeProcessor::try_create(input, output, max_file_size) + })?; + if max_file_size != usize::MAX { + pipeline.try_resize(max_threads)?; + } + pipeline.add_sink(|input| { + let gid = group_id.fetch_add(1, std::sync::atomic::Ordering::Relaxed); + RowBasedFileSink::try_create( + input, + table_info.clone(), + op.clone(), + prefix.clone(), + uuid.clone(), + gid, + ) + })?; + Ok(()) +} diff --git a/src/query/storages/stage/src/row_based_file/serialize_processor.rs b/src/query/storages/stage/src/row_based_file/serialize_processor.rs new file mode 100644 index 000000000000..c9f976cf4735 --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/serialize_processor.rs @@ -0,0 +1,61 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_exception::Result; +use common_expression::DataBlock; +use common_formats::output_format::OutputFormat; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::port::OutputPort; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_transforms::processors::transforms::Transform; +use common_pipeline_transforms::processors::transforms::Transformer; + +use crate::row_based_file::buffers::FileOutputBuffers; + +pub(super) struct SerializeProcessor { + output_format: Box, +} + +impl SerializeProcessor { + pub(super) fn try_create( + input: Arc, + output: Arc, + output_format: Box, + ) -> Result { + Ok(ProcessorPtr::create(Transformer::create( + input, + output, + SerializeProcessor { output_format }, + ))) + } +} + +impl Transform for SerializeProcessor { + const NAME: &'static str = "SerializeProcessor"; + + fn transform(&mut self, block: DataBlock) -> Result { + let mut buffers = vec![]; + let step = 1024; + let num_rows = block.num_rows(); + for i in (0..num_rows).step_by(step) { + let end = (i + step).min(num_rows); + let small_block = block.slice(i..end); + let bs = self.output_format.serialize_block(&small_block)?; + buffers.push(bs); + } + Ok(FileOutputBuffers::create_block(buffers)) + } +} diff --git a/src/query/storages/stage/src/row_based_file/sink_processor.rs b/src/query/storages/stage/src/row_based_file/sink_processor.rs new file mode 100644 index 000000000000..e0b41f60fb66 --- /dev/null +++ b/src/query/storages/stage/src/row_based_file/sink_processor.rs @@ -0,0 +1,126 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; +use std::mem; +use std::sync::Arc; + +use async_trait::async_trait; +use common_catalog::plan::StageTableInfo; +use common_exception::Result; +use common_expression::BlockMetaInfoDowncast; +use common_expression::DataBlock; +use common_pipeline_core::processors::port::InputPort; +use common_pipeline_core::processors::processor::Event; +use common_pipeline_core::processors::processor::ProcessorPtr; +use common_pipeline_core::processors::Processor; +use opendal::Operator; + +use crate::row_based_file::buffers::FileOutputBuffers; +use crate::stage_table::unload_path; + +pub struct RowBasedFileSink { + input: Arc, + table_info: StageTableInfo, + + // always blocks for a whole file if not empty + input_data: Option, + // always the data for a whole file if not empty + output_data: Vec, + + data_accessor: Operator, + prefix: Vec, + + uuid: String, + group_id: usize, + batch_id: usize, +} + +impl RowBasedFileSink { + #[allow(clippy::too_many_arguments)] + pub fn try_create( + input: Arc, + table_info: StageTableInfo, + data_accessor: Operator, + prefix: Vec, + uuid: String, + group_id: usize, + ) -> Result { + Ok(ProcessorPtr::create(Box::new(RowBasedFileSink { + table_info, + input, + input_data: None, + data_accessor, + prefix, + uuid, + group_id, + batch_id: 0, + output_data: vec![], + }))) + } +} + +#[async_trait] +impl Processor for RowBasedFileSink { + fn name(&self) -> String { + "StageSink".to_string() + } + + fn as_any(&mut self) -> &mut dyn Any { + self + } + + fn event(&mut self) -> Result { + if !self.output_data.is_empty() { + self.input.set_not_need_data(); + Ok(Event::Async) + } else if self.input_data.is_some() { + self.input.set_not_need_data(); + Ok(Event::Sync) + } else if self.input.is_finished() { + self.input.set_not_need_data(); + Ok(Event::Finished) + } else if self.input.has_data() { + self.input_data = Some(self.input.pull_data().unwrap()?); + self.input.set_not_need_data(); + Ok(Event::Sync) + } else { + self.input.set_need_data(); + Ok(Event::NeedData) + } + } + + fn process(&mut self) -> Result<()> { + let block = self.input_data.take().unwrap(); + let block_meta = block.get_owned_meta().unwrap(); + let buffers = FileOutputBuffers::downcast_from(block_meta).unwrap(); + let size = buffers.buffers.iter().map(|b| b.len()).sum::(); + let mut output = Vec::with_capacity(self.prefix.len() + size); + output.extend_from_slice(self.prefix.as_slice()); + for b in buffers.buffers { + output.extend_from_slice(b.as_slice()); + } + self.output_data = output; + Ok(()) + } + + #[async_backtrace::framed] + async fn async_process(&mut self) -> Result<()> { + let path = unload_path(&self.table_info, &self.uuid, self.group_id, self.batch_id); + let data = mem::take(&mut self.output_data); + self.data_accessor.write(&path, data).await?; + self.batch_id += 1; + Ok(()) + } +} diff --git a/src/query/storages/stage/src/stage_table.rs b/src/query/storages/stage/src/stage_table.rs index 40fe436a41da..6065e5971b27 100644 --- a/src/query/storages/stage/src/stage_table.rs +++ b/src/query/storages/stage/src/stage_table.rs @@ -34,6 +34,7 @@ use common_exception::ErrorCode; use common_exception::Result; use common_expression::BlockThresholds; use common_expression::TableSchemaRefExt; +use common_meta_app::principal::FileFormatParams; use common_meta_app::principal::StageInfo; use common_meta_app::schema::TableInfo; use common_pipeline_core::Pipeline; @@ -46,7 +47,8 @@ use opendal::Operator; use parking_lot::Mutex; use tracing::error; -use crate::stage_table_sink::StageTableSink; +use crate::parquet_file::append_data_to_parquet_files; +use crate::row_based_file::append_data_to_row_based_files; /// TODO: we need to track the data metrics in stage table. pub struct StageTable { table_info: StageTableInfo, @@ -225,44 +227,50 @@ impl Table for StageTable { pipeline: &mut Pipeline, _: AppendMode, ) -> Result<()> { + let settings = ctx.get_settings(); + let single = self.table_info.stage_info.copy_options.single; - let op = StageTable::get_op(&self.table_info.stage_info)?; + let max_file_size = if single { + usize::MAX + } else { + let max_file_size = self.table_info.stage_info.copy_options.max_file_size; + if max_file_size == 0 { + // 256M per file by default. + 256 * 1024 * 1024 + } else { + let mem_limit = (settings.get_max_memory_usage()? / 2) as usize; + max_file_size.min(mem_limit) + } + }; + let max_threads = settings.get_max_threads()? as usize; + let op = StageTable::get_op(&self.table_info.stage_info)?; + let fmt = self.table_info.stage_info.file_format_params.clone(); let uuid = uuid::Uuid::new_v4().to_string(); let group_id = AtomicUsize::new(0); - - // parallel compact unload, the partial block will flush into next operator - if !single && pipeline.output_len() > 1 { - pipeline.add_transform(|input, output| { - let gid = group_id.fetch_add(1, std::sync::atomic::Ordering::Relaxed); - StageTableSink::try_create( - input, - ctx.clone(), - self.table_info.clone(), - op.clone(), - Some(output), - uuid.clone(), - gid, - ) - })?; - } - - // final compact unload - pipeline.try_resize(1)?; - - // Add sink pipe. - pipeline.add_sink(|input| { - let gid = group_id.fetch_add(1, std::sync::atomic::Ordering::Relaxed); - StageTableSink::try_create( - input, + match fmt { + FileFormatParams::Parquet(_) => append_data_to_parquet_files( + pipeline, ctx.clone(), self.table_info.clone(), - op.clone(), - None, - uuid.clone(), - gid, - ) - }) + op, + max_file_size, + max_threads, + uuid, + &group_id, + )?, + _ => append_data_to_row_based_files( + pipeline, + ctx.clone(), + self.table_info.clone(), + op, + max_file_size, + max_threads, + uuid, + &group_id, + )?, + }; + Ok(()) } // Truncate the stage file. @@ -283,3 +291,30 @@ impl Table for StageTable { (*guard) = Some(thresholds) } } + +pub fn unload_path( + stage_table_info: &StageTableInfo, + uuid: &str, + group_id: usize, + batch_id: usize, +) -> String { + let format_name = format!( + "{:?}", + stage_table_info.stage_info.file_format_params.get_type() + ) + .to_ascii_lowercase(); + + let path = &stage_table_info.files_info.path; + + if path.ends_with("data_") { + format!( + "{}{}_{:0>4}_{:0>8}.{}", + path, uuid, group_id, batch_id, format_name + ) + } else { + format!( + "{}/data_{}_{:0>4}_{:0>8}.{}", + path, uuid, group_id, batch_id, format_name + ) + } +} diff --git a/src/query/storages/stage/src/stage_table_sink.rs b/src/query/storages/stage/src/stage_table_sink.rs deleted file mode 100644 index f953c25963e7..000000000000 --- a/src/query/storages/stage/src/stage_table_sink.rs +++ /dev/null @@ -1,279 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::any::Any; -use std::sync::Arc; - -use async_trait::async_trait; -use common_catalog::plan::StageTableInfo; -use common_catalog::table_context::TableContext; -use common_exception::ErrorCode; -use common_exception::Result; -use common_expression::DataBlock; -use common_formats::output_format::OutputFormat; -use common_formats::FileFormatOptionsExt; -use common_pipeline_core::processors::port::InputPort; -use common_pipeline_core::processors::port::OutputPort; -use common_pipeline_core::processors::processor::Event; -use common_pipeline_core::processors::processor::ProcessorPtr; -use common_pipeline_core::processors::Processor; -use opendal::Operator; - -#[derive(Debug)] -enum State { - None, - NeedSerialize(DataBlock), - NeedWrite(Vec, Option), - Finished, -} - -pub struct StageTableSink { - state: State, - input: Arc, - data_accessor: Operator, - output: Option>, - - table_info: StageTableInfo, - working_buffer: Vec, - working_datablocks: Vec, - output_format: Box, - write_header: bool, - - uuid: String, - group_id: usize, - batch_id: usize, - - single: bool, - max_file_size: usize, -} - -impl StageTableSink { - #[allow(clippy::too_many_arguments)] - pub fn try_create( - input: Arc, - ctx: Arc, - table_info: StageTableInfo, - data_accessor: Operator, - output: Option>, - - uuid: String, - group_id: usize, - ) -> Result { - let mut options_ext = - FileFormatOptionsExt::create_from_settings(&ctx.get_settings(), false)?; - let output_format = options_ext.get_output_format( - table_info.schema(), - table_info.stage_info.file_format_params.clone(), - )?; - - let max_file_size = Self::adjust_max_file_size(&ctx, &table_info)?; - let single = table_info.stage_info.copy_options.single; - - Ok(ProcessorPtr::create(Box::new(StageTableSink { - input, - data_accessor, - table_info, - state: State::None, - output, - single, - output_format, - working_buffer: Vec::with_capacity((max_file_size as f64 * 1.2) as usize), - working_datablocks: vec![], - write_header: false, - - uuid, - group_id, - batch_id: 0, - max_file_size, - }))) - } - - fn adjust_max_file_size( - ctx: &Arc, - stage_info: &StageTableInfo, - ) -> Result { - // 256M per file by default. - const DEFAULT_SIZE: usize = 256 * 1024 * 1024; - // max is half of the max memory usage. - let max_size = (ctx.get_settings().get_max_memory_usage()? / 2) as usize; - - let mut max_file_size = stage_info.stage_info.copy_options.max_file_size; - if max_file_size == 0 { - max_file_size = DEFAULT_SIZE; - } else if max_file_size > max_size { - max_file_size = max_size - } - - Ok(max_file_size) - } - - pub fn unload_path(&self) -> String { - let format_name = format!( - "{:?}", - self.table_info.stage_info.file_format_params.get_type() - ); - - // assert_eq!("00000110", format!("{:0>8}", "110")) - if self.table_info.files_info.path.ends_with("data_") { - format!( - "{}{}_{:0>4}_{:0>8}.{}", - self.table_info.files_info.path, - self.uuid, - self.group_id, - self.batch_id, - format_name.to_ascii_lowercase() - ) - } else { - format!( - "{}/data_{}_{:0>4}_{:0>8}.{}", - self.table_info.files_info.path, - self.uuid, - self.group_id, - self.batch_id, - format_name.to_ascii_lowercase() - ) - } - } -} - -#[async_trait] -impl Processor for StageTableSink { - fn name(&self) -> String { - "StageSink".to_string() - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if matches!(&self.state, State::NeedSerialize(_)) { - return Ok(Event::Sync); - } - - if matches!(&self.state, State::NeedWrite(_, _)) { - return Ok(Event::Async); - } - - if self.input.is_finished() { - if self.output_format.buffer_size() > 0 { - let bs = self.output_format.finalize()?; - self.working_buffer.extend_from_slice(&bs); - } - let data = std::mem::take(&mut self.working_buffer); - if data.len() >= self.max_file_size || (!data.is_empty() && self.output.is_none()) { - self.state = State::NeedWrite(data, None); - self.working_datablocks.clear(); - return Ok(Event::Async); - } - - match (&self.output, self.working_datablocks.is_empty()) { - (Some(output), false) => { - if output.can_push() { - let block = self.working_datablocks.pop().unwrap(); - output.push_data(Ok(block)); - } - return Ok(Event::NeedConsume); - } - _ => { - self.state = State::Finished; - if let Some(output) = self.output.as_mut() { - output.finish() - } - return Ok(Event::Finished); - } - } - } - - if !self.input.has_data() { - self.input.set_need_data(); - return Ok(Event::NeedData); - } - - self.state = State::NeedSerialize(self.input.pull_data().unwrap()?); - Ok(Event::Sync) - } - - fn process(&mut self) -> Result<()> { - match std::mem::replace(&mut self.state, State::None) { - State::NeedSerialize(datablock) => { - if !self.write_header { - let prefix = self.output_format.serialize_prefix()?; - self.working_buffer.extend_from_slice(&prefix); - self.write_header = true; - } - - if !self.single { - for i in (0..datablock.num_rows()).step_by(1024) { - let end = (i + 1024).min(datablock.num_rows()); - let small_block = datablock.slice(i..end); - - let bs = self.output_format.serialize_block(&small_block)?; - self.working_buffer.extend_from_slice(bs.as_slice()); - - if self.working_buffer.len() + self.output_format.buffer_size() - >= self.max_file_size - { - let bs = self.output_format.finalize()?; - self.working_buffer.extend_from_slice(&bs); - - let data = std::mem::take(&mut self.working_buffer); - self.working_datablocks.clear(); - if end != datablock.num_rows() { - let remain = datablock.slice(end..datablock.num_rows()); - self.state = State::NeedWrite(data, Some(remain)); - } else { - self.state = State::NeedWrite(data, None); - } - return Ok(()); - } - } - } else { - let bs = self.output_format.serialize_block(&datablock)?; - self.working_buffer.extend_from_slice(bs.as_slice()); - } - - // hold this datablock - if self.output.is_some() { - self.working_datablocks.push(datablock); - } - } - _state => { - return Err(ErrorCode::Internal("Unknown state for stage table sink.")); - } - } - Ok(()) - } - - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - match std::mem::replace(&mut self.state, State::None) { - State::NeedWrite(bytes, remaining_block) => { - let path = self.unload_path(); - - self.data_accessor.write(&path, bytes).await?; - - match remaining_block { - Some(block) => self.state = State::NeedSerialize(block), - None => self.state = State::None, - } - self.batch_id += 1; - Ok(()) - } - _state => { - return Err(ErrorCode::Internal("Unknown state for stage table sink.")); - } - } - } -} diff --git a/src/query/storages/system/src/backtrace_table.rs b/src/query/storages/system/src/backtrace_table.rs new file mode 100644 index 000000000000..9257a84c7088 --- /dev/null +++ b/src/query/storages/system/src/backtrace_table.rs @@ -0,0 +1,77 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use common_base::dump_backtrace; +use common_catalog::table::Table; +use common_catalog::table_context::TableContext; +use common_exception::Result; +use common_expression::types::DataType; +use common_expression::ColumnBuilder; +use common_expression::DataBlock; +use common_expression::Scalar; +use common_expression::TableDataType; +use common_expression::TableField; +use common_expression::TableSchemaRefExt; +use common_meta_app::schema::TableIdent; +use common_meta_app::schema::TableInfo; +use common_meta_app::schema::TableMeta; + +use crate::SyncOneBlockSystemTable; +use crate::SyncSystemTable; + +pub struct BacktraceTable { + table_info: TableInfo, +} + +#[async_trait::async_trait] +impl SyncSystemTable for BacktraceTable { + const NAME: &'static str = "system.backtrace"; + + fn get_table_info(&self) -> &TableInfo { + &self.table_info + } + + fn get_full_data(&self, _ctx: Arc) -> Result { + let stack = dump_backtrace(false); + + let mut stacks = ColumnBuilder::with_capacity(&DataType::String, 1); + stacks.push(Scalar::String(stack.as_bytes().to_vec()).as_ref()); + + Ok(DataBlock::new_from_columns(vec![stacks.build()])) + } +} + +impl BacktraceTable { + pub fn create(table_id: u64) -> Arc { + let schema = + TableSchemaRefExt::create(vec![TableField::new("stack", TableDataType::String)]); + + let table_info = TableInfo { + desc: "'system'.'backtrace'".to_string(), + name: "backtrace".to_string(), + ident: TableIdent::new(table_id, 0), + meta: TableMeta { + schema, + engine: "SystemBacktrace".to_string(), + + ..Default::default() + }, + ..Default::default() + }; + + SyncOneBlockSystemTable::create(BacktraceTable { table_info }) + } +} diff --git a/src/query/storages/system/src/lib.rs b/src/query/storages/system/src/lib.rs index 66d510d86ff7..67b83f71b411 100644 --- a/src/query/storages/system/src/lib.rs +++ b/src/query/storages/system/src/lib.rs @@ -20,6 +20,7 @@ extern crate core; mod background_jobs_table; mod background_tasks_table; +mod backtrace_table; mod build_options_table; mod caches_table; mod catalogs_table; @@ -54,6 +55,7 @@ mod util; pub use background_jobs_table::BackgroundJobTable; pub use background_tasks_table::BackgroundTaskTable; +pub use backtrace_table::BacktraceTable; pub use build_options_table::BuildOptionsTable; pub use caches_table::CachesTable; pub use catalogs_table::CatalogsTable; diff --git a/src/query/storages/system/src/metrics_table.rs b/src/query/storages/system/src/metrics_table.rs index 5bb260b1ff2c..ea6f77bb3828 100644 --- a/src/query/storages/system/src/metrics_table.rs +++ b/src/query/storages/system/src/metrics_table.rs @@ -28,8 +28,8 @@ use common_expression::TableSchemaRefExt; use common_meta_app::schema::TableIdent; use common_meta_app::schema::TableInfo; use common_meta_app::schema::TableMeta; +use common_metrics::reset_metrics; use common_metrics::MetricValue; -use common_storages_fuse::metrics_reset; use crate::SyncOneBlockSystemTable; use crate::SyncSystemTable; @@ -71,7 +71,11 @@ impl SyncSystemTable for MetricsTable { } fn truncate(&self, _ctx: Arc) -> Result<()> { - metrics_reset(); + let prometheus_handle = common_metrics::try_handle().ok_or_else(|| { + ErrorCode::InitPrometheusFailure("Prometheus recorder is not initialized yet.") + })?; + + reset_metrics(prometheus_handle)?; Ok(()) } } diff --git a/tests/data/variant.parquet b/tests/data/variant.parquet new file mode 100644 index 000000000000..915dde8b9c83 Binary files /dev/null and b/tests/data/variant.parquet differ diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history index 334d90d6f6f4..e42c1b884b38 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0006_func_fuse_history @@ -48,6 +48,38 @@ select block_size from fuse_block('db_09_0006', 't') order by block_size 8 16 +statement ok +create table t1(a int) row_per_block=3 + +statement ok +insert into t1 select number from numbers(9) + +statement ok +insert into t1 values(9),(10) + +query I +select count() from fuse_block('db_09_0006', 't1') +---- +4 + +query I +select count() from fuse_segment('db_09_0006', 't1') +---- +2 + +statement ok +insert into t1 values(11),(12) + +query I +select count() from fuse_segment('db_09_0006', 't1') +---- +3 + +query I +select count() from fuse_block('db_09_0006', 't1') +---- +5 + statement error 1025 select * from fuse_snapshot('db_09_0006', 'not_exist') @@ -72,6 +104,9 @@ select * from fuse_snapshot('db_09_0006', 't_in_memory') statement ok DROP TABLE t +statement ok +DROP TABLE t1 + statement ok DROP TABLE t_in_memory diff --git a/tests/sqllogictests/suites/base/20+_others/20_0010_expression_string_like b/tests/sqllogictests/suites/base/20+_others/20_0010_expression_string_like index 36bcbc256dac..616cedf8a78e 100644 --- a/tests/sqllogictests/suites/base/20+_others/20_0010_expression_string_like +++ b/tests/sqllogictests/suites/base/20+_others/20_0010_expression_string_like @@ -5,7 +5,7 @@ statement ok CREATE TABLE t_string_like(URL TEXT NOT NULL) statement ok -insert into t_string_like values('Arrow'), (''), ('Nicolas') +insert into t_string_like values('Arrow'), (''), ('Nicolas'), ('Databend'), ('modern data warehouse') query B select count(*)=1 from t_string_like where URL like 'Arrow' @@ -57,9 +57,83 @@ select count(*)=0 from t_string_like where URL like '%not_exist%' ---- 1 +query B +select count(*)=1 from t_string_like where URL like 'Data%bend' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like '%Data%bend' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like 'Data%bend%' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like '%Data%bend%' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like 'Datx%bend' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like '%Datx%bend' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like 'Datx%bend%' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like '%Datx%bend%' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like '%%D%%%a%%t%a%%%b%e%n%%d%%' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like '%%D%%%a%%t%a%%%x%e%n%%d%%' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like '%%D%%%a%%t%a%%%x%e%n%%d%%' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like 'modern%data%warehouse' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like 'modern%warehouse' +---- +1 + +query B +select count(*)=0 from t_string_like where URL like 'modern%warehouse%data' +---- +1 + +query B +select count(*)=1 from t_string_like where URL like '%%modern %%%dat%a%war%e%h%%ou%s%e%%' +---- +1 # test for empty string like '%%', we follow mysql/pg returns 1, but clickhouse returns 0 query B -select count(*)=3 from t_string_like where URL like '%%' +select count(*)=5 from t_string_like where URL like '%%' ---- 1 diff --git a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_agg_index_base b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_agg_index_base index bf172f0e8594..3ae014e9955e 100644 --- a/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_agg_index_base +++ b/tests/sqllogictests/suites/ee/02_ee_aggregating_index/02_0000_agg_index_base @@ -12,7 +12,7 @@ statement ok DROP AGGREGATING INDEX IF EXISTS testi; statement ok -CREATE TABLE t (a int, b int, c int) storage_format = 'parquet' +CREATE TABLE t (a int, b int, c int) statement ok INSERT INTO t VALUES (1,1,4), (1,2,1), (1,2,4), (2,2,5) @@ -188,7 +188,7 @@ statement ok DROP TABLE t statement ok -CREATE TABLE t (a int, b int, c int) storage_format = 'parquet' +CREATE TABLE t (a int, b int, c int) statement ok INSERT INTO t VALUES (1,1,4), (1,2,1), (1,2,4), (2,2,5) diff --git a/tests/sqllogictests/suites/mode/cluster/distributed_copy_into_table2_execption.test b/tests/sqllogictests/suites/mode/cluster/distributed_copy_into_table2_execption.test index 2ae4277d7669..4e08cf9f0843 100644 --- a/tests/sqllogictests/suites/mode/cluster/distributed_copy_into_table2_execption.test +++ b/tests/sqllogictests/suites/mode/cluster/distributed_copy_into_table2_execption.test @@ -49,13 +49,13 @@ copy into @s1 from (select a,b,c from table_random limit 1000000); statement ok copy into @s1 from (select a,b from table_random2 limit 1000000); -statement error 2000 +statement error 1046 copy into products from @s1 pattern = '.*[.]csv' purge = true; query I select count(*) from products; ---- -0 +8 statement ok select block_count from fuse_snapshot('default','products'); diff --git a/tests/sqllogictests/suites/mode/standalone/ee/explain b/tests/sqllogictests/suites/mode/standalone/ee/explain index 2b81b4f2c290..034708785e5e 100644 --- a/tests/sqllogictests/suites/mode/standalone/ee/explain +++ b/tests/sqllogictests/suites/mode/standalone/ee/explain @@ -168,7 +168,7 @@ EvalScalar ├── filters: [is_true(SUM(a) (#16) = scalar_subquery_15 (#15))] ├── estimated rows: 0.00 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [b (#13)] ├── probe keys: [b (#5)] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain/limit.test b/tests/sqllogictests/suites/mode/standalone/explain/limit.test index f6c6dd9fa13e..cbf007d287b0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/limit.test @@ -105,7 +105,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = if(CAST(is_not_null(scalar_subquery_5 (#5)) AS Boolean NULL), scalar_subquery_5 (#5), 0))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test index 96b3e28f52a6..abaaec100b28 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/prune_column.test @@ -126,7 +126,7 @@ EvalScalar ├── filters: [is_true(CAST(t1.a (#1) AS UInt64 NULL) = scalar_subquery_13 (#13))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [] ├── probe keys: [] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain/subquery.test b/tests/sqllogictests/suites/mode/standalone/explain/subquery.test index 89d51a5f96a3..6e1272790dc8 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain/subquery.test +++ b/tests/sqllogictests/suites/mode/standalone/explain/subquery.test @@ -8,7 +8,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = if(CAST(is_not_null(scalar_subquery_5 (#5)) AS Boolean NULL), scalar_subquery_5 (#5), 0))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] @@ -161,7 +161,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = scalar_subquery_1 (#1))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [] ├── probe keys: [] ├── filters: [] @@ -397,7 +397,7 @@ EvalScalar ├── filters: [is_true(try_to_boolean(if(CAST(is_not_null(scalar_subquery_4 (#4)) AS Boolean NULL), TRY_CAST(scalar_subquery_4 (#4) AS UInt64 NULL), 0)))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test b/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test index f6c6dd9fa13e..cbf007d287b0 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/limit.test @@ -105,7 +105,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = if(CAST(is_not_null(scalar_subquery_5 (#5)) AS Boolean NULL), scalar_subquery_5 (#5), 0))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test index 7a657ba72b29..2e63da0560fb 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/prune_column.test @@ -126,7 +126,7 @@ EvalScalar ├── filters: [is_true(CAST(t1.a (#1) AS UInt64 NULL) = scalar_subquery_13 (#13))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [] ├── probe keys: [] ├── filters: [] diff --git a/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test b/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test index 89d51a5f96a3..6e1272790dc8 100644 --- a/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test +++ b/tests/sqllogictests/suites/mode/standalone/explain_native/subquery.test @@ -8,7 +8,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = if(CAST(is_not_null(scalar_subquery_5 (#5)) AS Boolean NULL), scalar_subquery_5 (#5), 0))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] @@ -161,7 +161,7 @@ EvalScalar ├── filters: [is_true(CAST(t.number (#0) AS UInt64 NULL) = scalar_subquery_1 (#1))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [] ├── probe keys: [] ├── filters: [] @@ -397,7 +397,7 @@ EvalScalar ├── filters: [is_true(try_to_boolean(if(CAST(is_not_null(scalar_subquery_4 (#4)) AS Boolean NULL), TRY_CAST(scalar_subquery_4 (#4) AS UInt64 NULL), 0)))] ├── estimated rows: 0.20 └── HashJoin - ├── join type: SINGLE + ├── join type: LEFT SINGLE ├── build keys: [number (#2)] ├── probe keys: [number (#0)] ├── filters: [] diff --git a/tests/sqllogictests/suites/tpch/queries.test b/tests/sqllogictests/suites/tpch/queries.test index 02a6e95d5c04..90dc2ec43b62 100644 --- a/tests/sqllogictests/suites/tpch/queries.test +++ b/tests/sqllogictests/suites/tpch/queries.test @@ -1162,7 +1162,7 @@ order by s_name, p_partkey; ---- -HashJoin: SINGLE +HashJoin: RIGHT SINGLE ├── Build │ └── HashJoin: INNER │ ├── Build @@ -1176,7 +1176,11 @@ HashJoin: SINGLE │ │ └── Probe │ │ └── Scan: default.tpch.supplier, rows: 1000 │ └── Probe -│ └── Scan: default.tpch.partsupp, rows: 80000 +│ └── HashJoin: INNER +│ ├── Build +│ │ └── Scan: default.tpch.part, rows: 20000 +│ └── Probe +│ └── Scan: default.tpch.partsupp, rows: 80000 └── Probe └── HashJoin: INNER ├── Build @@ -1190,11 +1194,7 @@ HashJoin: SINGLE │ └── Probe │ └── Scan: default.tpch.supplier, rows: 1000 └── Probe - └── HashJoin: INNER - ├── Build - │ └── Scan: default.tpch.part, rows: 20000 - └── Probe - └── Scan: default.tpch.partsupp, rows: 80000 + └── Scan: default.tpch.partsupp, rows: 80000 # Q3 query I @@ -1594,7 +1594,7 @@ group by order by value desc limit 100; ---- -HashJoin: SINGLE +HashJoin: LEFT SINGLE ├── Build │ └── HashJoin: INNER │ ├── Build @@ -1740,7 +1740,7 @@ where order by s_suppkey; ---- -HashJoin: SINGLE +HashJoin: LEFT SINGLE ├── Build │ └── Scan: default.tpch.lineitem, rows: 600572 └── Probe @@ -1814,15 +1814,15 @@ where l_partkey = p_partkey ); ---- -HashJoin: SINGLE +HashJoin: RIGHT SINGLE ├── Build -│ └── Scan: default.tpch.lineitem, rows: 600572 +│ └── HashJoin: INNER +│ ├── Build +│ │ └── Scan: default.tpch.part, rows: 20000 +│ └── Probe +│ └── Scan: default.tpch.lineitem, rows: 600572 └── Probe - └── HashJoin: INNER - ├── Build - │ └── Scan: default.tpch.part, rows: 20000 - └── Probe - └── Scan: default.tpch.lineitem, rows: 600572 + └── Scan: default.tpch.lineitem, rows: 600572 #Q18 query I @@ -1999,7 +1999,7 @@ HashJoin: RIGHT SEMI │ └── Probe │ └── Scan: default.tpch.supplier, rows: 1000 └── Probe - └── HashJoin: SINGLE + └── HashJoin: LEFT SINGLE ├── Build │ └── Scan: default.tpch.lineitem, rows: 600572 └── Probe @@ -2115,7 +2115,7 @@ order by ---- HashJoin: RIGHT ANTI ├── Build -│ └── HashJoin: SINGLE +│ └── HashJoin: LEFT SINGLE │ ├── Build │ │ └── Scan: default.tpch.customer, rows: 15000 │ └── Probe diff --git a/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.result b/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.result new file mode 100644 index 000000000000..2e0eed92ce24 --- /dev/null +++ b/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.result @@ -0,0 +1,30 @@ +---csv +1 +20 +---csv_single +1 +20 +---csv_10 +10 +20 +---csv_20 +5 +20 +---parq +1 +20 +---parq_single +1 +20 +---parq_40 +10 +20 +---parq_80 +5 +20 +---csv_big_20 +2 +2000 +---parq_big_80 +1 +2000 diff --git a/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.sh b/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.sh new file mode 100755 index 000000000000..dd1fedfc6394 --- /dev/null +++ b/tests/suites/1_stateful/00_copy/0000_0000_copy_into_stage2.sh @@ -0,0 +1,72 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +echo "drop table if exists test_table;" | $MYSQL_CLIENT_CONNECT +echo "drop STAGE if exists s2;" | $MYSQL_CLIENT_CONNECT +echo "CREATE STAGE s2;" | $MYSQL_CLIENT_CONNECT + +STAGE_DIR=/tmp/copy_into_stage2 + +rm -rf "$STAGE_DIR" + +echo "drop stage if exists s1;" | $MYSQL_CLIENT_CONNECT +echo "create stage s1 url = 'fs:///$STAGE_DIR/' FILE_FORMAT = (type = PARQUET)" | $MYSQL_CLIENT_CONNECT + +echo "CREATE TABLE test_table ( + id INTEGER, + name VARCHAR, + age INT +);" | $MYSQL_CLIENT_CONNECT + +# each insert create a block +for i in `seq 1 10`;do + echo "insert into test_table (id,name,age) values(1,'2',3), (4, '5', 6);" | $MYSQL_CLIENT_CONNECT +done + +check_csv() { + echo "---${1}" + ls "$STAGE_DIR"/${1} | wc -l | sed 's/ //g' + cat "$STAGE_DIR"/${1}/* | wc -l | sed 's/ //g' +} + +# each block create a CSV chunk of size 16 +echo "copy into @s1/csv from test_table FILE_FORMAT = (type = CSV);" | $MYSQL_CLIENT_CONNECT +check_csv "csv" +echo "copy into @s1/csv_single from test_table FILE_FORMAT = (type = CSV) single=true;" | $MYSQL_CLIENT_CONNECT +check_csv "csv_single" +echo "copy into @s1/csv_10 from test_table FILE_FORMAT = (type = CSV) MAX_FILE_SIZE = 10;" | $MYSQL_CLIENT_CONNECT +check_csv "csv_10" +echo "copy into @s1/csv_20 from test_table FILE_FORMAT = (type = CSV) MAX_FILE_SIZE = 20;" | $MYSQL_CLIENT_CONNECT +check_csv "csv_20" + +check_parq() { + echo "---${1}" + ls "$STAGE_DIR"/${1} | wc -l | sed 's/ //g' + echo "select count(*) from @s1/${1}/ (FILE_FORMAT => 'PARQUET');" | $MYSQL_CLIENT_CONNECT +} + +# each block memory size is 42 +echo "copy into @s1/parq from test_table FILE_FORMAT = (type = PARQUET);" | $MYSQL_CLIENT_CONNECT +check_parq "parq" +echo "copy into @s1/parq_single from test_table FILE_FORMAT = (type = PARQUET) single=true;" | $MYSQL_CLIENT_CONNECT +check_parq "parq_single" +echo "copy into @s1/parq_40 from test_table FILE_FORMAT = (type = PARQUET) MAX_FILE_SIZE = 40;" | $MYSQL_CLIENT_CONNECT +check_parq "parq_40" +echo "copy into @s1/parq_80 from test_table FILE_FORMAT = (type = PARQUET) MAX_FILE_SIZE = 80;" | $MYSQL_CLIENT_CONNECT +check_parq "parq_80" + +# test big block +echo "drop table if exists t1;" | $MYSQL_CLIENT_CONNECT +echo "create table t1 (a int);" | $MYSQL_CLIENT_CONNECT + +## CSV slice block by 1024, so we should get 2 CSV files but one parquet file. +for i in $(seq 1 2000);do + echo "$i" >> "$STAGE_DIR"/big.csv +done +echo "copy into t1 from @s1/big.csv FILE_FORMAT = (type = CSV);" | $MYSQL_CLIENT_CONNECT +echo "copy into @s1/csv_big_20 from t1 FILE_FORMAT = (type = CSV) MAX_FILE_SIZE = 20;" | $MYSQL_CLIENT_CONNECT +check_csv "csv_big_20" +echo "copy into @s1/parq_big_80 from t1 FILE_FORMAT = (type = PARQUET) MAX_FILE_SIZE = 80;" | $MYSQL_CLIENT_CONNECT +check_parq "parq_big_80" \ No newline at end of file diff --git a/tests/suites/1_stateful/02_query/02_0004_attach_table.result b/tests/suites/1_stateful/02_query/02_0004_attach_table.result new file mode 100644 index 000000000000..327e073676e8 --- /dev/null +++ b/tests/suites/1_stateful/02_query/02_0004_attach_table.result @@ -0,0 +1,5 @@ +0 +1 +2 +0 +2 diff --git a/tests/suites/1_stateful/02_query/02_0004_attach_table.sh b/tests/suites/1_stateful/02_query/02_0004_attach_table.sh new file mode 100755 index 000000000000..b2818a093034 --- /dev/null +++ b/tests/suites/1_stateful/02_query/02_0004_attach_table.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +echo "drop table if exists table_from;" | $MYSQL_CLIENT_CONNECT +echo "drop table if exists table_to;" | $MYSQL_CLIENT_CONNECT + +## Create table +echo "create table table_from(a int) 's3://testbucket/admin/data/' connection=(aws_key_id='minioadmin' aws_secret_key='minioadmin' endpoint_url='${STORAGE_S3_ENDPOINT_URL}');" | $MYSQL_CLIENT_CONNECT + +table_inserts=( + "insert into table_from(a) values(0)" + "insert into table_from(a) values(1)" + "insert into table_from(a) values(2)" +) + +for i in "${table_inserts[@]}"; do + echo "$i" | $MYSQL_CLIENT_CONNECT +done + +storage_prefix=$(mysql -uroot -h127.0.0.1 -P3307 -e "set global hide_options_in_show_create_table=0;show create table table_from" | grep -i snapshot_location | awk -F'SNAPSHOT_LOCATION='"'"'|_ss' '{print $2}') + +echo "attach table table_to 's3://testbucket/admin/data/$storage_prefix' connection=(aws_key_id='minioadmin' aws_secret_key='minioadmin' endpoint_url='${STORAGE_S3_ENDPOINT_URL}');" | $MYSQL_CLIENT_CONNECT + + +# ## Select table +echo "select * from table_to order by a;" | $MYSQL_CLIENT_CONNECT + +echo "delete from table_to where a=1;" | $MYSQL_CLIENT_CONNECT + +echo "select * from table_to order by a;" | $MYSQL_CLIENT_CONNECT diff --git a/tests/suites/1_stateful/05_formats/05_05_parquet/05_05_01_parquet_load_unload.sh b/tests/suites/1_stateful/05_formats/05_05_parquet/05_05_01_parquet_load_unload.sh index 1aa8504b7f57..2152239e4d24 100755 --- a/tests/suites/1_stateful/05_formats/05_05_parquet/05_05_01_parquet_load_unload.sh +++ b/tests/suites/1_stateful/05_formats/05_05_parquet/05_05_01_parquet_load_unload.sh @@ -9,12 +9,13 @@ echo "CREATE TABLE test_load_unload ( a VARCHAR NULL, b float, - e timestamp + e timestamp, + f variant );" | $MYSQL_CLIENT_CONNECT insert_data() { echo "insert into test_load_unload values - ('a\"b', 1, '2044-05-06T03:25:02.868894-07:00') + ('a\"b', 1, '2044-05-06T03:25:02.868894-07:00', '{\"k1\":\"v\",\"k2\":[1,2]}') " | $MYSQL_CLIENT_CONNECT } @@ -24,24 +25,43 @@ test_format() { # unload clickhouse curl -s -u root: -XPOST "http://localhost:${QUERY_CLICKHOUSE_HTTP_HANDLER_PORT}" \ - -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload.txt + -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload.parquet echo "truncate table test_load_unload" | $MYSQL_CLIENT_CONNECT - curl -s -u root: -XPOST "http://localhost:${QUERY_CLICKHOUSE_HTTP_HANDLER_PORT}" \ - -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload.txt - # load streaming curl -sH "insert_sql:insert into test_load_unload file_format = (type = ${1})" \ - -F "upload=@/tmp/test_load_unload.txt" \ + -F "upload=@/tmp/test_load_unload.parquet" \ -u root: -XPUT "http://localhost:${QUERY_HTTP_HANDLER_PORT}/v1/streaming_load" | grep -c "SUCCESS" # unload clickhouse again curl -s -u root: -XPOST "http://localhost:${QUERY_CLICKHOUSE_HTTP_HANDLER_PORT}" \ - -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload2.txt + -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload2.parquet + + echo "truncate table test_load_unload" | $MYSQL_CLIENT_CONNECT + + # copy into table + echo "copy into test_load_unload from 'fs:///tmp/test_load_unload.parquet' file_format = (type = ${1});" | $MYSQL_CLIENT_CONNECT + + # unload clickhouse again + curl -s -u root: -XPOST "http://localhost:${QUERY_CLICKHOUSE_HTTP_HANDLER_PORT}" \ + -d "select * from test_load_unload FORMAT ${1}" > /tmp/test_load_unload3.parquet + + # copy into stage + rm -rf /tmp/test_load_unload_fs + echo "drop stage if exists data_fs;" | $MYSQL_CLIENT_CONNECT + echo "create stage data_fs url = 'fs:///tmp/test_load_unload_fs/' FILE_FORMAT = (type = ${1});" | $MYSQL_CLIENT_CONNECT + echo "copy into @data_fs from test_load_unload file_format = (type = ${1});" | $MYSQL_CLIENT_CONNECT + + # unload clickhouse again from stage + curl -s -u root: -XPOST "http://localhost:${QUERY_CLICKHOUSE_HTTP_HANDLER_PORT}" \ + -d "select * from @data_fs FORMAT ${1}" > /tmp/test_load_unload4.parquet - diff /tmp/test_load_unload2.txt /tmp/test_load_unload.txt - rm /tmp/test_load_unload2.txt /tmp/test_load_unload.txt + diff /tmp/test_load_unload2.parquet /tmp/test_load_unload.parquet + diff /tmp/test_load_unload3.parquet /tmp/test_load_unload.parquet + diff /tmp/test_load_unload4.parquet /tmp/test_load_unload.parquet + rm /tmp/test_load_unload2.parquet /tmp/test_load_unload.parquet + rm /tmp/test_load_unload4.parquet /tmp/test_load_unload3.parquet echo "truncate table test_load_unload" | $MYSQL_CLIENT_CONNECT } diff --git a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.result b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.result index 734c14fb981a..8989358cb978 100755 --- a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.result +++ b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.result @@ -42,3 +42,6 @@ yy__version INT 1 27 yy__us_core_race VARCHAR 1 28 yy__us_core_ethnicity VARCHAR 1 29 yy__us_core_birthsex TUPLE(VALUECODE STRING,) 1 30 +--- variant file: +a INT 0 0 +b VARIANT 0 1 diff --git a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.sh b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.sh index 4823b57aa92a..19e3ccc012b2 100755 --- a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.sh +++ b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_04_infer_schema.sh @@ -38,3 +38,8 @@ echo "--- complex:" cp "$CURDIR"/../../../../data/complex.parquet ${DATADIR_PATH}/data/complex.parquet echo "select * from infer_schema(location => '@s2/data/', pattern => 'complex.*');" | $MYSQL_CLIENT_CONNECT +cp "$CURDIR"/../../../../data/variant.parquet ${DATADIR_PATH}/data + +echo "--- variant file:" +echo "select * from infer_schema(location => '@s2/data/variant.parquet');" | $MYSQL_CLIENT_CONNECT + diff --git a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.result b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.result new file mode 100644 index 000000000000..d00491fd7e5b --- /dev/null +++ b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.result @@ -0,0 +1 @@ +1 diff --git a/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.sh b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.sh new file mode 100755 index 000000000000..a41683d6d666 --- /dev/null +++ b/tests/suites/1_stateful/08_select_stage/08_00_parquet/08_00_11_diff_schema.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../../shell_env.sh +DATADIR_PATH="$CURDIR/../../../../data/" + +echo "drop stage if exists data;" | $MYSQL_CLIENT_CONNECT +echo "create stage data url = 'fs://$DATADIR_PATH' " | $MYSQL_CLIENT_CONNECT +echo "select \$1 from @data/ (files=>('mytime.parquet', 'complex.parquet'))" | $MYSQL_CLIENT_CONNECT 2>&1 | grep -c "diff schema" diff --git a/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.result b/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.result new file mode 100644 index 000000000000..3a551d1032ad --- /dev/null +++ b/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.result @@ -0,0 +1,6 @@ +1 +2 +3 +4 +888 +1024 diff --git a/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.sh b/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.sh new file mode 100755 index 000000000000..b461cb3c8e96 --- /dev/null +++ b/tests/suites/5_ee/01_vacuum/01_0002_ee_vacuum_drop_table.sh @@ -0,0 +1,81 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +## Setup +echo "drop database if exists test_vacuum_drop" | $MYSQL_CLIENT_CONNECT +echo "drop database if exists test_vacuum_drop_2" | $MYSQL_CLIENT_CONNECT +echo "drop database if exists test_vacuum_drop_3" | $MYSQL_CLIENT_CONNECT + +echo "CREATE DATABASE test_vacuum_drop" | $MYSQL_CLIENT_CONNECT +echo "create table test_vacuum_drop.a(c int)" | $MYSQL_CLIENT_CONNECT + +echo "INSERT INTO test_vacuum_drop.a VALUES (1)" | $MYSQL_CLIENT_CONNECT + +echo "select * from test_vacuum_drop.a" | $MYSQL_CLIENT_CONNECT + +echo "drop table test_vacuum_drop.a" | $MYSQL_CLIENT_CONNECT + +echo "vacuum drop table from test_vacuum_drop retain 0 hours" | $MYSQL_CLIENT_CONNECT + +echo "create table test_vacuum_drop.b(c int)" | $MYSQL_CLIENT_CONNECT + +echo "INSERT INTO test_vacuum_drop.b VALUES (2)" | $MYSQL_CLIENT_CONNECT + +echo "drop table test_vacuum_drop.b" | $MYSQL_CLIENT_CONNECT + +echo "vacuum drop table from test_vacuum_drop" | $MYSQL_CLIENT_CONNECT + +echo "undrop table test_vacuum_drop.b" | $MYSQL_CLIENT_CONNECT + +# test_vacuum_drop.b has not been vacuum, MUST return [2] +echo "select * from test_vacuum_drop.b" | $MYSQL_CLIENT_CONNECT + +echo "CREATE DATABASE test_vacuum_drop_2" | $MYSQL_CLIENT_CONNECT +echo "create table test_vacuum_drop_2.a(c int)" | $MYSQL_CLIENT_CONNECT + +echo "INSERT INTO test_vacuum_drop_2.a VALUES (3)" | $MYSQL_CLIENT_CONNECT + +echo "CREATE DATABASE test_vacuum_drop_3" | $MYSQL_CLIENT_CONNECT +echo "create table test_vacuum_drop_3.a(c int)" | $MYSQL_CLIENT_CONNECT + +echo "INSERT INTO test_vacuum_drop_3.a VALUES (4)" | $MYSQL_CLIENT_CONNECT + +echo "select * from test_vacuum_drop_2.a" | $MYSQL_CLIENT_CONNECT +echo "select * from test_vacuum_drop_3.a" | $MYSQL_CLIENT_CONNECT + +echo "drop database test_vacuum_drop_2" | $MYSQL_CLIENT_CONNECT +echo "drop table test_vacuum_drop_3.a" | $MYSQL_CLIENT_CONNECT + +# vacuum without [from db] will vacuum all tables, including tables in drop db +echo "vacuum drop table retain 0 hours" | $MYSQL_CLIENT_CONNECT + +echo "drop database if exists test_vacuum_drop" | $MYSQL_CLIENT_CONNECT +echo "drop database if exists test_vacuum_drop_2" | $MYSQL_CLIENT_CONNECT +echo "drop database if exists test_vacuum_drop_3" | $MYSQL_CLIENT_CONNECT + +# test external table +echo "drop table if exists table_drop_external_location;" | $MYSQL_CLIENT_CONNECT + +## Create table +echo "create table table_drop_external_location(a int) 's3://testbucket/admin/data/' connection=(aws_key_id='minioadmin' aws_secret_key='minioadmin' endpoint_url='${STORAGE_S3_ENDPOINT_URL}');" | $MYSQL_CLIENT_CONNECT + +table_inserts=( + "insert into table_drop_external_location(a) values(888)" + "insert into table_drop_external_location(a) values(1024)" +) + +for i in "${table_inserts[@]}"; do + echo "$i" | $MYSQL_CLIENT_CONNECT +done + +## Select table +echo "select * from table_drop_external_location order by a;" | $MYSQL_CLIENT_CONNECT + +echo "drop table table_drop_external_location;" | $MYSQL_CLIENT_CONNECT + +echo "vacuum drop table retain 0 hours" | $MYSQL_CLIENT_CONNECT + +## Drop table +echo "drop table if exists table_drop_external_location;" | $MYSQL_CLIENT_CONNECT