From b8eae4dc0bda637dbb46eb44d37fe5d02e643dfc Mon Sep 17 00:00:00 2001 From: sundyli <543950155@qq.com> Date: Tue, 12 Apr 2022 08:47:08 +0800 Subject: [PATCH 01/27] feat(query): support new processor in insert query --- query/src/interpreters/interpreter_insert.rs | 100 +++++++- query/src/pipelines/new/executor/mod.rs | 2 + .../executor/pipeline_complete_executor.rs | 1 + .../new/executor/pipeline_executor.rs | 1 + query/src/pipelines/new/mod.rs | 2 + query/src/pipelines/new/processors/mod.rs | 4 + .../new/processors/sinks/empty_sink.rs | 39 +++ .../src/pipelines/new/processors/sinks/mod.rs | 2 + .../new/processors/sources/blocks_source.rs | 52 ++++ .../pipelines/new/processors/sources/mod.rs | 2 + .../new/processors/transforms/mod.rs | 3 + .../processors/transforms/transform_addon.rs | 114 +++++++++ .../transforms/transform_block_compact.rs | 10 +- query/src/sessions/query_ctx.rs | 15 ++ query/src/storages/fuse/fuse_table.rs | 4 + query/src/storages/fuse/io/mod.rs | 1 + query/src/storages/fuse/io/write/mod.rs | 1 + query/src/storages/fuse/operations/append.rs | 45 ++++ .../src/storages/fuse/operations/fuse_sink.rs | 237 ++++++++++++++++++ query/src/storages/fuse/operations/mod.rs | 2 + query/src/storages/memory/memory_table.rs | 37 +++ query/src/storages/null/null_table.rs | 13 + query/src/storages/storage_table.rs | 4 + 23 files changed, 684 insertions(+), 7 deletions(-) create mode 100644 query/src/pipelines/new/processors/sinks/empty_sink.rs create mode 100644 query/src/pipelines/new/processors/sources/blocks_source.rs create mode 100644 query/src/pipelines/new/processors/transforms/transform_addon.rs create mode 100644 query/src/storages/fuse/operations/fuse_sink.rs diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 5cc987f2bb47..483c6f6f7545 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::VecDeque; use std::sync::Arc; use common_exception::ErrorCode; use common_exception::Result; +use common_infallible::Mutex; use common_meta_types::GrantObject; use common_meta_types::UserPrivilegeType; use common_planners::InsertInputSource; @@ -28,17 +30,98 @@ use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; use crate::interpreters::plan_schedulers::InsertWithPlan; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::executor::PipelineCompleteExecutor; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::BlocksSource; +use crate::pipelines::new::processors::TransformAddOn; +use crate::pipelines::new::processors::TransformDummy; +use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::transforms::AddOnStream; use crate::sessions::QueryContext; pub struct InsertInterpreter { ctx: Arc, plan: InsertPlan, + source_processor: Option, } impl InsertInterpreter { pub fn try_create(ctx: Arc, plan: InsertPlan) -> Result { - Ok(Arc::new(InsertInterpreter { ctx, plan })) + Ok(Arc::new(InsertInterpreter { + ctx, + plan, + source_processor: None, + })) + } + + async fn execute_new( + &self, + input_stream: Option, + ) -> Result { + let plan = &self.plan; + let settings = self.ctx.get_settings(); + let table = self + .ctx + .get_table(&plan.database_name, &plan.table_name) + .await?; + + let mut pipeline = self.create_new_pipeline()?; + let mut builder = SourcePipeBuilder::create(); + + match &self.plan.source { + InsertInputSource::Values(values) => { + let blocks = Arc::new(Mutex::new(VecDeque::from_iter(vec![values.block.clone()]))); + + for _index in 0..settings.get_max_threads()? { + let output = OutputPort::create(); + builder.add_source( + output.clone(), + BlocksSource::create(self.ctx.clone(), output.clone(), blocks.clone())?, + ); + } + pipeline.add_pipe(builder.finalize()); + } + InsertInputSource::StreamingWithFormat(_) => {} + InsertInputSource::SelectPlan(_) => { + // // todo!(); + // Some(BlocksSource::create(vec![])) + } + }; + + let need_fill_missing_columns = table.schema() != plan.schema(); + if need_fill_missing_columns { + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformAddOn::try_create( + transform_input_port, + transform_output_port, + self.plan.schema(), + self.plan.schema(), + self.ctx.clone(), + ) + })?; + } + + table.append2(self.ctx.clone(), &mut pipeline)?; + + let async_runtime = self.ctx.get_storage_runtime(); + + pipeline.set_max_threads(settings.get_max_threads()? as usize); + let executor = PipelineCompleteExecutor::try_create(async_runtime, pipeline)?; + executor.execute()?; + drop(executor); + + let append_entries = self.ctx.consume_precommit_blocks(); + table + .commit_insertion(self.ctx.clone(), append_entries, self.plan.overwrite) + .await?; + + Ok(Box::pin(DataBlockStream::create( + self.plan.schema(), + None, + vec![], + ))) } } @@ -52,8 +135,16 @@ impl Interpreter for InsertInterpreter { &self, mut input_stream: Option, ) -> Result { - let plan = &self.plan; + let settings = self.ctx.get_settings(); + + if true + || (settings.get_enable_new_processor_framework()? == 2 + && self.ctx.get_cluster().is_empty()) + { + return self.execute_new(input_stream).await; + } + let plan = &self.plan; self.ctx .get_current_session() .validate_privilege( @@ -128,4 +219,9 @@ impl Interpreter for InsertInterpreter { vec![], ))) } + + fn create_new_pipeline(&self) -> Result { + let new_pipeline = NewPipeline::create(); + Ok(new_pipeline) + } } diff --git a/query/src/pipelines/new/executor/mod.rs b/query/src/pipelines/new/executor/mod.rs index 1b3acfe09f6b..c01e25b12484 100644 --- a/query/src/pipelines/new/executor/mod.rs +++ b/query/src/pipelines/new/executor/mod.rs @@ -23,5 +23,7 @@ mod pipeline_pulling_executor; mod pipeline_pushing_executor; pub use executor_graph::RunningGraph; +pub use pipeline_complete_executor::PipelineCompleteExecutor; pub use pipeline_executor::PipelineExecutor; pub use pipeline_pulling_executor::PipelinePullingExecutor; +pub use pipeline_pushing_executor::PipelinePushingExecutor; diff --git a/query/src/pipelines/new/executor/pipeline_complete_executor.rs b/query/src/pipelines/new/executor/pipeline_complete_executor.rs index 1fdc788079be..da1da34468be 100644 --- a/query/src/pipelines/new/executor/pipeline_complete_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_complete_executor.rs @@ -39,6 +39,7 @@ impl PipelineCompleteExecutor { } let executor = PipelineExecutor::create(async_runtime, pipeline)?; + Ok(PipelineCompleteExecutor { executor }) } diff --git a/query/src/pipelines/new/executor/pipeline_executor.rs b/query/src/pipelines/new/executor/pipeline_executor.rs index 70ab48acfe60..b7b7c37e3adc 100644 --- a/query/src/pipelines/new/executor/pipeline_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_executor.rs @@ -52,6 +52,7 @@ impl PipelineExecutor { } global_tasks_queue.init_tasks(tasks); + Ok(Arc::new(PipelineExecutor { graph, threads_num, diff --git a/query/src/pipelines/new/mod.rs b/query/src/pipelines/new/mod.rs index c41d08a0dd64..acc117f619f0 100644 --- a/query/src/pipelines/new/mod.rs +++ b/query/src/pipelines/new/mod.rs @@ -27,6 +27,8 @@ pub mod processors; mod unsafe_cell_wrap; pub use pipe::NewPipe; +pub use pipe::SinkPipeBuilder; pub use pipe::SourcePipeBuilder; +pub use pipe::TransformPipeBuilder; pub use pipeline::NewPipeline; pub use pipeline_builder::QueryPipelineBuilder; diff --git a/query/src/pipelines/new/processors/mod.rs b/query/src/pipelines/new/processors/mod.rs index 69c5cecf54de..a8f0d8c851a9 100644 --- a/query/src/pipelines/new/processors/mod.rs +++ b/query/src/pipelines/new/processors/mod.rs @@ -30,21 +30,25 @@ pub use processor::Processors; pub use resize_processor::ResizeProcessor; pub use sinks::AsyncSink; pub use sinks::AsyncSinker; +pub use sinks::EmptySink; pub use sinks::Sink; pub use sinks::Sinker; pub use sinks::SyncSenderSink; pub use sources::AsyncSource; pub use sources::AsyncSourcer; +pub use sources::BlocksSource; pub use sources::EmptySource; pub use sources::SyncReceiverSource; pub use sources::SyncSource; pub use sources::SyncSourcer; pub use transforms::AggregatorParams; pub use transforms::AggregatorTransformParams; +pub use transforms::BlockCompactor; pub use transforms::ExpressionTransform; pub use transforms::ProjectionTransform; pub use transforms::SortMergeCompactor; pub use transforms::SubQueriesPuller; +pub use transforms::TransformAddOn; pub use transforms::TransformAggregator; pub use transforms::TransformBlockCompact; pub use transforms::TransformCompact; diff --git a/query/src/pipelines/new/processors/sinks/empty_sink.rs b/query/src/pipelines/new/processors/sinks/empty_sink.rs new file mode 100644 index 000000000000..130d60e49182 --- /dev/null +++ b/query/src/pipelines/new/processors/sinks/empty_sink.rs @@ -0,0 +1,39 @@ +// Copyright 2022 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_datablocks::DataBlock; +use common_exception::Result; + +use super::Sink; +use super::Sinker; +use crate::pipelines::new::processors::port::InputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; + +pub struct EmptySink; + +impl EmptySink { + pub fn create(input: Arc) -> ProcessorPtr { + Sinker::create(input, EmptySink {}) + } +} + +impl Sink for EmptySink { + const NAME: &'static str = "EmptySink"; + + fn consume(&mut self, _: DataBlock) -> Result<()> { + Ok(()) + } +} diff --git a/query/src/pipelines/new/processors/sinks/mod.rs b/query/src/pipelines/new/processors/sinks/mod.rs index 6b175f322f8a..0db6b4009f88 100644 --- a/query/src/pipelines/new/processors/sinks/mod.rs +++ b/query/src/pipelines/new/processors/sinks/mod.rs @@ -13,11 +13,13 @@ // limitations under the License. mod async_sink; +mod empty_sink; mod sync_sink; mod sync_sink_sender; pub use async_sink::AsyncSink; pub use async_sink::AsyncSinker; +pub use empty_sink::EmptySink; pub use sync_sink::Sink; pub use sync_sink::Sinker; pub use sync_sink_sender::SyncSenderSink; diff --git a/query/src/pipelines/new/processors/sources/blocks_source.rs b/query/src/pipelines/new/processors/sources/blocks_source.rs new file mode 100644 index 000000000000..4a3fec95a33a --- /dev/null +++ b/query/src/pipelines/new/processors/sources/blocks_source.rs @@ -0,0 +1,52 @@ +// Copyright 2022 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::VecDeque; +use std::sync::Arc; + +use common_datablocks::DataBlock; +use common_exception::Result; +use common_infallible::Mutex; + +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::SyncSource; +use crate::pipelines::new::processors::SyncSourcer; +use crate::sessions::QueryContext; + +pub struct BlocksSource { + data_blocks: Arc>>, +} + +impl BlocksSource { + pub fn create( + ctx: Arc, + output: Arc, + data_blocks: Arc>>, + ) -> Result { + SyncSourcer::create(ctx, output, BlocksSource { data_blocks }) + } +} + +impl SyncSource for BlocksSource { + const NAME: &'static str = "BlocksSource"; + + fn generate(&mut self) -> Result> { + let mut blocks_guard = self.data_blocks.lock(); + match blocks_guard.pop_front() { + None => Ok(None), + Some(data_block) => Ok(Some(data_block)), + } + } +} diff --git a/query/src/pipelines/new/processors/sources/mod.rs b/query/src/pipelines/new/processors/sources/mod.rs index fcb8f245a6ee..f7c08c1b32d9 100644 --- a/query/src/pipelines/new/processors/sources/mod.rs +++ b/query/src/pipelines/new/processors/sources/mod.rs @@ -13,12 +13,14 @@ // limitations under the License. mod async_source; +mod blocks_source; mod empty_source; mod sync_source; mod sync_source_receiver; pub use async_source::AsyncSource; pub use async_source::AsyncSourcer; +pub use blocks_source::BlocksSource; pub use empty_source::EmptySource; pub use sync_source::SyncSource; pub use sync_source::SyncSourcer; diff --git a/query/src/pipelines/new/processors/transforms/mod.rs b/query/src/pipelines/new/processors/transforms/mod.rs index 35feed6b8c7c..96c48f1ab01f 100644 --- a/query/src/pipelines/new/processors/transforms/mod.rs +++ b/query/src/pipelines/new/processors/transforms/mod.rs @@ -14,6 +14,7 @@ mod aggregator; mod transform; +mod transform_addon; mod transform_aggregator; mod transform_block_compact; mod transform_compact; @@ -28,7 +29,9 @@ mod transform_sort_partial; pub use aggregator::AggregatorParams; pub use aggregator::AggregatorTransformParams; +pub use transform_addon::TransformAddOn; pub use transform_aggregator::TransformAggregator; +pub use transform_block_compact::BlockCompactor; pub use transform_block_compact::TransformBlockCompact; pub use transform_compact::Compactor; pub use transform_compact::TransformCompact; diff --git a/query/src/pipelines/new/processors/transforms/transform_addon.rs b/query/src/pipelines/new/processors/transforms/transform_addon.rs new file mode 100644 index 000000000000..1cc1f6f7235c --- /dev/null +++ b/query/src/pipelines/new/processors/transforms/transform_addon.rs @@ -0,0 +1,114 @@ +// Copyright 2022 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_datablocks::DataBlock; +use common_datavalues::DataField; +use common_datavalues::DataSchema; +use common_datavalues::DataSchemaRef; +use common_exception::Result; +use common_planners::Expression; + +use crate::pipelines::new::processors::port::InputPort; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::transforms::transform::Transform; +use crate::pipelines::new::processors::transforms::transform::Transformer; +use crate::pipelines::transforms::ExpressionExecutor; +use crate::sessions::QueryContext; + +pub struct TransformAddOn { + default_expr_fields: Vec, + default_nonexpr_fields: Vec, + + expression_executor: ExpressionExecutor, + output_schema: DataSchemaRef, +} + +impl TransformAddOn +where Self: Transform +{ + pub fn try_create( + input: Arc, + output: Arc, + input_schema: DataSchemaRef, + output_schema: DataSchemaRef, + ctx: Arc, + ) -> Result { + let mut default_expr_fields = Vec::new(); + let mut default_exprs = Vec::new(); + let mut default_nonexpr_fields = Vec::new(); + + for f in output_schema.fields() { + if !input_schema.has_field(f.name()) { + if let Some(expr) = f.default_expr() { + let expression: Expression = serde_json::from_slice::(expr)?; + let expression = Expression::Alias( + f.name().to_string(), + Box::new(Expression::Cast { + expr: Box::new(expression), + data_type: f.data_type().clone(), + }), + ); + + default_expr_fields.push(f.clone()); + default_exprs.push(expression); + } else { + default_nonexpr_fields.push(f.clone()); + } + } + } + let schema_after_default_expr = Arc::new(DataSchema::new(default_expr_fields.clone())); + let expression_executor = ExpressionExecutor::try_create( + "stream_addon", + input_schema, + schema_after_default_expr, + default_exprs, + true, + ctx, + )?; + + Ok(Transformer::create(input, output, Self { + default_expr_fields, + default_nonexpr_fields, + expression_executor, + output_schema, + })) + } +} + +impl Transform for TransformAddOn { + const NAME: &'static str = "AddOnTransform"; + + fn transform(&mut self, mut block: DataBlock) -> Result { + let num_rows = block.num_rows(); + let expr_block = self.expression_executor.execute(&block)?; + + for f in self.default_expr_fields.iter() { + block = + block.add_column(expr_block.try_column_by_name(f.name())?.clone(), f.clone())?; + } + + for f in &self.default_nonexpr_fields { + let default_value = f.data_type().default_value(); + let column = f + .data_type() + .create_constant_column(&default_value, num_rows)?; + + block = block.add_column(column, f.clone())?; + } + block.resort(self.output_schema.clone()) + } +} diff --git a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs index 983143f3d438..50eff0a4a75a 100644 --- a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs +++ b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs @@ -18,21 +18,21 @@ use common_exception::Result; use super::Compactor; use super::TransformCompact; -pub struct BlockCompactCompactor { +pub struct BlockCompactor { max_row_per_block: usize, min_row_per_block: usize, } -impl BlockCompactCompactor { +impl BlockCompactor { pub fn new(max_row_per_block: usize, min_row_per_block: usize) -> Self { - BlockCompactCompactor { + BlockCompactor { max_row_per_block, min_row_per_block, } } } -impl Compactor for BlockCompactCompactor { +impl Compactor for BlockCompactor { fn name() -> &'static str { "BlockCompactTransform" } @@ -128,4 +128,4 @@ impl Compactor for BlockCompactCompactor { } } -pub type TransformBlockCompact = TransformCompact; +pub type TransformBlockCompact = TransformCompact; diff --git a/query/src/sessions/query_ctx.rs b/query/src/sessions/query_ctx.rs index 416fcd53331e..c291deecde88 100644 --- a/query/src/sessions/query_ctx.rs +++ b/query/src/sessions/query_ctx.rs @@ -26,6 +26,7 @@ use common_base::Runtime; use common_base::TrySpawn; use common_contexts::DalContext; use common_contexts::DalMetrics; +use common_datablocks::DataBlock; use common_exception::ErrorCode; use common_exception::Result; use common_infallible::RwLock; @@ -68,6 +69,7 @@ pub struct QueryContext { statistics: Arc>, partition_queue: Arc>>, shared: Arc, + precommit_blocks: Arc>>, } impl QueryContext { @@ -85,6 +87,7 @@ impl QueryContext { partition_queue: Arc::new(RwLock::new(VecDeque::new())), version: format!("DatabendQuery {}", *crate::configs::DATABEND_COMMIT_VERSION), shared, + precommit_blocks: Arc::new(RwLock::new(Vec::new())), }) } @@ -365,6 +368,18 @@ impl QueryContext { pub fn get_query_logger(&self) -> Option> { self.shared.session.session_mgr.get_query_logger() } + + pub fn push_precommit_block(&self, block: DataBlock) { + let mut blocks = self.precommit_blocks.write(); + blocks.push(block); + } + + pub fn consume_precommit_blocks(&self) -> Vec { + let mut blocks = self.precommit_blocks.write(); + let result = blocks.clone(); + blocks.clear(); + result + } } impl TrySpawn for QueryContext { diff --git a/query/src/storages/fuse/fuse_table.rs b/query/src/storages/fuse/fuse_table.rs index de9255034e02..ccc04e9d1091 100644 --- a/query/src/storages/fuse/fuse_table.rs +++ b/query/src/storages/fuse/fuse_table.rs @@ -132,6 +132,10 @@ impl Table for FuseTable { self.do_read2(ctx, plan, pipeline) } + fn append2(&self, ctx: Arc, pipeline: &mut NewPipeline) -> Result<()> { + self.do_append2(ctx, pipeline) + } + #[tracing::instrument(level = "debug", name = "fuse_table_append_data", skip(self, ctx, stream), fields(ctx.id = ctx.get_id().as_str()))] async fn append_data( &self, diff --git a/query/src/storages/fuse/io/mod.rs b/query/src/storages/fuse/io/mod.rs index 8d9e8034d82e..ffec1873a6ec 100644 --- a/query/src/storages/fuse/io/mod.rs +++ b/query/src/storages/fuse/io/mod.rs @@ -21,6 +21,7 @@ pub use read::BlockReader; pub use read::MetaReaders; pub use read::SegmentInfoReader; pub use read::TableSnapshotReader; +pub use write::write_block; pub use write::BlockCompactor; pub use write::BlockStreamWriter; pub use write::SegmentInfoStream; diff --git a/query/src/storages/fuse/io/write/mod.rs b/query/src/storages/fuse/io/write/mod.rs index 587779d94191..c147eeee7470 100644 --- a/query/src/storages/fuse/io/write/mod.rs +++ b/query/src/storages/fuse/io/write/mod.rs @@ -19,3 +19,4 @@ mod block_writer; pub use block_stream_writer::BlockCompactor; pub use block_stream_writer::BlockStreamWriter; pub use block_stream_writer::SegmentInfoStream; +pub use block_writer::write_block; diff --git a/query/src/storages/fuse/operations/append.rs b/query/src/storages/fuse/operations/append.rs index 5118ed4d4990..834ae73a59f6 100644 --- a/query/src/storages/fuse/operations/append.rs +++ b/query/src/storages/fuse/operations/append.rs @@ -19,12 +19,19 @@ use std::sync::Arc; use async_stream::stream; use common_cache::Cache; use common_exception::Result; +use common_infallible::RwLock; use common_streams::SendableDataBlockStream; use futures::StreamExt; +use crate::pipelines::new::processors::port::InputPort; +use crate::pipelines::new::processors::BlockCompactor; +use crate::pipelines::new::processors::TransformCompact; +use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SinkPipeBuilder; use crate::sessions::QueryContext; use crate::storages::fuse::io::BlockStreamWriter; use crate::storages::fuse::operations::AppendOperationLogEntry; +use crate::storages::fuse::operations::FuseSink; use crate::storages::fuse::FuseTable; use crate::storages::fuse::DEFAULT_BLOCK_PER_SEGMENT; use crate::storages::fuse::DEFAULT_ROW_PER_BLOCK; @@ -88,6 +95,44 @@ impl FuseTable { Ok(Box::pin(log_entries)) } + pub fn do_append2(&self, ctx: Arc, pipeline: &mut NewPipeline) -> Result<()> { + let max_row_per_block = self.get_option(FUSE_OPT_KEY_ROW_PER_BLOCK, DEFAULT_ROW_PER_BLOCK); + let min_rows_per_block = (max_row_per_block as f64 * 0.8) as usize; + let block_per_seg = + self.get_option(FUSE_OPT_KEY_BLOCK_PER_SEGMENT, DEFAULT_BLOCK_PER_SEGMENT); + + let da = ctx.get_storage_operator()?; + + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformCompact::try_create( + transform_input_port, + transform_output_port, + BlockCompactor::new(max_row_per_block, min_rows_per_block), + ) + })?; + + let mut sink_pipeline_builder = SinkPipeBuilder::create(); + let acc = Arc::new(RwLock::new(None)); + for _ in 0..pipeline.output_len() { + let input_port = InputPort::create(); + sink_pipeline_builder.add_sink( + input_port.clone(), + FuseSink::create_processor( + input_port, + ctx.clone(), + block_per_seg, + da.clone(), + self.table_info.schema().clone(), + self.meta_location_generator().clone(), + acc.clone(), + ), + ); + } + + pipeline.add_pipe(sink_pipeline_builder.finalize()); + Ok(()) + } + fn get_option(&self, opt_key: &str, default: T) -> T { self.table_info .options() diff --git a/query/src/storages/fuse/operations/fuse_sink.rs b/query/src/storages/fuse/operations/fuse_sink.rs new file mode 100644 index 000000000000..9fcfb93879c2 --- /dev/null +++ b/query/src/storages/fuse/operations/fuse_sink.rs @@ -0,0 +1,237 @@ +// 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::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use common_arrow::parquet::FileMetaData; +use common_cache::Cache; +use common_datablocks::DataBlock; +use common_datavalues::DataSchema; +use common_exception::ErrorCode; +use common_exception::Result; +use common_infallible::RwLock; +use futures::Future; +use opendal::Operator; + +use super::AppendOperationLogEntry; +use crate::pipelines::new::processors::port::InputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::AsyncSink; +use crate::pipelines::new::processors::AsyncSinker; +use crate::sessions::QueryContext; +use crate::storages::fuse::io::write_block; +use crate::storages::fuse::io::TableMetaLocationGenerator; +use crate::storages::fuse::meta::ColumnId; +use crate::storages::fuse::meta::ColumnMeta; +use crate::storages::fuse::meta::SegmentInfo; +use crate::storages::fuse::meta::Statistics; +use crate::storages::fuse::statistics::StatisticsAccumulator; + +pub struct FuseSink { + ctx: Arc, + num_block_threshold: usize, + data_accessor: Operator, + data_schema: Arc, + number_of_blocks_accumulated: usize, + meta_locations: TableMetaLocationGenerator, + statistics_accumulator: Arc>>, +} + +#[async_trait] +impl AsyncSink for FuseSink { + const NAME: &'static str = "FuseSink"; + type ConsumeFuture<'a> = impl Future> where Self: 'a; + + /// We don't use async_trait for consume method, using GAT instead to make it more static dispatchable. + fn consume(&mut self, data_block: DataBlock) -> Self::ConsumeFuture<'_> { + async move { + if let Some(seg_info) = self.write_block(data_block).await? { + let block = self + .seg_info_to_log_entry(seg_info) + .await + .and_then(DataBlock::try_from)?; + self.ctx.push_precommit_block(block); + } + Ok(()) + } + } + + async fn on_finish(&mut self) -> Result<()> { + let seg_info = self.flush().await?; + + if let Some(seg_info) = seg_info { + let block = self + .seg_info_to_log_entry(seg_info) + .await + .and_then(DataBlock::try_from)?; + self.ctx.push_precommit_block(block); + } + Ok(()) + } +} + +impl FuseSink { + pub fn create_processor( + input: Arc, + ctx: Arc, + num_block_threshold: usize, + data_accessor: Operator, + data_schema: Arc, + meta_locations: TableMetaLocationGenerator, + statistics_accumulator: Arc>>, + ) -> ProcessorPtr { + let sink = FuseSink { + ctx, + num_block_threshold, + data_accessor, + data_schema, + meta_locations, + statistics_accumulator, + + number_of_blocks_accumulated: 0, + }; + + AsyncSinker::create(input, sink) + } + + async fn write_block(&mut self, block: DataBlock) -> Result> { + let schema = block.schema().to_arrow(); + let location = self.meta_locations.gen_block_location(); + let (file_size, file_meta_data) = write_block( + &schema, + block.clone(), + self.data_accessor.clone(), + &location, + ) + .await?; + + let mut acc_writer = self.statistics_accumulator.write(); + let mut acc = acc_writer.take().unwrap_or_default(); + let partial_acc = acc.begin(&block)?; + + let col_metas = Self::column_metas(&file_meta_data)?; + acc = partial_acc.end(file_size, location, col_metas); + self.number_of_blocks_accumulated += 1; + if self.number_of_blocks_accumulated >= self.num_block_threshold { + let summary = acc.summary(self.data_schema.as_ref())?; + let seg = SegmentInfo::new(acc.blocks_metas, Statistics { + row_count: acc.summary_row_count, + block_count: acc.summary_block_count, + uncompressed_byte_size: acc.in_memory_size, + compressed_byte_size: acc.file_size, + col_stats: summary, + }); + + // Reset state + self.number_of_blocks_accumulated = 0; + *acc_writer = None; + + Ok(Some(seg)) + } else { + // Stash the state + *acc_writer = Some(acc); + Ok(None) + } + } + + async fn flush(&mut self) -> Result> { + let mut acc_writer = self.statistics_accumulator.write(); + let acc = acc_writer.take(); + + if let Some(acc) = acc { + let summary = acc.summary(self.data_schema.as_ref())?; + let seg = SegmentInfo::new(acc.blocks_metas, Statistics { + row_count: acc.summary_row_count, + block_count: acc.summary_block_count, + uncompressed_byte_size: acc.in_memory_size, + compressed_byte_size: acc.file_size, + col_stats: summary, + }); + + Ok(Some(seg)) + } else { + Ok(None) + } + } + + async fn seg_info_to_log_entry( + &self, + seg_info: SegmentInfo, + ) -> Result { + let locs = self.meta_locations.clone(); + let segment_info_cache = self + .ctx + .get_storage_cache_manager() + .get_table_segment_cache(); + + let seg_loc = locs.gen_segment_info_location(); + let bytes = serde_json::to_vec(&seg_info)?; + self.data_accessor.object(&seg_loc).write(bytes).await?; + let seg = Arc::new(seg_info); + let log_entry = AppendOperationLogEntry::new(seg_loc.clone(), seg.clone()); + + if let Some(ref cache) = segment_info_cache { + let cache = &mut cache.write().await; + cache.put(seg_loc, seg); + } + Ok(log_entry) + } + + fn column_metas(file_meta: &FileMetaData) -> Result> { + // currently we use one group only + let num_row_groups = file_meta.row_groups.len(); + if num_row_groups != 1 { + return Err(ErrorCode::ParquetError(format!( + "invalid parquet file, expects only one row group, but got {}", + num_row_groups + ))); + } + let row_group = &file_meta.row_groups[0]; + let mut col_metas = HashMap::with_capacity(row_group.columns.len()); + for (idx, col_chunk) in row_group.columns.iter().enumerate() { + match &col_chunk.meta_data { + Some(chunk_meta) => { + let col_start = + if let Some(dict_page_offset) = chunk_meta.dictionary_page_offset { + dict_page_offset + } else { + chunk_meta.data_page_offset + }; + let col_len = chunk_meta.total_compressed_size; + assert!( + col_start >= 0 && col_len >= 0, + "column start and length should not be negative" + ); + let num_values = chunk_meta.num_values as u64; + let res = ColumnMeta { + offset: col_start as u64, + len: col_len as u64, + num_values, + }; + col_metas.insert(idx as u32, res); + } + None => { + return Err(ErrorCode::ParquetError(format!( + "invalid parquet file, meta data of column idx {} is empty", + idx + ))) + } + } + } + Ok(col_metas) + } +} diff --git a/query/src/storages/fuse/operations/mod.rs b/query/src/storages/fuse/operations/mod.rs index a6f3a9fe42e9..928270dbb958 100644 --- a/query/src/storages/fuse/operations/mod.rs +++ b/query/src/storages/fuse/operations/mod.rs @@ -14,11 +14,13 @@ mod append; mod commit; +mod fuse_sink; mod operation_log; mod optimize; mod read; mod read_partitions; mod truncate; +pub use fuse_sink::FuseSink; pub use operation_log::AppendOperationLogEntry; pub use operation_log::TableOperationLog; diff --git a/query/src/storages/memory/memory_table.rs b/query/src/storages/memory/memory_table.rs index d9b7d16c9acd..2826eb07595b 100644 --- a/query/src/storages/memory/memory_table.rs +++ b/query/src/storages/memory/memory_table.rs @@ -31,11 +31,15 @@ use common_planners::Statistics; use common_planners::TruncateTablePlan; use common_streams::SendableDataBlockStream; +use crate::pipelines::new::processors::port::InputPort; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::Sink; +use crate::pipelines::new::processors::Sinker; use crate::pipelines::new::processors::SyncSource; use crate::pipelines::new::processors::SyncSourcer; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SinkPipeBuilder; use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::memory::memory_part::MemoryPartInfo; @@ -234,6 +238,20 @@ impl Table for MemoryTable { Ok(()) } + fn append2(&self, ctx: Arc, pipeline: &mut NewPipeline) -> Result<()> { + let mut sink_pipeline_builder = SinkPipeBuilder::create(); + for _ in 0..pipeline.output_len() { + let input_port = InputPort::create(); + sink_pipeline_builder.add_sink( + input_port.clone(), + MemoryTableSink::create(input_port, ctx.clone()), + ); + } + + pipeline.add_pipe(sink_pipeline_builder.finalize()); + Ok(()) + } + async fn append_data( &self, _ctx: Arc, @@ -323,3 +341,22 @@ impl SyncSource for MemoryTableSource { } } } + +struct MemoryTableSink { + ctx: Arc, +} + +impl MemoryTableSink { + pub fn create(input: Arc, ctx: Arc) -> ProcessorPtr { + Sinker::create(input, MemoryTableSink { ctx }) + } +} + +impl Sink for MemoryTableSink { + const NAME: &'static str = "MemoryTableSink"; + + fn consume(&mut self, data_block: DataBlock) -> Result<()> { + self.ctx.push_precommit_block(data_block); + Ok(()) + } +} diff --git a/query/src/storages/null/null_table.rs b/query/src/storages/null/null_table.rs index 02d251033644..10f5613c8ec1 100644 --- a/query/src/storages/null/null_table.rs +++ b/query/src/storages/null/null_table.rs @@ -30,12 +30,15 @@ use common_streams::SendableDataBlockStream; use common_tracing::tracing::info; use futures::stream::StreamExt; +use crate::pipelines::new::processors::port::InputPort; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::EmptySink; use crate::pipelines::new::processors::SyncSource; use crate::pipelines::new::processors::SyncSourcer; use crate::pipelines::new::NewPipe; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SinkPipeBuilder; use crate::sessions::QueryContext; use crate::storages::StorageContext; use crate::storages::StorageDescription; @@ -123,6 +126,16 @@ impl Table for NullTable { ))) } + fn append2(&self, _: Arc, pipeline: &mut NewPipeline) -> Result<()> { + let mut sink_pipeline_builder = SinkPipeBuilder::create(); + for _ in 0..pipeline.output_len() { + let input_port = InputPort::create(); + sink_pipeline_builder.add_sink(input_port.clone(), EmptySink::create(input_port)); + } + pipeline.add_pipe(sink_pipeline_builder.finalize()); + Ok(()) + } + async fn truncate( &self, _ctx: Arc, diff --git a/query/src/storages/storage_table.rs b/query/src/storages/storage_table.rs index 2bf0eb1ba0c2..a7167d15871d 100644 --- a/query/src/storages/storage_table.rs +++ b/query/src/storages/storage_table.rs @@ -104,6 +104,10 @@ pub trait Table: Sync + Send { unimplemented!() } + fn append2(&self, _: Arc, _: &mut NewPipeline) -> Result<()> { + unimplemented!() + } + async fn append_data( &self, _ctx: Arc, From f21f6b3d2fed6509cf06e683cf90bf4d1507a47f Mon Sep 17 00:00:00 2001 From: xudong963 Date: Sat, 16 Apr 2022 23:22:39 +0800 Subject: [PATCH 02/27] make insert with select plan work --- query/src/interpreters/interpreter_insert.rs | 68 ++++++++++++++++- query/src/pipelines/new/processors/mod.rs | 1 + .../new/processors/transforms/mod.rs | 2 + .../transforms/transform_cast_schema.rs | 73 +++++++++++++++++++ 4 files changed, 140 insertions(+), 4 deletions(-) create mode 100644 query/src/pipelines/new/processors/transforms/transform_cast_schema.rs diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 483c6f6f7545..5faed7cc0688 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -17,26 +17,36 @@ use std::sync::Arc; use common_exception::ErrorCode; use common_exception::Result; +use common_functions::scalars::CastFunction; +use common_functions::scalars::FunctionContext; use common_infallible::Mutex; use common_meta_types::GrantObject; use common_meta_types::UserPrivilegeType; use common_planners::InsertInputSource; use common_planners::InsertPlan; +use common_planners::PlanNode; +use common_planners::SelectPlan; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; +use common_tracing::tracing; use futures::TryStreamExt; +use poem::trace; use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; +use crate::interpreters::plan_schedulers; use crate::interpreters::plan_schedulers::InsertWithPlan; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelineCompleteExecutor; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::BlocksSource; use crate::pipelines::new::processors::TransformAddOn; +use crate::pipelines::new::processors::TransformCastSchema; use crate::pipelines::new::processors::TransformDummy; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::transforms::AddOnStream; use crate::sessions::QueryContext; @@ -69,7 +79,7 @@ impl InsertInterpreter { let mut pipeline = self.create_new_pipeline()?; let mut builder = SourcePipeBuilder::create(); - + let mut need_cast_schema = false; match &self.plan.source { InsertInputSource::Values(values) => { let blocks = Arc::new(Mutex::new(VecDeque::from_iter(vec![values.block.clone()]))); @@ -84,9 +94,14 @@ impl InsertInterpreter { pipeline.add_pipe(builder.finalize()); } InsertInputSource::StreamingWithFormat(_) => {} - InsertInputSource::SelectPlan(_) => { - // // todo!(); - // Some(BlocksSource::create(vec![])) + InsertInputSource::SelectPlan(plan) => { + let builder = QueryPipelineBuilder::create(self.ctx.clone()); + need_cast_schema = self.check_schema_cast(plan)?; + let optimized_plan = self.rewrite_plan(plan)?; + let select_plan = SelectPlan { + input: Arc::new(optimized_plan), + }; + pipeline = builder.finalize(&select_plan)?; } }; @@ -103,6 +118,30 @@ impl InsertInterpreter { })?; } + // cast schema + if need_cast_schema { + let mut functions = Vec::with_capacity(self.plan.schema().fields().len()); + for field in self.plan.schema().fields() { + let name = format!("{:?}", field.data_type()); + let cast_function = CastFunction::create("cast", &name).unwrap(); + functions.push(cast_function); + } + let tz = self.ctx.get_settings().get_timezone()?; + let tz = String::from_utf8(tz).map_err(|_| { + ErrorCode::LogicalError("Timezone has beeen checked and should be valid.") + })?; + let func_ctx = FunctionContext { tz }; + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformCastSchema::try_create( + transform_input_port, + transform_output_port, + self.plan.schema(), + functions.clone(), + func_ctx.clone(), + ) + })?; + } + table.append2(self.ctx.clone(), &mut pipeline)?; let async_runtime = self.ctx.get_storage_runtime(); @@ -123,6 +162,27 @@ impl InsertInterpreter { vec![], ))) } + + /// Call this method to optimize the logical plan before executing + fn rewrite_plan(&self, select_plan: &PlanNode) -> Result { + plan_schedulers::apply_plan_rewrite(Optimizers::create(self.ctx.clone()), select_plan) + } + + fn check_schema_cast(&self, plan_node: &PlanNode) -> common_exception::Result { + let output_schema = &self.plan.schema; + let select_schema = plan_node.schema(); + + // validate schema + if select_schema.fields().len() < output_schema.fields().len() { + return Err(ErrorCode::BadArguments( + "Fields in select statement is less than expected", + )); + } + + // check if cast needed + let cast_needed = select_schema != *output_schema; + Ok(cast_needed) + } } #[async_trait::async_trait] diff --git a/query/src/pipelines/new/processors/mod.rs b/query/src/pipelines/new/processors/mod.rs index a8f0d8c851a9..c7b46a7c067d 100644 --- a/query/src/pipelines/new/processors/mod.rs +++ b/query/src/pipelines/new/processors/mod.rs @@ -51,6 +51,7 @@ pub use transforms::SubQueriesPuller; pub use transforms::TransformAddOn; pub use transforms::TransformAggregator; pub use transforms::TransformBlockCompact; +pub use transforms::TransformCastSchema; pub use transforms::TransformCompact; pub use transforms::TransformCreateSets; pub use transforms::TransformDummy; diff --git a/query/src/pipelines/new/processors/transforms/mod.rs b/query/src/pipelines/new/processors/transforms/mod.rs index 96c48f1ab01f..08c2b33ba756 100644 --- a/query/src/pipelines/new/processors/transforms/mod.rs +++ b/query/src/pipelines/new/processors/transforms/mod.rs @@ -17,6 +17,7 @@ mod transform; mod transform_addon; mod transform_aggregator; mod transform_block_compact; +mod transform_cast_schema; mod transform_compact; mod transform_create_sets; mod transform_dummy; @@ -33,6 +34,7 @@ pub use transform_addon::TransformAddOn; pub use transform_aggregator::TransformAggregator; pub use transform_block_compact::BlockCompactor; pub use transform_block_compact::TransformBlockCompact; +pub use transform_cast_schema::TransformCastSchema; pub use transform_compact::Compactor; pub use transform_compact::TransformCompact; pub use transform_create_sets::SubQueriesPuller; diff --git a/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs new file mode 100644 index 000000000000..37f22e220824 --- /dev/null +++ b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs @@ -0,0 +1,73 @@ +// 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_datablocks::DataBlock; +use common_datavalues::ColumnWithField; +use common_datavalues::DataSchemaRef; +use common_exception::Result; +use common_functions::scalars::Function; +use common_functions::scalars::FunctionContext; +use common_streams::SendableDataBlockStream; + +use crate::pipelines::new::processors::port::InputPort; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::transforms::transform::Transform; +use crate::pipelines::new::processors::transforms::transform::Transformer; +use crate::pipelines::new::processors::ResizeProcessor; + +pub struct TransformCastSchema { + output_schema: DataSchemaRef, + functions: Vec>, + func_ctx: FunctionContext, +} + +impl TransformCastSchema +where Self: Transform +{ + pub fn try_create( + input_port: Arc, + output_port: Arc, + output_schema: DataSchemaRef, + functions: Vec>, + func_ctx: FunctionContext, + ) -> Result { + Ok(Transformer::create(input_port, output_port, Self { + output_schema, + functions, + func_ctx, + })) + } +} + +impl Transform for TransformCastSchema { + const NAME: &'static str = "CastSchemaTransform"; + + fn transform(&mut self, data: DataBlock) -> Result { + let rows = data.num_rows(); + let iter = self + .functions + .iter() + .zip(data.schema().fields()) + .zip(data.columns()); + let mut columns = Vec::with_capacity(data.num_columns()); + for ((cast_func, input_field), column) in iter { + let column = ColumnWithField::new(column.clone(), input_field.clone()); + columns.push(cast_func.eval(self.func_ctx.clone(), &[column], rows)?); + } + Ok(DataBlock::create(self.output_schema.clone(), columns)) + } +} From c7927dc9dda22dd54ef26cf2572aa1d2c3f598c6 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Sun, 17 Apr 2022 17:39:04 +0800 Subject: [PATCH 03/27] make insert with format streaming work --- query/src/interpreters/interpreter.rs | 3 + query/src/interpreters/interpreter_call.rs | 4 +- query/src/interpreters/interpreter_copy.rs | 5 +- .../interpreter_database_create.rs | 4 +- .../interpreters/interpreter_database_drop.rs | 2 + .../interpreter_database_show_create.rs | 2 + query/src/interpreters/interpreter_explain.rs | 2 + .../interpreter_factory_interceptor.rs | 7 ++- query/src/interpreters/interpreter_insert.rs | 16 ++++- query/src/interpreters/interpreter_kill.rs | 2 + query/src/interpreters/interpreter_list.rs | 4 +- .../interpreter_privilege_grant.rs | 4 +- .../interpreter_privilege_revoke.rs | 4 +- .../interpreters/interpreter_role_create.rs | 4 +- .../src/interpreters/interpreter_role_drop.rs | 4 +- .../interpreters/interpreter_role_grant.rs | 4 +- .../interpreters/interpreter_role_revoke.rs | 4 +- query/src/interpreters/interpreter_select.rs | 5 +- query/src/interpreters/interpreter_setting.rs | 2 + .../interpreter_show_databases.rs | 4 +- .../interpreters/interpreter_show_engines.rs | 4 +- .../interpreter_show_functions.rs | 4 +- .../interpreters/interpreter_show_grants.rs | 2 + .../interpreters/interpreter_show_metrics.rs | 4 +- .../interpreter_show_processlist.rs | 4 +- .../interpreters/interpreter_show_roles.rs | 4 +- .../interpreters/interpreter_show_settings.rs | 4 +- .../interpreters/interpreter_show_tab_stat.rs | 4 +- .../interpreters/interpreter_show_tables.rs | 4 +- .../interpreters/interpreter_show_users.rs | 4 +- .../interpreters/interpreter_table_create.rs | 4 +- .../interpreter_table_describe.rs | 2 + .../interpreters/interpreter_table_drop.rs | 2 + .../interpreter_table_optimize.rs | 4 +- .../interpreters/interpreter_table_rename.rs | 2 + .../interpreter_table_show_create.rs | 2 + .../interpreter_table_truncate.rs | 2 + .../interpreters/interpreter_use_database.rs | 2 + .../interpreters/interpreter_user_alter.rs | 4 +- .../interpreters/interpreter_user_create.rs | 4 +- .../src/interpreters/interpreter_user_drop.rs | 4 +- .../interpreter_user_stage_create.rs | 4 +- .../interpreter_user_stage_describe.rs | 4 +- .../interpreter_user_stage_drop.rs | 4 +- .../interpreter_user_udf_alter.rs | 4 +- .../interpreter_user_udf_create.rs | 4 +- .../interpreters/interpreter_user_udf_drop.rs | 4 +- .../interpreters/interpreter_view_alter.rs | 7 ++- .../interpreters/interpreter_view_create.rs | 7 ++- .../src/interpreters/interpreter_view_drop.rs | 2 + query/src/pipelines/new/processors/mod.rs | 1 + .../pipelines/new/processors/sources/mod.rs | 2 + .../sources/sync_ck_source_receiver.rs | 59 +++++++++++++++++++ .../clickhouse/interactive_worker_base.rs | 26 ++++++-- query/src/servers/clickhouse/mod.rs | 1 + query/src/servers/http/clickhouse_handler.rs | 2 +- query/src/servers/http/v1/load.rs | 2 +- .../servers/http/v1/query/execute_state.rs | 2 +- query/src/servers/mod.rs | 2 +- .../servers/mysql/mysql_interactive_worker.rs | 2 +- 60 files changed, 247 insertions(+), 48 deletions(-) create mode 100644 query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index 4f2a78eac90f..751a3ebc84f1 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -18,7 +18,9 @@ use common_exception::ErrorCode; use common_exception::Result; use common_streams::SendableDataBlockStream; +use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SourcePipeBuilder; #[async_trait::async_trait] /// Interpreter is a trait for different PlanNode @@ -36,6 +38,7 @@ pub trait Interpreter: Sync + Send { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result; /// Create the new pipeline for databend's new execution model diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index bf6d8b3bb962..22c40b15b5e4 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use super::Interpreter; use super::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::procedures::ProcedureFactory; use crate::sessions::QueryContext; @@ -42,10 +43,11 @@ impl Interpreter for CallInterpreter { "CallInterpreter" } - #[tracing::instrument(level = "debug", name = "call_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", name = "call_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = &self.plan; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 9a191f61a8e4..e8ff71dd2880 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -32,7 +32,9 @@ use crate::interpreters::stream::ProcessorExecutorStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::pipelines::new::executor::PipelinePullingExecutor; +use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::StageSource; @@ -137,10 +139,11 @@ impl Interpreter for CopyInterpreter { "CopyInterpreter" } - #[tracing::instrument(level = "debug", name = "copy_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", name = "copy_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let mut files = self.list_files().await?; diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index e451b7036d89..983f6681ebc2 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -25,6 +25,7 @@ use common_tracing::tracing; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -45,10 +46,11 @@ impl Interpreter for CreateDatabaseInterpreter { "CreateDatabaseInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index 721a290a76bc..b3e8b434ba41 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -24,6 +24,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct DropDatabaseInterpreter { @@ -46,6 +47,7 @@ impl Interpreter for DropDatabaseInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index 640cbadfa931..7c924d6b88fa 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -24,6 +24,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct ShowCreateDatabaseInterpreter { @@ -49,6 +50,7 @@ impl Interpreter for ShowCreateDatabaseInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let calalog = self.ctx.get_catalog(); diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 54e7a9451025..754e1dafcaff 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -26,6 +26,7 @@ use crate::interpreters::plan_schedulers; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::processors::PipelineBuilder; use crate::sessions::QueryContext; @@ -43,6 +44,7 @@ impl Interpreter for ExplainInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let schema = self.schema(); diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index a1294f93e272..91f32c976c6d 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -23,6 +23,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::InterpreterQueryLog; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct InterceptorInterpreter { @@ -50,8 +51,12 @@ impl Interpreter for InterceptorInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { - let result_stream = self.inner.execute(input_stream).await?; + let result_stream = self + .inner + .execute(input_stream, source_pipe_builder) + .await?; let metric_stream = ProgressStream::try_create(result_stream, self.ctx.get_result_progress())?; Ok(Box::pin(metric_stream)) diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 5faed7cc0688..904d76da0673 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -28,6 +28,7 @@ use common_planners::PlanNode; use common_planners::SelectPlan; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; +use common_streams::Source; use common_tracing::tracing; use futures::TryStreamExt; use poem::trace; @@ -69,6 +70,7 @@ impl InsertInterpreter { async fn execute_new( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let plan = &self.plan; let settings = self.ctx.get_settings(); @@ -93,7 +95,14 @@ impl InsertInterpreter { } pipeline.add_pipe(builder.finalize()); } - InsertInputSource::StreamingWithFormat(_) => {} + InsertInputSource::StreamingWithFormat(_) => { + tracing::info!("come here"); + pipeline.add_pipe( + source_pipe_builder + .ok_or_else(|| ErrorCode::EmptyData("empty"))? + .finalize(), + ); + } InsertInputSource::SelectPlan(plan) => { let builder = QueryPipelineBuilder::create(self.ctx.clone()); need_cast_schema = self.check_schema_cast(plan)?; @@ -128,7 +137,7 @@ impl InsertInterpreter { } let tz = self.ctx.get_settings().get_timezone()?; let tz = String::from_utf8(tz).map_err(|_| { - ErrorCode::LogicalError("Timezone has beeen checked and should be valid.") + ErrorCode::LogicalError("Timezone has been checked and should be valid.") })?; let func_ctx = FunctionContext { tz }; pipeline.add_transform(|transform_input_port, transform_output_port| { @@ -194,6 +203,7 @@ impl Interpreter for InsertInterpreter { async fn execute( &self, mut input_stream: Option, + source_pipe_builder: Option, ) -> Result { let settings = self.ctx.get_settings(); @@ -201,7 +211,7 @@ impl Interpreter for InsertInterpreter { || (settings.get_enable_new_processor_framework()? == 2 && self.ctx.get_cluster().is_empty()) { - return self.execute_new(input_stream).await; + return self.execute_new(input_stream, source_pipe_builder).await; } let plan = &self.plan; diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index d4de47ce7c01..4cbf24ab2600 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -25,6 +25,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct KillInterpreter { @@ -47,6 +48,7 @@ impl Interpreter for KillInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index cfbac9613639..49a8a0609eaa 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -29,6 +29,7 @@ use regex::Regex; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::StageSource; @@ -74,10 +75,11 @@ impl Interpreter for ListInterpreter { "ListInterpreter" } - #[tracing::instrument(level = "debug", name = "list_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", name = "list_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let files = self.list_files().await?; tracing::info!("list file list:{:?}, pattern:{}", &files, self.plan.pattern); diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index bb76097f36e5..9aceec9c1807 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -26,6 +26,7 @@ use common_tracing::tracing; use crate::interpreters::interpreter_common::validate_grant_object_exists; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -46,10 +47,11 @@ impl Interpreter for GrantPrivilegeInterpreter { "GrantPrivilegeInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index 48538395e468..a2c5713d321a 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -24,6 +24,7 @@ use common_tracing::tracing; use crate::interpreters::interpreter_common::validate_grant_object_exists; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -44,10 +45,11 @@ impl Interpreter for RevokePrivilegeInterpreter { "RevokePrivilegeInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index a1828cc81a22..24904cfa8c94 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -23,6 +23,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -43,10 +44,11 @@ impl Interpreter for CreateRoleInterpreter { "CreateRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { // TODO: add privilege check about CREATE ROLE let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index f12c2e29b76c..1b5b5fe44fdc 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for DropRoleInterpreter { "DropRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { // TODO: add privilege check about DROP role let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index cc8c9dcf03c5..18ce9482965c 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -23,6 +23,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -43,10 +44,11 @@ impl Interpreter for GrantRoleInterpreter { "GrantRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index 7f784eb4e7f0..8221a3405333 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -23,6 +23,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -43,10 +44,11 @@ impl Interpreter for RevokeRoleInterpreter { "RevokeRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index af5e3b9b4a14..2fdfb8dd6fd2 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -27,8 +27,10 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelinePullingExecutor; +use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; /// SelectInterpreter struct which interprets SelectPlan @@ -64,13 +66,14 @@ impl Interpreter for SelectInterpreter { self.select.schema() } - #[tracing::instrument(level = "debug", name = "select_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", name = "select_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] /// Currently, the method has two sets of logic, if `get_enable_new_processor_framework` is turned on in the settings, /// the execution will use the new processor, otherwise the old processing logic will be executed. /// Note: there is an issue to track the progress of the new processor: https://github.com/datafuselabs/databend/issues/3379 async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let settings = self.ctx.get_settings(); diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index 996cda81cfe0..af45fa895c5b 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -24,6 +24,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct SettingInterpreter { @@ -46,6 +47,7 @@ impl Interpreter for SettingInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.set.clone(); for var in plan.vars { diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 42431cc52500..7d094acb33d0 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -25,6 +25,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -64,6 +65,7 @@ impl Interpreter for ShowDatabasesInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -71,7 +73,7 @@ impl Interpreter for ShowDatabasesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show databases build query error")); } diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 26d0a82febd9..62f8925e2eae 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -52,6 +53,7 @@ impl Interpreter for ShowEnginesInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -59,7 +61,7 @@ impl Interpreter for ShowEnginesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show engines build query error")); } diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index d5f0c30ee827..29b7115a6231 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -25,6 +25,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -65,6 +66,7 @@ impl Interpreter for ShowFunctionsInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -72,7 +74,7 @@ impl Interpreter for ShowFunctionsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show functions build query error")); } diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index 7706e2ca397c..23c7cb8db530 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -24,6 +24,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct ShowGrantsInterpreter { @@ -46,6 +47,7 @@ impl Interpreter for ShowGrantsInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let schema = DataSchemaRefExt::create(vec![DataField::new("Grants", Vu8::to_data_type())]); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index e60251abb446..326d9700cef7 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -52,6 +53,7 @@ impl Interpreter for ShowMetricsInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -59,7 +61,7 @@ impl Interpreter for ShowMetricsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show metrics build query error")); } diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index 03f689f63fc2..77c94a2dbf1a 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -55,6 +56,7 @@ impl Interpreter for ShowProcessListInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -62,7 +64,7 @@ impl Interpreter for ShowProcessListInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError( "Show processlist build query error", diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index eb360225bec0..b1a0460ca261 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -52,6 +53,7 @@ impl Interpreter for ShowRolesInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -59,7 +61,7 @@ impl Interpreter for ShowRolesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show roles build query error")); } diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index 343276035949..9831d2f742da 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -52,6 +53,7 @@ impl Interpreter for ShowSettingsInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -59,7 +61,7 @@ impl Interpreter for ShowSettingsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show settings build query error")); } diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index d07b786ad612..64a1ae4adde2 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -26,6 +26,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -91,6 +92,7 @@ impl Interpreter for ShowTabStatInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -98,7 +100,7 @@ impl Interpreter for ShowTabStatInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError( "Show table status build query error", diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index 8dcf1c257f78..ff7634c3595f 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -26,6 +26,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -82,6 +83,7 @@ impl Interpreter for ShowTablesInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -89,7 +91,7 @@ impl Interpreter for ShowTablesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show tables build query error")); } diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index 95681955aa40..8b6b72d452f9 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -24,6 +24,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -55,6 +56,7 @@ impl Interpreter for ShowUsersInterpreter { async fn execute( &self, input_stream: Option, + source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -62,7 +64,7 @@ impl Interpreter for ShowUsersInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream).await + interpreter.execute(input_stream, source_pipe_builder).await } else { return Err(ErrorCode::LogicalError("Show users build query error")); } diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index 48f8067efe43..a71a9f0f2a36 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -31,6 +31,7 @@ use super::InsertInterpreter; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct CreateTableInterpreter { @@ -53,6 +54,7 @@ impl Interpreter for CreateTableInterpreter { async fn execute( &self, input_stream: Option, + _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() @@ -138,7 +140,7 @@ impl CreateTableInterpreter { source: InsertInputSource::SelectPlan(select_plan_node), }; let insert_interpreter = InsertInterpreter::try_create(self.ctx.clone(), insert_plan)?; - insert_interpreter.execute(input_stream).await?; + insert_interpreter.execute(input_stream, None).await?; Ok(Box::pin(DataBlockStream::create( self.plan.schema(), diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index 6d1a4f6664f9..d3696eb94a79 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -24,6 +24,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct DescribeTableInterpreter { @@ -46,6 +47,7 @@ impl Interpreter for DescribeTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let database = self.plan.db.as_str(); let table = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index 8aa9f99fec55..529d24c7d614 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -25,6 +25,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; @@ -48,6 +49,7 @@ impl Interpreter for DropTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.as_str(); let tbl_name = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index e958589445a0..e8f07c478cfb 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -25,6 +25,7 @@ use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterFactory; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; @@ -48,6 +49,7 @@ impl Interpreter for OptimizeTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = &self.plan; let mut table = self.ctx.get_table(&plan.database, &plan.table).await?; @@ -64,7 +66,7 @@ impl Interpreter for OptimizeTableInterpreter { let rewritten_plan = PlanParser::parse(self.ctx.clone(), rewritten_query.as_str()).await?; let interpreter = InterpreterFactory::get(self.ctx.clone(), rewritten_plan)?; - let mut stream = interpreter.execute(None).await?; + let mut stream = interpreter.execute(None, None).await?; while let Some(Ok(_)) = stream.next().await {} if do_purge { // currently, context caches the table, we have to "refresh" diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index ec4bc6bbade9..eafef90329a5 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -23,6 +23,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct RenameTableInterpreter { @@ -45,6 +46,7 @@ impl Interpreter for RenameTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { // TODO check privileges // You must have ALTER and DROP privileges for the original table, diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index 299367a0c2b6..c512e0b60c1a 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -26,6 +26,7 @@ use common_tracing::tracing; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::is_internal_opt_key; @@ -49,6 +50,7 @@ impl Interpreter for ShowCreateTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let catalog = self.ctx.get_catalog(); diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index 24c4fa4ad548..2d67af27f5fa 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -23,6 +23,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct TruncateTableInterpreter { @@ -45,6 +46,7 @@ impl Interpreter for TruncateTableInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.as_str(); let tbl_name = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index c7d63d0c9137..f47a87eda302 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -22,6 +22,7 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct UseDatabaseInterpreter { @@ -44,6 +45,7 @@ impl Interpreter for UseDatabaseInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { self.ctx.set_current_database(self.plan.db.clone()).await?; let schema = Arc::new(DataSchema::empty()); diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index d06e7bd66353..749e42ed31b9 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for AlterUserInterpreter { "AlterUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index a6f2f7515692..b21e3a116d41 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -25,6 +25,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -45,10 +46,11 @@ impl Interpreter for CreateUserInterpreter { "CreateUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index 7a506af93e4c..5cb7593f400c 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for DropUserInterpreter { "DropUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index 3981911161e5..b60245e5548d 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -23,6 +23,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -43,10 +44,11 @@ impl Interpreter for CreateUserStageInterpreter { "CreateUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let user_mgr = self.ctx.get_user_manager(); diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index 4cfed88d87dc..952f545f3286 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -25,6 +25,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -48,10 +49,11 @@ impl Interpreter for DescribeUserStageInterpreter { "DescribeUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let user_mgr = self.ctx.get_user_manager(); diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index c294c6ddd494..b861be26942c 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for DropUserStageInterpreter { "DropUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index dfb435b081a5..1cd11c7c05b3 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for AlterUserUDFInterpreter { "AlterUserUDFInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index 40f82a027d32..bfdeef03afe1 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for CreateUserUDFInterpreter { "CreateUserUDFInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index 9b3a1831a465..c57bdac4a533 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -22,6 +22,7 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] @@ -42,10 +43,11 @@ impl Interpreter for DropUserUDFInterpreter { "DropUserUDFInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index 2b910944f7b6..75ad7c45338a 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -29,6 +29,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; @@ -49,7 +50,11 @@ impl Interpreter for AlterViewInterpreter { "AlterViewInterpreter" } - async fn execute(&self, _: Option) -> Result { + async fn execute( + &self, + _: Option, + _source_pipe_builder: Option, + ) -> Result { // check privilige self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 209bef29bf3d..8afc32563f61 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -28,6 +28,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; @@ -48,7 +49,11 @@ impl Interpreter for CreateViewInterpreter { "CreateViewInterpreter" } - async fn execute(&self, _: Option) -> Result { + async fn execute( + &self, + _: Option, + _source_pipe_builder: Option, + ) -> Result { // check privilige self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index 5efc82e89fc1..ec0053ad4133 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -26,6 +26,7 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; @@ -49,6 +50,7 @@ impl Interpreter for DropViewInterpreter { async fn execute( &self, _input_stream: Option, + _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.clone(); let viewname = self.plan.viewname.clone(); diff --git a/query/src/pipelines/new/processors/mod.rs b/query/src/pipelines/new/processors/mod.rs index c7b46a7c067d..868f3791234c 100644 --- a/query/src/pipelines/new/processors/mod.rs +++ b/query/src/pipelines/new/processors/mod.rs @@ -38,6 +38,7 @@ pub use sources::AsyncSource; pub use sources::AsyncSourcer; pub use sources::BlocksSource; pub use sources::EmptySource; +pub use sources::SyncReceiverCkSource; pub use sources::SyncReceiverSource; pub use sources::SyncSource; pub use sources::SyncSourcer; diff --git a/query/src/pipelines/new/processors/sources/mod.rs b/query/src/pipelines/new/processors/sources/mod.rs index f7c08c1b32d9..8ee836f76299 100644 --- a/query/src/pipelines/new/processors/sources/mod.rs +++ b/query/src/pipelines/new/processors/sources/mod.rs @@ -15,6 +15,7 @@ mod async_source; mod blocks_source; mod empty_source; +mod sync_ck_source_receiver; mod sync_source; mod sync_source_receiver; @@ -22,6 +23,7 @@ pub use async_source::AsyncSource; pub use async_source::AsyncSourcer; pub use blocks_source::BlocksSource; pub use empty_source::EmptySource; +pub use sync_ck_source_receiver::SyncReceiverCkSource; pub use sync_source::SyncSource; pub use sync_source::SyncSourcer; pub use sync_source_receiver::SyncReceiverSource; diff --git a/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs new file mode 100644 index 000000000000..8ff493712ab8 --- /dev/null +++ b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs @@ -0,0 +1,59 @@ +// Copyright 2022 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::tokio::sync::mpsc::Receiver; +use common_datablocks::DataBlock; +use common_datavalues::DataSchemaRef; +use common_exception::Result; +use opensrv_clickhouse::types::Block; + +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::sources::sync_source::SyncSource; +use crate::pipelines::new::processors::sources::SyncSourcer; +use crate::servers::clickhouse::from_clickhouse_block; +use crate::sessions::QueryContext; + +pub struct SyncReceiverCkSource { + schema: DataSchemaRef, + receiver: Receiver, +} + +impl SyncReceiverCkSource { + pub fn create( + ctx: Arc, + rx: Receiver, + out: Arc, + schema: DataSchemaRef, + ) -> Result { + SyncSourcer::create(ctx, out, SyncReceiverCkSource { + schema, + receiver: rx, + }) + } +} + +#[async_trait::async_trait] +impl SyncSource for SyncReceiverCkSource { + const NAME: &'static str = "SyncReceiverCkSource"; + + fn generate(&mut self) -> Result> { + match self.receiver.blocking_recv() { + None => Ok(None), + Some(block) => Ok(Some(from_clickhouse_block(self.schema.clone(), block)?)), + } + } +} diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 605c69af3fed..2d3deb84aa0a 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -34,6 +34,7 @@ use futures::channel::mpsc; use futures::channel::mpsc::Receiver; use futures::SinkExt; use futures::StreamExt; +use futures::TryFutureExt; use metrics::histogram; use opensrv_clickhouse::types::Block as ClickHouseBlock; use opensrv_clickhouse::CHContext; @@ -42,6 +43,10 @@ use tokio_stream::wrappers::ReceiverStream; use super::writers::from_clickhouse_block; use crate::interpreters::InterpreterFactory; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::SyncReceiverCkSource; +use crate::pipelines::new::processors::SyncReceiverSource; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sessions::SessionRef; use crate::sql::PlanParser; @@ -92,11 +97,19 @@ impl InteractiveWorkerBase { let sc = sample_block.schema().clone(); let stream = ReceiverStream::new(rec); - let stream = FromClickHouseBlockStream { + let ck_stream = FromClickHouseBlockStream { input: stream, - schema: sc, + schema: sc.clone(), }; - + let output_port = OutputPort::create(); + let sync_receiver_ck_source = SyncReceiverCkSource::create( + ctx.clone(), + ck_stream.input.into_inner(), + output_port.clone(), + sc, + )?; + let mut source_pipe_builder = SourcePipeBuilder::create(); + source_pipe_builder.add_source(output_port, sync_receiver_ck_source); let interpreter = InterpreterFactory::get(ctx.clone(), PlanNode::Insert(insert))?; let name = interpreter.name().to_string(); @@ -107,7 +120,10 @@ impl InteractiveWorkerBase { let sent_all_data = ch_ctx.state.sent_all_data.clone(); let start = Instant::now(); ctx.try_spawn(async move { - interpreter.execute(Some(Box::pin(stream))).await.unwrap(); + interpreter + .execute(None, Some(source_pipe_builder)) + .await + .unwrap(); sent_all_data.notify_one(); })?; histogram!( @@ -165,7 +181,7 @@ impl InteractiveWorkerBase { .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); // Execute and read stream data. - let async_data_stream = interpreter.execute(None); + let async_data_stream = interpreter.execute(None, None); let mut data_stream = async_data_stream.await?; while let Some(block) = data_stream.next().await { data_tx.send(BlockItem::Block(block)).await.ok(); diff --git a/query/src/servers/clickhouse/mod.rs b/query/src/servers/clickhouse/mod.rs index e5ebd31ad0de..856b536c8adf 100644 --- a/query/src/servers/clickhouse/mod.rs +++ b/query/src/servers/clickhouse/mod.rs @@ -22,3 +22,4 @@ mod interactive_worker_base; mod reject_connection; pub use clickhouse_handler::ClickHouseHandler; +pub use writers::from_clickhouse_block; diff --git a/query/src/servers/http/clickhouse_handler.rs b/query/src/servers/http/clickhouse_handler.rs index 29610e7ffc56..708235f99cdb 100644 --- a/query/src/servers/http/clickhouse_handler.rs +++ b/query/src/servers/http/clickhouse_handler.rs @@ -71,7 +71,7 @@ async fn execute( .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - let data_stream = interpreter.execute(input_stream).await?; + let data_stream = interpreter.execute(input_stream, None).await?; let mut data_stream = ctx.try_create_abortable(data_stream)?; let stream = stream! { diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 92d1f7bc3068..55b941378ba2 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -154,7 +154,7 @@ pub async fn streaming_load( // this runs inside the runtime of poem, load is not cpu densive so it's ok let mut data_stream = interpreter - .execute(Some(source_stream)) + .execute(Some(source_stream), None) .await .map_err(InternalServerError)?; while let Some(_block) = data_stream.next().await {} diff --git a/query/src/servers/http/v1/query/execute_state.rs b/query/src/servers/http/v1/query/execute_state.rs index b151d82e77e3..93292d3d6e29 100644 --- a/query/src/servers/http/v1/query/execute_state.rs +++ b/query/src/servers/http/v1/query/execute_state.rs @@ -191,7 +191,7 @@ async fn execute( block_tx: mpsc::Sender, abort_rx: &mut mpsc::Receiver<()>, ) -> Result<()> { - let data_stream = interpreter.execute(None).await?; + let data_stream = interpreter.execute(None, None).await?; let mut data_stream = ctx.try_create_abortable(data_stream)?; while let Some(block_r) = data_stream.next().await { match block_r { diff --git a/query/src/servers/mod.rs b/query/src/servers/mod.rs index f2343f3c7f6e..b509819f3bb4 100644 --- a/query/src/servers/mod.rs +++ b/query/src/servers/mod.rs @@ -23,7 +23,7 @@ pub use self::mysql::MySQLConnection; pub use self::mysql::MySQLFederated; pub use self::mysql::MySQLHandler; -mod clickhouse; +pub(crate) mod clickhouse; pub mod http; mod mysql; pub(crate) mod server; diff --git a/query/src/servers/mysql/mysql_interactive_worker.rs b/query/src/servers/mysql/mysql_interactive_worker.rs index 74e6e45429c5..4acc057aea8c 100644 --- a/query/src/servers/mysql/mysql_interactive_worker.rs +++ b/query/src/servers/mysql/mysql_interactive_worker.rs @@ -318,7 +318,7 @@ impl InteractiveWorkerBase { .start() .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - let data_stream = interpreter.execute(None).await?; + let data_stream = interpreter.execute(None, None).await?; histogram!( super::mysql_metrics::METRIC_INTERPRETER_USEDTIME, instant.elapsed() From b0d026a4054dc7b89fa8a94f1a0423702fcb66fc Mon Sep 17 00:00:00 2001 From: xudong963 Date: Sun, 17 Apr 2022 17:59:27 +0800 Subject: [PATCH 04/27] cargo fix to delete unused import --- query/benches/suites/mod.rs | 2 +- query/src/interpreters/interpreter.rs | 2 +- query/src/interpreters/interpreter_copy.rs | 2 +- query/src/interpreters/interpreter_insert.rs | 8 ++--- query/src/interpreters/interpreter_select.rs | 2 +- .../transforms/transform_cast_schema.rs | 4 +-- .../clickhouse/interactive_worker_base.rs | 2 -- .../tests/it/interpreters/interpreter_call.rs | 22 +++++++------- .../interpreter_database_create.rs | 2 +- .../interpreters/interpreter_database_drop.rs | 2 +- .../interpreter_database_show_create.rs | 4 +-- .../it/interpreters/interpreter_explain.rs | 2 +- .../interpreter_factory_interceptor.rs | 8 ++--- .../it/interpreters/interpreter_insert.rs | 18 +++++------ .../interpreter_privilege_grant.rs | 2 +- .../interpreter_privilege_revoke.rs | 4 +-- .../it/interpreters/interpreter_role_grant.rs | 10 +++---- .../interpreters/interpreter_role_revoke.rs | 12 ++++---- .../it/interpreters/interpreter_select.rs | 4 +-- .../it/interpreters/interpreter_setting.rs | 4 +-- .../interpreter_show_databases.rs | 4 +-- .../interpreters/interpreter_show_engines.rs | 2 +- .../interpreter_show_functions.rs | 2 +- .../it/interpreters/interpreter_show_grant.rs | 14 ++++----- .../interpreters/interpreter_show_metrics.rs | 2 +- .../interpreter_show_processlist.rs | 2 +- .../it/interpreters/interpreter_show_roles.rs | 4 +-- .../interpreters/interpreter_show_settings.rs | 2 +- .../interpreters/interpreter_show_tab_stat.rs | 20 ++++++------- .../interpreters/interpreter_show_tables.rs | 30 +++++++++---------- .../it/interpreters/interpreter_show_users.rs | 4 +-- .../interpreters/interpreter_table_create.rs | 8 ++--- .../interpreter_table_describe.rs | 4 +-- .../it/interpreters/interpreter_table_drop.rs | 4 +-- .../interpreters/interpreter_table_rename.rs | 6 ++-- .../interpreter_table_show_create.rs | 4 +-- .../interpreter_table_truncate.rs | 10 +++---- .../interpreters/interpreter_use_database.rs | 4 +-- .../it/interpreters/interpreter_user_alter.rs | 6 ++-- .../interpreters/interpreter_user_create.rs | 2 +- .../it/interpreters/interpreter_user_drop.rs | 6 ++-- .../it/interpreters/interpreter_user_stage.rs | 6 ++-- .../interpreter_user_udf_alter.rs | 4 +-- .../interpreter_user_udf_create.rs | 6 ++-- .../interpreters/interpreter_user_udf_drop.rs | 12 ++++---- .../it/storages/fuse/operations/read_plan.rs | 2 +- query/tests/it/storages/fuse/pruning.rs | 4 +-- query/tests/it/storages/fuse/table.rs | 8 ++--- .../table_functions/fuse_history_table.rs | 2 +- .../it/storages/fuse/table_test_fixture.rs | 4 +-- .../tests/it/table_functions/numbers_table.rs | 2 +- 51 files changed, 152 insertions(+), 154 deletions(-) diff --git a/query/benches/suites/mod.rs b/query/benches/suites/mod.rs index 414c6d3e6618..298e261bca65 100644 --- a/query/benches/suites/mod.rs +++ b/query/benches/suites/mod.rs @@ -35,7 +35,7 @@ pub async fn select_executor(sql: &str) -> Result<()> { if let PlanNode::Select(plan) = PlanParser::parse(ctx.clone(), sql).await? { let executor = SelectInterpreter::try_create(ctx, plan)?; - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} } else { unreachable!() diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index 751a3ebc84f1..f6609b6a2b9f 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -18,7 +18,7 @@ use common_exception::ErrorCode; use common_exception::Result; use common_streams::SendableDataBlockStream; -use crate::pipelines::new::processors::processor::ProcessorPtr; + use crate::pipelines::new::NewPipeline; use crate::pipelines::new::SourcePipeBuilder; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index e8ff71dd2880..aecf090e2197 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -32,7 +32,7 @@ use crate::interpreters::stream::ProcessorExecutorStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::pipelines::new::executor::PipelinePullingExecutor; -use crate::pipelines::new::processors::processor::ProcessorPtr; + use crate::pipelines::new::NewPipeline; use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 904d76da0673..3286170a5843 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -28,10 +28,10 @@ use common_planners::PlanNode; use common_planners::SelectPlan; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; -use common_streams::Source; + use common_tracing::tracing; use futures::TryStreamExt; -use poem::trace; + use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; use crate::interpreters::plan_schedulers; @@ -45,7 +45,7 @@ use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::BlocksSource; use crate::pipelines::new::processors::TransformAddOn; use crate::pipelines::new::processors::TransformCastSchema; -use crate::pipelines::new::processors::TransformDummy; + use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; @@ -69,7 +69,7 @@ impl InsertInterpreter { async fn execute_new( &self, - input_stream: Option, + _input_stream: Option, source_pipe_builder: Option, ) -> Result { let plan = &self.plan; diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index 2fdfb8dd6fd2..2af4898d1ca6 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -27,7 +27,7 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelinePullingExecutor; -use crate::pipelines::new::processors::processor::ProcessorPtr; + use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; diff --git a/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs index 37f22e220824..d539cfeb75bd 100644 --- a/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs +++ b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs @@ -20,14 +20,14 @@ use common_datavalues::DataSchemaRef; use common_exception::Result; use common_functions::scalars::Function; use common_functions::scalars::FunctionContext; -use common_streams::SendableDataBlockStream; + use crate::pipelines::new::processors::port::InputPort; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::transforms::transform::Transform; use crate::pipelines::new::processors::transforms::transform::Transformer; -use crate::pipelines::new::processors::ResizeProcessor; + pub struct TransformCastSchema { output_schema: DataSchemaRef, diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 2d3deb84aa0a..81b3b3426d8d 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -34,7 +34,6 @@ use futures::channel::mpsc; use futures::channel::mpsc::Receiver; use futures::SinkExt; use futures::StreamExt; -use futures::TryFutureExt; use metrics::histogram; use opensrv_clickhouse::types::Block as ClickHouseBlock; use opensrv_clickhouse::CHContext; @@ -45,7 +44,6 @@ use super::writers::from_clickhouse_block; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::SyncReceiverCkSource; -use crate::pipelines::new::processors::SyncReceiverSource; use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sessions::SessionRef; diff --git a/query/tests/it/interpreters/interpreter_call.rs b/query/tests/it/interpreters/interpreter_call.rs index ed0779861c19..21fb869f7fc1 100644 --- a/query/tests/it/interpreters/interpreter_call.rs +++ b/query/tests/it/interpreters/interpreter_call.rs @@ -32,7 +32,7 @@ async fn test_call_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "call system$test()").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); assert_eq!( res.err().unwrap().code(), @@ -52,7 +52,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "call system$fuse_history()").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1028, displayText = Function `FUSE_HISTORY` expect to have 2 arguments, but got 0."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -64,7 +64,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "call system$fuse_history(default, test)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); assert_eq!( res.err().unwrap().code(), @@ -78,7 +78,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "call system$fuse_history(system, tables)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1006, displayText = expecting fuse table, but got table of engine type: SystemTables."; @@ -93,14 +93,14 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // FuseHistory { let plan = PlanParser::parse(ctx.clone(), "call system$fuse_history(default, a)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) @@ -115,7 +115,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "call admin$bootstrap_tenant()").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1028, displayText = Function `BOOTSTRAP_TENANT` expect to have 5 arguments, but got 0."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -129,7 +129,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1062, displayText = Access denied: 'BOOTSTRAP_TENANT' only used in management-mode."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -148,7 +148,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1063, displayText = Access denied: 'BOOTSTRAP_TENANT' requires user TENANTSETTING option flag."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -168,7 +168,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None).await?; + executor.execute(None, None).await?; let user_mgr = ctx.get_user_manager(); let user_info = user_mgr @@ -193,7 +193,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None).await?; + executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_database_create.rs b/query/tests/it/interpreters/interpreter_database_create.rs index af13ad7d24d8..3aa4791e9318 100644 --- a/query/tests/it/interpreters/interpreter_database_create.rs +++ b/query/tests/it/interpreters/interpreter_database_create.rs @@ -28,7 +28,7 @@ async fn test_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CreateDatabaseInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} Ok(()) diff --git a/query/tests/it/interpreters/interpreter_database_drop.rs b/query/tests/it/interpreters/interpreter_database_drop.rs index 21f727cce86e..e79a148b0b4b 100644 --- a/query/tests/it/interpreters/interpreter_database_drop.rs +++ b/query/tests/it/interpreters/interpreter_database_drop.rs @@ -26,7 +26,7 @@ async fn test_drop_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "drop database default").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "DropDatabaseInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_database_show_create.rs b/query/tests/it/interpreters/interpreter_database_show_create.rs index e84e0a5fcc7e..9df227e89eb2 100644 --- a/query/tests/it/interpreters/interpreter_database_show_create.rs +++ b/query/tests/it/interpreters/interpreter_database_show_create.rs @@ -28,7 +28,7 @@ async fn test_show_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show create database default").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateDatabaseInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+---------------------------+", @@ -45,7 +45,7 @@ async fn test_show_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show create database system").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateDatabaseInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+----------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_explain.rs b/query/tests/it/interpreters/interpreter_explain.rs index aa29f030c344..f42d56744d42 100644 --- a/query/tests/it/interpreters/interpreter_explain.rs +++ b/query/tests/it/interpreters/interpreter_explain.rs @@ -32,7 +32,7 @@ async fn test_explain_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx, plan)?; assert_eq!(executor.name(), "ExplainInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); diff --git a/query/tests/it/interpreters/interpreter_factory_interceptor.rs b/query/tests/it/interpreters/interpreter_factory_interceptor.rs index 9776cd2ac480..8a454fbb7c70 100644 --- a/query/tests/it/interpreters/interpreter_factory_interceptor.rs +++ b/query/tests/it/interpreters/interpreter_factory_interceptor.rs @@ -29,7 +29,7 @@ async fn test_interpreter_interceptor() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; interpreter.start().await?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); @@ -59,7 +59,7 @@ async fn test_interpreter_interceptor() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -87,7 +87,7 @@ async fn test_interpreter_interceptor_for_insert() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; interpreter.start().await?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; stream.try_collect::>().await?; interpreter.finish().await?; } @@ -98,7 +98,7 @@ async fn test_interpreter_interceptor_for_insert() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ diff --git a/query/tests/it/interpreters/interpreter_insert.rs b/query/tests/it/interpreters/interpreter_insert.rs index 1aa4de8b6d9b..61b216228ed0 100644 --- a/query/tests/it/interpreters/interpreter_insert.rs +++ b/query/tests/it/interpreters/interpreter_insert.rs @@ -27,7 +27,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.default_value_table(a String, b String DEFAULT 'b') Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Create input table. @@ -35,7 +35,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.input_table(a String, b String, c String, d String, e String) Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Create output table. @@ -43,7 +43,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.output_table(a UInt8, b Int8, c UInt16, d Int16, e String) Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Insert into default value table. @@ -53,7 +53,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.default_value_table(a) values('a')"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // insert into select. @@ -61,7 +61,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.default_value_table(a) select a from default.default_value_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // select. @@ -69,7 +69,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "select * from default.default_value_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---+---+", @@ -88,7 +88,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.input_table values(1,1,1,1,1), (2,2,2,2,2)"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Insert into output table. @@ -97,7 +97,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // select. @@ -105,7 +105,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "select * from default.output_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---+---+---+---+---+", diff --git a/query/tests/it/interpreters/interpreter_privilege_grant.rs b/query/tests/it/interpreters/interpreter_privilege_grant.rs index e14a22bdf938..2f90335968f4 100644 --- a/query/tests/it/interpreters/interpreter_privilege_grant.rs +++ b/query/tests/it/interpreters/interpreter_privilege_grant.rs @@ -115,7 +115,7 @@ async fn test_grant_privilege_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &tt.query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantPrivilegeInterpreter"); - let r = match executor.execute(None).await { + let r = match executor.execute(None, None).await { Err(err) => Err(err), Ok(mut stream) => { while let Some(_block) = stream.next().await {} diff --git a/query/tests/it/interpreters/interpreter_privilege_revoke.rs b/query/tests/it/interpreters/interpreter_privilege_revoke.rs index b691e856b6ac..14790faa3d1d 100644 --- a/query/tests/it/interpreters/interpreter_privilege_revoke.rs +++ b/query/tests/it/interpreters/interpreter_privilege_revoke.rs @@ -49,7 +49,7 @@ async fn test_revoke_privilege_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "RevokePrivilegeInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let new_user = user_mgr.get_user(&tenant, user_info.identity()).await?; assert_eq!(new_user.grants, UserGrantSet::empty()); @@ -78,7 +78,7 @@ async fn test_revoke_privilege_interpreter_on_role() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "RevokePrivilegeInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let role = user_mgr.get_role(&tenant, "role1".to_string()).await?; diff --git a/query/tests/it/interpreters/interpreter_role_grant.rs b/query/tests/it/interpreters/interpreter_role_grant.rs index 0e78f1673bbd..736ceba58b0b 100644 --- a/query/tests/it/interpreters/interpreter_role_grant.rs +++ b/query/tests/it/interpreters/interpreter_role_grant.rs @@ -33,7 +33,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -48,7 +48,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownUser("").code()) } @@ -63,7 +63,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let query = "GRANT ROLE 'test' TO 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let user_info = user_mgr.get_user(&tenant, user_info.identity()).await?; let roles = user_info.grants.roles(); @@ -77,7 +77,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -93,7 +93,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let query = "GRANT ROLE 'test' TO ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); diff --git a/query/tests/it/interpreters/interpreter_role_revoke.rs b/query/tests/it/interpreters/interpreter_role_revoke.rs index 7ed5f5fcf9dc..4bc9d0c117ed 100644 --- a/query/tests/it/interpreters/interpreter_role_revoke.rs +++ b/query/tests/it/interpreters/interpreter_role_revoke.rs @@ -34,7 +34,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RevokeRoleInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownUser("").code()) } @@ -50,7 +50,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let user_info = user_mgr.get_user(&tenant, test_user.identity()).await?; let roles = user_info.grants.roles(); @@ -62,7 +62,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let user_info = user_mgr .get_user(&tenant, UserIdentity::new("test_user", "%")) @@ -77,7 +77,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RevokeRoleInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -93,7 +93,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); @@ -105,7 +105,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); diff --git a/query/tests/it/interpreters/interpreter_select.rs b/query/tests/it/interpreters/interpreter_select.rs index 0ae9a1b94407..73aadd4c6bb2 100644 --- a/query/tests/it/interpreters/interpreter_select.rs +++ b/query/tests/it/interpreters/interpreter_select.rs @@ -30,7 +30,7 @@ async fn test_select_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SelectInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); @@ -60,7 +60,7 @@ async fn test_select_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SelectInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 4); diff --git a/query/tests/it/interpreters/interpreter_setting.rs b/query/tests/it/interpreters/interpreter_setting.rs index c16f34e89a21..711dcc3a0cd2 100644 --- a/query/tests/it/interpreters/interpreter_setting.rs +++ b/query/tests/it/interpreters/interpreter_setting.rs @@ -27,7 +27,7 @@ async fn test_setting_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SettingInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} Ok(()) @@ -39,7 +39,7 @@ async fn test_setting_interpreter_error() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SET max_block_size=1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan)?; - if let Err(e) = executor.execute(None).await { + if let Err(e) = executor.execute(None, None).await { let expect = "Code: 1020, displayText = Unknown variable: \"xx\"."; assert_eq!(expect, format!("{}", e)); } diff --git a/query/tests/it/interpreters/interpreter_show_databases.rs b/query/tests/it/interpreters/interpreter_show_databases.rs index 7e2898600821..9faa34791293 100644 --- a/query/tests/it/interpreters/interpreter_show_databases.rs +++ b/query/tests/it/interpreters/interpreter_show_databases.rs @@ -28,7 +28,7 @@ async fn test_show_databases_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show databases").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowDatabasesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------------------+", @@ -46,7 +46,7 @@ async fn test_show_databases_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "show databases like '%tem%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+", diff --git a/query/tests/it/interpreters/interpreter_show_engines.rs b/query/tests/it/interpreters/interpreter_show_engines.rs index 77126f024857..f6c0633d3abd 100644 --- a/query/tests/it/interpreters/interpreter_show_engines.rs +++ b/query/tests/it/interpreters/interpreter_show_engines.rs @@ -28,7 +28,7 @@ async fn test_show_engines_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show engines").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowEnginesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------+-----------------------------+", diff --git a/query/tests/it/interpreters/interpreter_show_functions.rs b/query/tests/it/interpreters/interpreter_show_functions.rs index d8b9db3eb79a..a5085b5564f7 100644 --- a/query/tests/it/interpreters/interpreter_show_functions.rs +++ b/query/tests/it/interpreters/interpreter_show_functions.rs @@ -26,7 +26,7 @@ async fn test_show_functions_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show functions").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowFunctionsInterpreter"); - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_grant.rs b/query/tests/it/interpreters/interpreter_show_grant.rs index 837234cb5a5e..a0fcf1abb7e6 100644 --- a/query/tests/it/interpreters/interpreter_show_grant.rs +++ b/query/tests/it/interpreters/interpreter_show_grant.rs @@ -43,7 +43,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowGrantsInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["+--------+", "| Grants |", "+--------+", "+--------+"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -54,7 +54,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowGrantsInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["+--------+", "| Grants |", "+--------+", "+--------+"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -73,7 +73,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role2'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------+", @@ -98,7 +98,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR 'test'@'localhost'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------------------------------------------------+", @@ -125,7 +125,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR 'test'@'localhost'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------------------------+", @@ -145,7 +145,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role1'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------+", @@ -169,7 +169,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role1'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_show_metrics.rs b/query/tests/it/interpreters/interpreter_show_metrics.rs index 469f82fc7bb5..4357c5862e5c 100644 --- a/query/tests/it/interpreters/interpreter_show_metrics.rs +++ b/query/tests/it/interpreters/interpreter_show_metrics.rs @@ -28,7 +28,7 @@ async fn test_show_metrics_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show metrics").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowMetricsInterpreter"); - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_processlist.rs b/query/tests/it/interpreters/interpreter_show_processlist.rs index 3d8b301572b6..9bfdc3212076 100644 --- a/query/tests/it/interpreters/interpreter_show_processlist.rs +++ b/query/tests/it/interpreters/interpreter_show_processlist.rs @@ -26,7 +26,7 @@ async fn test_show_processlist_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show processlist").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowProcesslistInterpreter"); - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_roles.rs b/query/tests/it/interpreters/interpreter_show_roles.rs index 19e3999a2bd2..39fc374707bb 100644 --- a/query/tests/it/interpreters/interpreter_show_roles.rs +++ b/query/tests/it/interpreters/interpreter_show_roles.rs @@ -26,7 +26,7 @@ async fn test_show_roles_interpreter() -> Result<()> { let query = "CREATE ROLE test"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // show roles. @@ -35,7 +35,7 @@ async fn test_show_roles_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowRolesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------+-----------------+", diff --git a/query/tests/it/interpreters/interpreter_show_settings.rs b/query/tests/it/interpreters/interpreter_show_settings.rs index 266ea2e109eb..7ac9e4b48124 100644 --- a/query/tests/it/interpreters/interpreter_show_settings.rs +++ b/query/tests/it/interpreters/interpreter_show_settings.rs @@ -28,7 +28,7 @@ async fn test_show_settings_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowSettingsInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; // let expected = vec![ // "+------------------------------------+---------+---------+---------+--------------------------------------------------------------------------------------------------------------------------------------------+--------+", diff --git a/query/tests/it/interpreters/interpreter_show_tab_stat.rs b/query/tests/it/interpreters/interpreter_show_tab_stat.rs index d007197f137a..29351bb799e6 100644 --- a/query/tests/it/interpreters/interpreter_show_tab_stat.rs +++ b/query/tests/it/interpreters/interpreter_show_tab_stat.rs @@ -29,26 +29,26 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Use database. { let plan = PlanParser::parse(ctx.clone(), "use db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Create table. { let plan = PlanParser::parse(ctx.clone(), "create table data(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } { let plan = PlanParser::parse(ctx.clone(), "create table bend(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } } @@ -57,7 +57,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -75,7 +75,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -92,7 +92,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status where Name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -109,7 +109,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -127,7 +127,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -145,7 +145,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "drop database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_tables.rs b/query/tests/it/interpreters/interpreter_show_tables.rs index a7ed39b9a4cb..91942b75b5d9 100644 --- a/query/tests/it/interpreters/interpreter_show_tables.rs +++ b/query/tests/it/interpreters/interpreter_show_tables.rs @@ -29,26 +29,26 @@ async fn test_show_tables_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Use database. { let plan = PlanParser::parse(ctx.clone(), "use db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Create table. { let plan = PlanParser::parse(ctx.clone(), "create table data(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } { let plan = PlanParser::parse(ctx.clone(), "create table bend(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } } @@ -57,7 +57,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -75,7 +75,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -93,7 +93,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -110,7 +110,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -127,7 +127,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables where table_name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -145,7 +145,7 @@ async fn test_show_tables_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "show full tables where table_name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -162,7 +162,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -180,7 +180,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -198,7 +198,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -217,7 +217,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -235,7 +235,7 @@ async fn test_show_tables_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "drop database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_users.rs b/query/tests/it/interpreters/interpreter_show_users.rs index dc7107a03e2f..5404890f4e85 100644 --- a/query/tests/it/interpreters/interpreter_show_users.rs +++ b/query/tests/it/interpreters/interpreter_show_users.rs @@ -26,7 +26,7 @@ async fn test_show_users_interpreter() -> Result<()> { let query = "CREATE USER 'test'@'localhost' IDENTIFIED BY 'password'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // show users. @@ -35,7 +35,7 @@ async fn test_show_users_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowUsersInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------+-----------+----------------------+------------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_table_create.rs b/query/tests/it/interpreters/interpreter_table_create.rs index 76b2cabd76c9..177bf4e4f3d8 100644 --- a/query/tests/it/interpreters/interpreter_table_create.rs +++ b/query/tests/it/interpreters/interpreter_table_create.rs @@ -31,7 +31,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = interpreter.execute(None).await?; + let mut stream = interpreter.execute(None, None).await?; while let Some(_block) = stream.next().await {} let schema = plan.schema(); @@ -58,7 +58,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), TEST_CREATE_QUERY).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } { @@ -67,7 +67,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), TEST_CREATE_QUERY_SELECT).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = interpreter.execute(None).await?; + let mut stream = interpreter.execute(None, None).await?; while let Some(_block) = stream.next().await {} let schema = plan.schema(); @@ -97,7 +97,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - assert!(interpreter.execute(None).await.is_err()); + assert!(interpreter.execute(None, None).await.is_err()); } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_table_describe.rs b/query/tests/it/interpreters/interpreter_table_describe.rs index e4be0d46a04d..f39f1eae7cdb 100644 --- a/query/tests/it/interpreters/interpreter_table_describe.rs +++ b/query/tests/it/interpreters/interpreter_table_describe.rs @@ -33,7 +33,7 @@ async fn interpreter_describe_table_test() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = interpreter.execute(None).await?; + let _ = interpreter.execute(None, None).await?; } // describe table. @@ -42,7 +42,7 @@ async fn interpreter_describe_table_test() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DescribeTableInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------+----------+------+---------+-------+", diff --git a/query/tests/it/interpreters/interpreter_table_drop.rs b/query/tests/it/interpreters/interpreter_table_drop.rs index e44ebeecacf2..d9f4ea2cde17 100644 --- a/query/tests/it/interpreters/interpreter_table_drop.rs +++ b/query/tests/it/interpreters/interpreter_table_drop.rs @@ -33,7 +33,7 @@ async fn test_drop_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Drop table. @@ -41,7 +41,7 @@ async fn test_drop_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "DROP TABLE a").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropTableInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_table_rename.rs b/query/tests/it/interpreters/interpreter_table_rename.rs index 60d93c272ec6..c23ca19ff0fa 100644 --- a/query/tests/it/interpreters/interpreter_table_rename.rs +++ b/query/tests/it/interpreters/interpreter_table_rename.rs @@ -33,7 +33,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // Rename table. @@ -41,7 +41,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "RENAME TABLE a TO b").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RenameTableInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -52,7 +52,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "DROP TABLE b").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropTableInterpreter"); - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_table_show_create.rs b/query/tests/it/interpreters/interpreter_table_show_create.rs index fd553116986f..602fcedf4ab1 100644 --- a/query/tests/it/interpreters/interpreter_table_show_create.rs +++ b/query/tests/it/interpreters/interpreter_table_show_create.rs @@ -76,13 +76,13 @@ async fn interpreter_show_create_table_test() -> Result<()> { for stmt in case.create_stmt { let plan = PlanParser::parse(ctx.clone(), stmt).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } let plan = PlanParser::parse(ctx.clone(), case.show_stmt).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateTableInterpreter"); let result = executor - .execute(None) + .execute(None, None) .await? .try_collect::>() .await?; diff --git a/query/tests/it/interpreters/interpreter_table_truncate.rs b/query/tests/it/interpreters/interpreter_table_truncate.rs index a8b94056aa43..122c22704440 100644 --- a/query/tests/it/interpreters/interpreter_table_truncate.rs +++ b/query/tests/it/interpreters/interpreter_table_truncate.rs @@ -33,7 +33,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = interpreter.execute(None).await?; + let _ = interpreter.execute(None, None).await?; } // Insert into. @@ -41,7 +41,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "INSERT INTO default.a VALUES('1,1', '2,2')"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None).await?; + let _ = executor.execute(None, None).await?; } // select. @@ -49,7 +49,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "SELECT * FROM default.a"; let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-----+-----+", @@ -68,7 +68,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(interpreter.name(), "TruncateTableInterpreter"); - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -79,7 +79,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "SELECT * FROM default.a"; let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = interpreter.execute(None).await?; + let stream = interpreter.execute(None, None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_use_database.rs b/query/tests/it/interpreters/interpreter_use_database.rs index 76f97e0a1945..8e9530258697 100644 --- a/query/tests/it/interpreters/interpreter_use_database.rs +++ b/query/tests/it/interpreters/interpreter_use_database.rs @@ -27,7 +27,7 @@ async fn test_use_interpreter() -> Result<()> { let interpreter = InterpreterFactory::get(ctx, plan)?; assert_eq!(interpreter.name(), "UseDatabaseInterpreter"); - let mut stream = interpreter.execute(None).await?; + let mut stream = interpreter.execute(None, None).await?; while let Some(_block) = stream.next().await {} Ok(()) @@ -40,7 +40,7 @@ async fn test_use_database_interpreter_error() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "USE xx").await?; let interpreter = InterpreterFactory::get(ctx, plan)?; - if let Err(e) = interpreter.execute(None).await { + if let Err(e) = interpreter.execute(None, None).await { let expect = "Code: 1003, displayText = Cannot USE 'xx', because the 'xx' doesn't exist."; assert_eq!(expect, format!("{}", e)); } diff --git a/query/tests/it/interpreters/interpreter_user_alter.rs b/query/tests/it/interpreters/interpreter_user_alter.rs index 160f462a3fb4..563b8942baa4 100644 --- a/query/tests/it/interpreters/interpreter_user_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_alter.rs @@ -57,7 +57,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let new_user = user_mgr.get_user(tenant, user_info.identity()).await?; assert_eq!( @@ -75,7 +75,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - executor.execute(None).await?; + executor.execute(None, None).await?; let user_info = user_mgr.get_user(tenant, user_info.identity()).await?; assert!(user_info.has_option_flag(UserOptionFlag::TenantSetting)); assert_eq!( @@ -89,7 +89,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - executor.execute(None).await?; + executor.execute(None, None).await?; let user_info = user_mgr.get_user(tenant, user_info.identity()).await?; assert!(!user_info.has_option_flag(UserOptionFlag::TenantSetting)); } diff --git a/query/tests/it/interpreters/interpreter_user_create.rs b/query/tests/it/interpreters/interpreter_user_create.rs index af3530c85db3..e27347656a92 100644 --- a/query/tests/it/interpreters/interpreter_user_create.rs +++ b/query/tests/it/interpreters/interpreter_user_create.rs @@ -29,7 +29,7 @@ async fn test_create_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CreateUserInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} Ok(()) diff --git a/query/tests/it/interpreters/interpreter_user_drop.rs b/query/tests/it/interpreters/interpreter_user_drop.rs index 8eaf840ff15e..a6f829046f17 100644 --- a/query/tests/it/interpreters/interpreter_user_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_drop.rs @@ -33,7 +33,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - let ret = executor.execute(None).await; + let ret = executor.execute(None, None).await; assert!(ret.is_err()) } @@ -42,7 +42,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - let ret = executor.execute(None).await; + let ret = executor.execute(None, None).await; assert!(ret.is_ok()) } @@ -69,7 +69,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - executor.execute(None).await?; + executor.execute(None, None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_user_stage.rs b/query/tests/it/interpreters/interpreter_user_stage.rs index b30dfc989b2f..52d16bfd939c 100644 --- a/query/tests/it/interpreters/interpreter_user_stage.rs +++ b/query/tests/it/interpreters/interpreter_user_stage.rs @@ -32,7 +32,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserStageInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} } @@ -43,7 +43,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DescribeUserStageInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; let mut blocks = vec![]; while let Some(block) = stream.next().await { @@ -74,7 +74,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserStageInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} } diff --git a/query/tests/it/interpreters/interpreter_user_udf_alter.rs b/query/tests/it/interpreters/interpreter_user_udf_alter.rs index df90c0aee11e..08d84110a0d6 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_alter.rs @@ -32,7 +32,7 @@ async fn test_alter_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -52,7 +52,7 @@ async fn test_alter_udf_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserUDFInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let udf = ctx diff --git a/query/tests/it/interpreters/interpreter_user_udf_create.rs b/query/tests/it/interpreters/interpreter_user_udf_create.rs index f2ea46485ff3..4cc6510ff3a0 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_create.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_create.rs @@ -34,7 +34,7 @@ async fn test_create_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -51,7 +51,7 @@ async fn test_create_udf_interpreter() -> Result<()> { // IF NOT EXISTS. let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None).await?; + executor.execute(None, None).await?; let udf = ctx .get_user_manager() @@ -69,7 +69,7 @@ async fn test_create_udf_interpreter() -> Result<()> { "CREATE FUNCTION isnotempty AS (p) -> not(is_null(p)) DESC = 'This is a description'"; let plan = PlanParser::parse(ctx.clone(), query1).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let r = executor.execute(None).await; + let r = executor.execute(None, None).await; assert!(r.is_err()); let e = r.err(); assert_eq!(e.unwrap().code(), ErrorCode::udf_already_exists_code()); diff --git a/query/tests/it/interpreters/interpreter_user_udf_drop.rs b/query/tests/it/interpreters/interpreter_user_udf_drop.rs index 0c389a78d301..4c8587a0f771 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_drop.rs @@ -35,7 +35,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), CREATE_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -52,7 +52,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_ok()); } @@ -60,7 +60,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF_IF_EXISTS).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_ok()); } @@ -68,7 +68,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_err()); } @@ -76,7 +76,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), CREATE_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None).await?; + let mut stream = executor.execute(None, None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -92,7 +92,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF_IF_EXISTS).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None).await; + let res = executor.execute(None, None).await; assert!(res.is_ok()); } diff --git a/query/tests/it/storages/fuse/operations/read_plan.rs b/query/tests/it/storages/fuse/operations/read_plan.rs index 7f8e08ac940d..a94a8aa6866e 100644 --- a/query/tests/it/storages/fuse/operations/read_plan.rs +++ b/query/tests/it/storages/fuse/operations/read_plan.rs @@ -118,7 +118,7 @@ async fn test_fuse_table_exact_statistic() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; let mut table = fixture.latest_default_table().await?; diff --git a/query/tests/it/storages/fuse/pruning.rs b/query/tests/it/storages/fuse/pruning.rs index bbaff98061a2..f6537c50d3b5 100644 --- a/query/tests/it/storages/fuse/pruning.rs +++ b/query/tests/it/storages/fuse/pruning.rs @@ -90,7 +90,7 @@ async fn test_block_pruner() -> Result<()> { }; let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; // get table let catalog = ctx.get_catalog(); @@ -231,7 +231,7 @@ async fn test_block_pruner_monotonic() -> Result<()> { let catalog = ctx.get_catalog(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; // get table let table = catalog diff --git a/query/tests/it/storages/fuse/table.rs b/query/tests/it/storages/fuse/table.rs index d1db318745b3..7238ec4b2f3f 100644 --- a/query/tests/it/storages/fuse/table.rs +++ b/query/tests/it/storages/fuse/table.rs @@ -38,7 +38,7 @@ async fn test_fuse_table_normal_case() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; let mut table = fixture.latest_default_table().await?; @@ -163,7 +163,7 @@ async fn test_fuse_table_truncate() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; let table = fixture.latest_default_table().await?; let truncate_plan = TruncateTablePlan { @@ -234,7 +234,7 @@ async fn test_fuse_table_optimize() -> Result<()> { let tbl_name = create_table_plan.table.clone(); let db_name = create_table_plan.db.clone(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; // insert 5 times let n = 5; @@ -266,7 +266,7 @@ async fn test_fuse_table_optimize() -> Result<()> { // To avoid flaky test, the value of setting `max_threads` is set to be 1, so that pipeline_builder will // only arrange one worker for the `ReadDataSourcePlan`. ctx.get_settings().set_max_threads(1)?; - let data_stream = interpreter.execute(None).await?; + let data_stream = interpreter.execute(None, None).await?; let _ = data_stream.try_collect::>(); // verify compaction diff --git a/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs b/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs index 3c5bce49fedb..3a3df3127eda 100644 --- a/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs +++ b/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs @@ -80,7 +80,7 @@ async fn test_fuse_history_table_read() -> Result<()> { // test db & table let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; // func args let arg_db = Expression::create_literal(DataValue::String(db.as_bytes().to_vec())); diff --git a/query/tests/it/storages/fuse/table_test_fixture.rs b/query/tests/it/storages/fuse/table_test_fixture.rs index 0533b2c5df16..c9ea2e04a2e8 100644 --- a/query/tests/it/storages/fuse/table_test_fixture.rs +++ b/query/tests/it/storages/fuse/table_test_fixture.rs @@ -131,7 +131,7 @@ impl TestFixture { pub async fn create_default_table(&self) -> Result<()> { let create_table_plan = self.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(self.ctx.clone(), create_table_plan)?; - interpreter.execute(None).await?; + interpreter.execute(None, None).await?; Ok(()) } @@ -269,7 +269,7 @@ pub async fn expects_ok( pub async fn execute_query(ctx: Arc, query: &str) -> Result { let plan = PlanParser::parse(ctx.clone(), query).await?; InterpreterFactory::get(ctx.clone(), plan)? - .execute(None) + .execute(None, None) .await } diff --git a/query/tests/it/table_functions/numbers_table.rs b/query/tests/it/table_functions/numbers_table.rs index 187671b0f20b..7a4355ed02d9 100644 --- a/query/tests/it/table_functions/numbers_table.rs +++ b/query/tests/it/table_functions/numbers_table.rs @@ -114,7 +114,7 @@ async fn test_limit_push_down() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = executor.execute(None).await?; + let stream = executor.execute(None, None).await?; let result = stream.try_collect::>().await?; let expect = test.result; let actual = result.as_slice(); From 3202703c3e09f86e8b0502379b832f6254d209bf Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 17:22:27 +0800 Subject: [PATCH 05/27] refactor the method of execute in interpreter --- query/benches/suites/mod.rs | 2 +- query/src/interpreters/interpreter.rs | 8 +++- query/src/interpreters/interpreter_call.rs | 8 +++- query/src/interpreters/interpreter_copy.rs | 9 ++-- .../interpreter_database_create.rs | 8 +++- .../interpreters/interpreter_database_drop.rs | 6 ++- .../interpreter_database_show_create.rs | 6 ++- query/src/interpreters/interpreter_explain.rs | 6 ++- .../interpreter_factory_interceptor.rs | 23 +++++++--- query/src/interpreters/interpreter_insert.rs | 39 ++++++++++------ query/src/interpreters/interpreter_kill.rs | 6 ++- query/src/interpreters/interpreter_list.rs | 8 +++- .../interpreter_privilege_grant.rs | 8 +++- .../interpreter_privilege_revoke.rs | 8 +++- .../src/interpreters/interpreter_query_log.rs | 1 + .../interpreters/interpreter_role_create.rs | 8 +++- .../src/interpreters/interpreter_role_drop.rs | 8 +++- .../interpreters/interpreter_role_grant.rs | 8 +++- .../interpreters/interpreter_role_revoke.rs | 8 +++- query/src/interpreters/interpreter_select.rs | 9 ++-- query/src/interpreters/interpreter_setting.rs | 6 ++- .../interpreter_show_databases.rs | 8 +++- .../interpreters/interpreter_show_engines.rs | 8 +++- .../interpreter_show_functions.rs | 8 +++- .../interpreters/interpreter_show_grants.rs | 6 ++- .../interpreters/interpreter_show_metrics.rs | 8 +++- .../interpreter_show_processlist.rs | 8 +++- .../interpreters/interpreter_show_roles.rs | 8 +++- .../interpreters/interpreter_show_settings.rs | 8 +++- .../interpreters/interpreter_show_tab_stat.rs | 8 +++- .../interpreters/interpreter_show_tables.rs | 8 +++- .../interpreters/interpreter_show_users.rs | 8 +++- .../interpreters/interpreter_table_create.rs | 8 +++- .../interpreter_table_describe.rs | 6 ++- .../interpreters/interpreter_table_drop.rs | 6 ++- .../interpreter_table_optimize.rs | 8 +++- .../interpreters/interpreter_table_rename.rs | 6 ++- .../interpreter_table_show_create.rs | 6 ++- .../interpreter_table_truncate.rs | 6 ++- .../interpreters/interpreter_use_database.rs | 6 ++- .../interpreters/interpreter_user_alter.rs | 8 +++- .../interpreters/interpreter_user_create.rs | 8 +++- .../src/interpreters/interpreter_user_drop.rs | 8 +++- .../interpreter_user_stage_create.rs | 8 +++- .../interpreter_user_stage_describe.rs | 8 +++- .../interpreter_user_stage_drop.rs | 8 +++- .../interpreter_user_udf_alter.rs | 8 +++- .../interpreter_user_udf_create.rs | 8 +++- .../interpreters/interpreter_user_udf_drop.rs | 8 +++- .../interpreters/interpreter_view_alter.rs | 11 ++--- .../interpreters/interpreter_view_create.rs | 11 ++--- .../src/interpreters/interpreter_view_drop.rs | 6 ++- query/src/pipelines/new/pipe.rs | 1 + .../sources/sync_ck_source_receiver.rs | 9 ++-- .../transforms/transform_cast_schema.rs | 2 - .../clickhouse/interactive_worker_base.rs | 44 ++++++++++++++++--- query/src/servers/http/clickhouse_handler.rs | 2 +- query/src/servers/http/v1/load.rs | 2 +- .../servers/http/v1/query/execute_state.rs | 2 +- .../servers/mysql/mysql_interactive_worker.rs | 2 +- .../tests/it/interpreters/interpreter_call.rs | 22 +++++----- .../interpreter_database_create.rs | 2 +- .../interpreters/interpreter_database_drop.rs | 2 +- .../interpreter_database_show_create.rs | 4 +- .../it/interpreters/interpreter_explain.rs | 2 +- .../interpreter_factory_interceptor.rs | 8 ++-- .../it/interpreters/interpreter_insert.rs | 18 ++++---- .../interpreter_privilege_grant.rs | 2 +- .../interpreter_privilege_revoke.rs | 4 +- .../it/interpreters/interpreter_role_grant.rs | 10 ++--- .../interpreters/interpreter_role_revoke.rs | 12 ++--- .../it/interpreters/interpreter_select.rs | 4 +- .../it/interpreters/interpreter_setting.rs | 4 +- .../interpreter_show_databases.rs | 4 +- .../interpreters/interpreter_show_engines.rs | 2 +- .../interpreter_show_functions.rs | 2 +- .../it/interpreters/interpreter_show_grant.rs | 14 +++--- .../interpreters/interpreter_show_metrics.rs | 2 +- .../interpreter_show_processlist.rs | 2 +- .../it/interpreters/interpreter_show_roles.rs | 4 +- .../interpreters/interpreter_show_settings.rs | 2 +- .../interpreters/interpreter_show_tab_stat.rs | 20 ++++----- .../interpreters/interpreter_show_tables.rs | 30 ++++++------- .../it/interpreters/interpreter_show_users.rs | 4 +- .../interpreters/interpreter_table_create.rs | 8 ++-- .../interpreter_table_describe.rs | 4 +- .../it/interpreters/interpreter_table_drop.rs | 4 +- .../interpreters/interpreter_table_rename.rs | 6 +-- .../interpreter_table_show_create.rs | 4 +- .../interpreter_table_truncate.rs | 10 ++--- .../interpreters/interpreter_use_database.rs | 4 +- .../it/interpreters/interpreter_user_alter.rs | 6 +-- .../interpreters/interpreter_user_create.rs | 2 +- .../it/interpreters/interpreter_user_drop.rs | 6 +-- .../it/interpreters/interpreter_user_stage.rs | 6 +-- .../interpreter_user_udf_alter.rs | 4 +- .../interpreter_user_udf_create.rs | 6 +-- .../interpreters/interpreter_user_udf_drop.rs | 12 ++--- .../it/storages/fuse/operations/read_plan.rs | 2 +- query/tests/it/storages/fuse/pruning.rs | 4 +- query/tests/it/storages/fuse/table.rs | 8 ++-- .../table_functions/fuse_history_table.rs | 2 +- .../it/storages/fuse/table_test_fixture.rs | 4 +- .../tests/it/table_functions/numbers_table.rs | 2 +- 104 files changed, 509 insertions(+), 270 deletions(-) diff --git a/query/benches/suites/mod.rs b/query/benches/suites/mod.rs index 298e261bca65..414c6d3e6618 100644 --- a/query/benches/suites/mod.rs +++ b/query/benches/suites/mod.rs @@ -35,7 +35,7 @@ pub async fn select_executor(sql: &str) -> Result<()> { if let PlanNode::Select(plan) = PlanParser::parse(ctx.clone(), sql).await? { let executor = SelectInterpreter::try_create(ctx, plan)?; - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} } else { unreachable!() diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index f6609b6a2b9f..844e2470b59e 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -12,13 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; +use std::sync::Arc; + use common_datavalues::DataSchemaRef; use common_datavalues::DataSchemaRefExt; use common_exception::ErrorCode; use common_exception::Result; +use common_infallible::Mutex; use common_streams::SendableDataBlockStream; - use crate::pipelines::new::NewPipeline; use crate::pipelines::new::SourcePipeBuilder; @@ -29,6 +32,8 @@ pub trait Interpreter: Sync + Send { /// Return the name of Interpreter, such as "CreateDatabaseInterpreter" fn name(&self) -> &str; + fn as_any(&self) -> &dyn Any; + /// Return the schema of Interpreter fn schema(&self) -> DataSchemaRef { DataSchemaRefExt::create(vec![]) @@ -38,7 +43,6 @@ pub trait Interpreter: Sync + Send { async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result; /// Create the new pipeline for databend's new execution model diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index 22c40b15b5e4..21e59d18b42f 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for CallInterpreter { "CallInterpreter" } - #[tracing::instrument(level = "debug", name = "call_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", name = "call_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = &self.plan; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index aecf090e2197..7f385a029b80 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::path::Path; use std::sync::Arc; @@ -32,7 +33,6 @@ use crate::interpreters::stream::ProcessorExecutorStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::pipelines::new::executor::PipelinePullingExecutor; - use crate::pipelines::new::NewPipeline; use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; @@ -139,11 +139,14 @@ impl Interpreter for CopyInterpreter { "CopyInterpreter" } - #[tracing::instrument(level = "debug", name = "copy_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", name = "copy_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let mut files = self.list_files().await?; diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index 983f6681ebc2..c78b716da2b2 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -12,6 +12,7 @@ // 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; @@ -46,11 +47,14 @@ impl Interpreter for CreateDatabaseInterpreter { "CreateDatabaseInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index b3e8b434ba41..55ef4e324c21 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -12,6 +12,7 @@ // 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; @@ -44,10 +45,13 @@ impl Interpreter for DropDatabaseInterpreter { "DropDatabaseInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index 7c924d6b88fa..bbf0045fd0db 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -47,10 +48,13 @@ impl Interpreter for ShowCreateDatabaseInterpreter { "ShowCreateDatabaseInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let calalog = self.ctx.get_catalog(); diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 754e1dafcaff..9fed342b72d0 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -41,10 +42,13 @@ impl Interpreter for ExplainInterpreter { "ExplainInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let schema = self.schema(); diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 91f32c976c6d..63ff898f2c66 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use std::time::SystemTime; @@ -26,6 +27,7 @@ use crate::interpreters::InterpreterQueryLog; use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; +#[derive(Clone)] pub struct InterceptorInterpreter { ctx: Arc, inner: InterpreterPtr, @@ -40,6 +42,17 @@ impl InterceptorInterpreter { query_log: InterpreterQueryLog::create(ctx, plan), } } + + pub fn get_inner(&self) -> &InterpreterPtr { + &self.inner + } + + pub fn set_insert_inner(&mut self, interpreter: Box) { + self.inner = InterpreterPtr::from(interpreter); + } + pub fn get_box(self) -> Box { + Box::new(self) + } } #[async_trait::async_trait] @@ -48,15 +61,15 @@ impl Interpreter for InterceptorInterpreter { self.inner.name() } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { - let result_stream = self - .inner - .execute(input_stream, source_pipe_builder) - .await?; + let result_stream = self.inner.execute(input_stream).await?; let metric_stream = ProgressStream::try_create(result_stream, self.ctx.get_result_progress())?; Ok(Box::pin(metric_stream)) diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 3286170a5843..afae4e78eb0d 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; @@ -28,11 +29,9 @@ use common_planners::PlanNode; use common_planners::SelectPlan; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; - use common_tracing::tracing; use futures::TryStreamExt; - use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; use crate::interpreters::plan_schedulers; use crate::interpreters::plan_schedulers::InsertWithPlan; @@ -45,17 +44,17 @@ use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::BlocksSource; use crate::pipelines::new::processors::TransformAddOn; use crate::pipelines::new::processors::TransformCastSchema; - use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::transforms::AddOnStream; use crate::sessions::QueryContext; +#[derive(Clone)] pub struct InsertInterpreter { ctx: Arc, plan: InsertPlan, - source_processor: Option, + source_pipe_builder: Option, } impl InsertInterpreter { @@ -63,14 +62,21 @@ impl InsertInterpreter { Ok(Arc::new(InsertInterpreter { ctx, plan, - source_processor: None, + source_pipe_builder: None, })) } + pub fn set_source_pipe_builder(&mut self, builder: Option) { + self.source_pipe_builder = builder; + } + + pub fn get_box(self) -> Box { + Box::new(self) + } + async fn execute_new( &self, _input_stream: Option, - source_pipe_builder: Option, ) -> Result { let plan = &self.plan; let settings = self.ctx.get_settings(); @@ -96,10 +102,13 @@ impl InsertInterpreter { pipeline.add_pipe(builder.finalize()); } InsertInputSource::StreamingWithFormat(_) => { - tracing::info!("come here"); pipeline.add_pipe( - source_pipe_builder - .ok_or_else(|| ErrorCode::EmptyData("empty"))? + self.clone() + .get_box() + .as_mut() + .source_pipe_builder + .take() + .ok_or_else(|| ErrorCode::EmptyData("empty source pipe builder"))? .finalize(), ); } @@ -200,18 +209,20 @@ impl Interpreter for InsertInterpreter { "InsertIntoInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, mut input_stream: Option, - source_pipe_builder: Option, ) -> Result { let settings = self.ctx.get_settings(); - if true - || (settings.get_enable_new_processor_framework()? == 2 - && self.ctx.get_cluster().is_empty()) + /// Use insert in new processor + if settings.get_enable_new_processor_framework()? != 0 && self.ctx.get_cluster().is_empty() { - return self.execute_new(input_stream, source_pipe_builder).await; + return self.execute_new(input_stream).await; } let plan = &self.plan; diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index 4cbf24ab2600..bfe38dd9c216 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datavalues::DataSchema; @@ -45,10 +46,13 @@ impl Interpreter for KillInterpreter { "KillInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index 49a8a0609eaa..ad42d0e4adb5 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -75,11 +76,14 @@ impl Interpreter for ListInterpreter { "ListInterpreter" } - #[tracing::instrument(level = "debug", name = "list_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", name = "list_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, mut _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let files = self.list_files().await?; tracing::info!("list file list:{:?}, pattern:{}", &files, self.plan.pattern); diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index 9aceec9c1807..6c222e0f7c58 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -12,6 +12,7 @@ // 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; @@ -47,11 +48,14 @@ impl Interpreter for GrantPrivilegeInterpreter { "GrantPrivilegeInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index a2c5713d321a..93289cb2a332 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -12,6 +12,7 @@ // 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; @@ -45,11 +46,14 @@ impl Interpreter for RevokePrivilegeInterpreter { "RevokePrivilegeInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_query_log.rs b/query/src/interpreters/interpreter_query_log.rs index 8380f04cc8d9..b39638284df7 100644 --- a/query/src/interpreters/interpreter_query_log.rs +++ b/query/src/interpreters/interpreter_query_log.rs @@ -98,6 +98,7 @@ pub struct LogEvent { pub extra: String, } +#[derive(Clone)] pub struct InterpreterQueryLog { ctx: Arc, plan: PlanNode, diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index 24904cfa8c94..fa95e361901c 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -12,6 +12,7 @@ // 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; @@ -44,11 +45,14 @@ impl Interpreter for CreateRoleInterpreter { "CreateRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { // TODO: add privilege check about CREATE ROLE let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index 1b5b5fe44fdc..31de47e41d21 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for DropRoleInterpreter { "DropRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { // TODO: add privilege check about DROP role let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index 18ce9482965c..61deb483f5c2 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -12,6 +12,7 @@ // 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; @@ -44,11 +45,14 @@ impl Interpreter for GrantRoleInterpreter { "GrantRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index 8221a3405333..e1aa75c809d6 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -12,6 +12,7 @@ // 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; @@ -44,11 +45,14 @@ impl Interpreter for RevokeRoleInterpreter { "RevokeRoleInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index 2af4898d1ca6..f9385d17670b 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datavalues::DataSchemaRef; @@ -27,7 +28,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelinePullingExecutor; - use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; @@ -61,19 +61,22 @@ impl Interpreter for SelectInterpreter { "SelectInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + /// Get the schema of SelectPlan fn schema(&self) -> DataSchemaRef { self.select.schema() } - #[tracing::instrument(level = "debug", name = "select_interpreter_execute", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + #[tracing::instrument(level = "debug", name = "select_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] /// Currently, the method has two sets of logic, if `get_enable_new_processor_framework` is turned on in the settings, /// the execution will use the new processor, otherwise the old processing logic will be executed. /// Note: there is an issue to track the progress of the new processor: https://github.com/datafuselabs/databend/issues/3379 async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let settings = self.ctx.get_settings(); diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index af45fa895c5b..6b519c77fe3e 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use chrono_tz::Tz; @@ -44,10 +45,13 @@ impl Interpreter for SettingInterpreter { "SettingInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.set.clone(); for var in plan.vars { diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 7d094acb33d0..85fbce98d238 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -62,10 +63,13 @@ impl Interpreter for ShowDatabasesInterpreter { "ShowDatabasesInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -73,7 +77,7 @@ impl Interpreter for ShowDatabasesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show databases build query error")); } diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 62f8925e2eae..137db02c849a 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -50,10 +51,13 @@ impl Interpreter for ShowEnginesInterpreter { "ShowEnginesInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -61,7 +65,7 @@ impl Interpreter for ShowEnginesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show engines build query error")); } diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index 29b7115a6231..d3504447fff5 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -63,10 +64,13 @@ impl Interpreter for ShowFunctionsInterpreter { "ShowFunctionsInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -74,7 +78,7 @@ impl Interpreter for ShowFunctionsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show functions build query error")); } diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index 23c7cb8db530..af2b2ce3c1af 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -44,10 +45,13 @@ impl Interpreter for ShowGrantsInterpreter { "ShowGrantsInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let schema = DataSchemaRefExt::create(vec![DataField::new("Grants", Vu8::to_data_type())]); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index 326d9700cef7..38528fbc4608 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -50,10 +51,13 @@ impl Interpreter for ShowMetricsInterpreter { "ShowMetricsInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -61,7 +65,7 @@ impl Interpreter for ShowMetricsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show metrics build query error")); } diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index 77c94a2dbf1a..f59f9c7bbe41 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -53,10 +54,13 @@ impl Interpreter for ShowProcessListInterpreter { "ShowProcesslistInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -64,7 +68,7 @@ impl Interpreter for ShowProcessListInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError( "Show processlist build query error", diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index b1a0460ca261..a243b17f204f 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -50,10 +51,13 @@ impl Interpreter for ShowRolesInterpreter { "ShowRolesInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -61,7 +65,7 @@ impl Interpreter for ShowRolesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show roles build query error")); } diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index 9831d2f742da..38e4521bd7ab 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -50,10 +51,13 @@ impl Interpreter for ShowSettingsInterpreter { "ShowSettingsInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -61,7 +65,7 @@ impl Interpreter for ShowSettingsInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show settings build query error")); } diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index 64a1ae4adde2..32f8cfcaa675 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -89,10 +90,13 @@ impl Interpreter for ShowTabStatInterpreter { "ShowTabStatInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -100,7 +104,7 @@ impl Interpreter for ShowTabStatInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError( "Show table status build query error", diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index ff7634c3595f..e58e57c52349 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -80,10 +81,13 @@ impl Interpreter for ShowTablesInterpreter { "ShowTablesInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -91,7 +95,7 @@ impl Interpreter for ShowTablesInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show tables build query error")); } diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index 8b6b72d452f9..ee92f53bb29d 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -53,10 +54,13 @@ impl Interpreter for ShowUsersInterpreter { "ShowUsersInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - source_pipe_builder: Option, ) -> Result { let query = self.build_query()?; let plan = PlanParser::parse(self.ctx.clone(), &query).await?; @@ -64,7 +68,7 @@ impl Interpreter for ShowUsersInterpreter { if let PlanNode::Select(plan) = optimized { let interpreter = SelectInterpreter::try_create(self.ctx.clone(), plan)?; - interpreter.execute(input_stream, source_pipe_builder).await + interpreter.execute(input_stream).await } else { return Err(ErrorCode::LogicalError("Show users build query error")); } diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index a71a9f0f2a36..4e3637457e5a 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datavalues::DataField; @@ -51,10 +52,13 @@ impl Interpreter for CreateTableInterpreter { "CreateTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, input_stream: Option, - _source_pipe_builder: Option, ) -> Result { self.ctx .get_current_session() @@ -140,7 +144,7 @@ impl CreateTableInterpreter { source: InsertInputSource::SelectPlan(select_plan_node), }; let insert_interpreter = InsertInterpreter::try_create(self.ctx.clone(), insert_plan)?; - insert_interpreter.execute(input_stream, None).await?; + insert_interpreter.execute(input_stream).await?; Ok(Box::pin(DataBlockStream::create( self.plan.schema(), diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index d3696eb94a79..686cea0c3fa1 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -44,10 +45,13 @@ impl Interpreter for DescribeTableInterpreter { "DescribeTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let database = self.plan.db.as_str(); let table = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index 529d24c7d614..f1f6109e7b5e 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -46,10 +47,13 @@ impl Interpreter for DropTableInterpreter { "DropTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.as_str(); let tbl_name = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index e8f07c478cfb..65750d695a3b 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -12,6 +12,7 @@ // 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; @@ -46,10 +47,13 @@ impl Interpreter for OptimizeTableInterpreter { "OptimizeTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = &self.plan; let mut table = self.ctx.get_table(&plan.database, &plan.table).await?; @@ -66,7 +70,7 @@ impl Interpreter for OptimizeTableInterpreter { let rewritten_plan = PlanParser::parse(self.ctx.clone(), rewritten_query.as_str()).await?; let interpreter = InterpreterFactory::get(self.ctx.clone(), rewritten_plan)?; - let mut stream = interpreter.execute(None, None).await?; + let mut stream = interpreter.execute(None).await?; while let Some(Ok(_)) = stream.next().await {} if do_purge { // currently, context caches the table, we have to "refresh" diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index eafef90329a5..0111ba714bf2 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -12,6 +12,7 @@ // 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; @@ -43,10 +44,13 @@ impl Interpreter for RenameTableInterpreter { "RenameTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { // TODO check privileges // You must have ALTER and DROP privileges for the original table, diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index c512e0b60c1a..7a1af0bb2497 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -47,10 +48,13 @@ impl Interpreter for ShowCreateTableInterpreter { "ShowCreateTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let catalog = self.ctx.get_catalog(); diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index 2d67af27f5fa..c20c3dbb8c07 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -12,6 +12,7 @@ // 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; @@ -43,10 +44,13 @@ impl Interpreter for TruncateTableInterpreter { "TruncateTableInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.as_str(); let tbl_name = self.plan.table.as_str(); diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index f47a87eda302..bf169b99b457 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datavalues::DataSchema; @@ -42,10 +43,13 @@ impl Interpreter for UseDatabaseInterpreter { "UseDatabaseInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { self.ctx.set_current_database(self.plan.db.clone()).await?; let schema = Arc::new(DataSchema::empty()); diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index 749e42ed31b9..2e63ed2c612e 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for AlterUserInterpreter { "AlterUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index b21e3a116d41..e344090c827b 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -12,6 +12,7 @@ // 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; @@ -46,11 +47,14 @@ impl Interpreter for CreateUserInterpreter { "CreateUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index 5cb7593f400c..4ea5451492cb 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for DropUserInterpreter { "DropUserInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index b60245e5548d..ef8c415c79d5 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -12,6 +12,7 @@ // 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; @@ -44,11 +45,14 @@ impl Interpreter for CreateUserStageInterpreter { "CreateUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let user_mgr = self.ctx.get_user_manager(); diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index 952f545f3286..5d4743b3ab56 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::sync::Arc; use common_datablocks::DataBlock; @@ -49,11 +50,14 @@ impl Interpreter for DescribeUserStageInterpreter { "DescribeUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let tenant = self.ctx.get_tenant(); let user_mgr = self.ctx.get_user_manager(); diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index b861be26942c..4a75388a202e 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for DropUserStageInterpreter { "DropUserStageInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index 1cd11c7c05b3..e834189a0834 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for AlterUserUDFInterpreter { "AlterUserUDFInterpreter" } - #[tracing::instrument(level = "debug", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index bfdeef03afe1..9f1695211488 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for CreateUserUDFInterpreter { "CreateUserUDFInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index c57bdac4a533..b48f0e005b6a 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -12,6 +12,7 @@ // 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; @@ -43,11 +44,14 @@ impl Interpreter for DropUserUDFInterpreter { "DropUserUDFInterpreter" } - #[tracing::instrument(level = "info", skip(self, _input_stream, _source_pipe_builder), fields(ctx.id = self.ctx.get_id().as_str()))] + fn as_any(&self) -> &dyn Any { + self + } + + #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let plan = self.plan.clone(); let tenant = self.ctx.get_tenant(); diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index 75ad7c45338a..b8c30107c686 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeMap; use std::sync::Arc; @@ -50,11 +51,11 @@ impl Interpreter for AlterViewInterpreter { "AlterViewInterpreter" } - async fn execute( - &self, - _: Option, - _source_pipe_builder: Option, - ) -> Result { + fn as_any(&self) -> &dyn Any { + self + } + + async fn execute(&self, _: Option) -> Result { // check privilige self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 8afc32563f61..97db6f571423 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::any::Any; use std::collections::BTreeMap; use std::sync::Arc; @@ -49,11 +50,11 @@ impl Interpreter for CreateViewInterpreter { "CreateViewInterpreter" } - async fn execute( - &self, - _: Option, - _source_pipe_builder: Option, - ) -> Result { + fn as_any(&self) -> &dyn Any { + self + } + + async fn execute(&self, _: Option) -> Result { // check privilige self.ctx .get_current_session() diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index ec0053ad4133..cd24aab06b95 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -12,6 +12,7 @@ // 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::ErrorCode; @@ -47,10 +48,13 @@ impl Interpreter for DropViewInterpreter { "DropViewInterpreter" } + fn as_any(&self) -> &dyn Any { + self + } + async fn execute( &self, _input_stream: Option, - _source_pipe_builder: Option, ) -> Result { let db_name = self.plan.db.clone(); let viewname = self.plan.viewname.clone(); diff --git a/query/src/pipelines/new/pipe.rs b/query/src/pipelines/new/pipe.rs index 81b956aa5739..a142a3d29304 100644 --- a/query/src/pipelines/new/pipe.rs +++ b/query/src/pipelines/new/pipe.rs @@ -55,6 +55,7 @@ impl NewPipe { } } +#[derive(Clone)] pub struct SourcePipeBuilder { processors: Vec, outputs_port: Vec>, diff --git a/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs index 8ff493712ab8..a348811fa89f 100644 --- a/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs +++ b/query/src/pipelines/new/processors/sources/sync_ck_source_receiver.rs @@ -35,14 +35,11 @@ pub struct SyncReceiverCkSource { impl SyncReceiverCkSource { pub fn create( ctx: Arc, - rx: Receiver, - out: Arc, + receiver: Receiver, + output_port: Arc, schema: DataSchemaRef, ) -> Result { - SyncSourcer::create(ctx, out, SyncReceiverCkSource { - schema, - receiver: rx, - }) + SyncSourcer::create(ctx, output_port, SyncReceiverCkSource { schema, receiver }) } } diff --git a/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs index d539cfeb75bd..8590a02c3dda 100644 --- a/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs +++ b/query/src/pipelines/new/processors/transforms/transform_cast_schema.rs @@ -21,14 +21,12 @@ use common_exception::Result; use common_functions::scalars::Function; use common_functions::scalars::FunctionContext; - use crate::pipelines::new::processors::port::InputPort; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::transforms::transform::Transform; use crate::pipelines::new::processors::transforms::transform::Transformer; - pub struct TransformCastSchema { output_schema: DataSchemaRef, functions: Vec>, diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 81b3b3426d8d..2cac22b9466f 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::BorrowMut; +use std::ops::Deref; +use std::ops::DerefMut; use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; @@ -41,6 +44,9 @@ use tokio_stream::wrappers::IntervalStream; use tokio_stream::wrappers::ReceiverStream; use super::writers::from_clickhouse_block; +use crate::interpreters::InsertInterpreter; +use crate::interpreters::InterceptorInterpreter; +use crate::interpreters::Interpreter; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::SyncReceiverCkSource; @@ -108,7 +114,38 @@ impl InteractiveWorkerBase { )?; let mut source_pipe_builder = SourcePipeBuilder::create(); source_pipe_builder.add_source(output_port, sync_receiver_ck_source); + let interpreter = InterpreterFactory::get(ctx.clone(), PlanNode::Insert(insert))?; + // Get the specific `InterceptorInterpreter`, then the inner `InsertInterpreter` + let interceptor_interpreter = match interpreter + .as_any() + .downcast_ref::() + { + Some(interceptor) => interceptor, + None => panic!("Interpreter isn't a InterceptorInterpreter!"), + }; + // In the context of this function, inner must be `InsertInterpreter` + let insert_interpreter = interceptor_interpreter.get_inner(); + // Get the specific `InsertInterpreter` + let insert_interpreter = match insert_interpreter + .as_any() + .downcast_ref::() + { + Some(insert) => insert, + None => panic!("Interpreter isn't a InsertInterpreter!"), + }; + let mut insert_interpreter_box = insert_interpreter.clone().get_box(); + // Set `SourcePipeBuilder` to `InsertInterpreter`, used in insert source is `StreamingWithFormat` + insert_interpreter_box + .as_mut() + .set_source_pipe_builder(Some(source_pipe_builder)); + + // Set the newest `InsertInterpreter` to `InterceptorInterpreter` + let mut interceptor_interpreter_box = interceptor_interpreter.clone().get_box(); + interceptor_interpreter_box + .as_mut() + .set_insert_inner(insert_interpreter_box as Box); + let name = interpreter.name().to_string(); let (mut tx, rx) = mpsc::channel(20); @@ -118,10 +155,7 @@ impl InteractiveWorkerBase { let sent_all_data = ch_ctx.state.sent_all_data.clone(); let start = Instant::now(); ctx.try_spawn(async move { - interpreter - .execute(None, Some(source_pipe_builder)) - .await - .unwrap(); + interceptor_interpreter_box.execute(None).await.unwrap(); sent_all_data.notify_one(); })?; histogram!( @@ -179,7 +213,7 @@ impl InteractiveWorkerBase { .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); // Execute and read stream data. - let async_data_stream = interpreter.execute(None, None); + let async_data_stream = interpreter.execute(None); let mut data_stream = async_data_stream.await?; while let Some(block) = data_stream.next().await { data_tx.send(BlockItem::Block(block)).await.ok(); diff --git a/query/src/servers/http/clickhouse_handler.rs b/query/src/servers/http/clickhouse_handler.rs index 708235f99cdb..29610e7ffc56 100644 --- a/query/src/servers/http/clickhouse_handler.rs +++ b/query/src/servers/http/clickhouse_handler.rs @@ -71,7 +71,7 @@ async fn execute( .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - let data_stream = interpreter.execute(input_stream, None).await?; + let data_stream = interpreter.execute(input_stream).await?; let mut data_stream = ctx.try_create_abortable(data_stream)?; let stream = stream! { diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 55b941378ba2..92d1f7bc3068 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -154,7 +154,7 @@ pub async fn streaming_load( // this runs inside the runtime of poem, load is not cpu densive so it's ok let mut data_stream = interpreter - .execute(Some(source_stream), None) + .execute(Some(source_stream)) .await .map_err(InternalServerError)?; while let Some(_block) = data_stream.next().await {} diff --git a/query/src/servers/http/v1/query/execute_state.rs b/query/src/servers/http/v1/query/execute_state.rs index 93292d3d6e29..b151d82e77e3 100644 --- a/query/src/servers/http/v1/query/execute_state.rs +++ b/query/src/servers/http/v1/query/execute_state.rs @@ -191,7 +191,7 @@ async fn execute( block_tx: mpsc::Sender, abort_rx: &mut mpsc::Receiver<()>, ) -> Result<()> { - let data_stream = interpreter.execute(None, None).await?; + let data_stream = interpreter.execute(None).await?; let mut data_stream = ctx.try_create_abortable(data_stream)?; while let Some(block_r) = data_stream.next().await { match block_r { diff --git a/query/src/servers/mysql/mysql_interactive_worker.rs b/query/src/servers/mysql/mysql_interactive_worker.rs index 4acc057aea8c..74e6e45429c5 100644 --- a/query/src/servers/mysql/mysql_interactive_worker.rs +++ b/query/src/servers/mysql/mysql_interactive_worker.rs @@ -318,7 +318,7 @@ impl InteractiveWorkerBase { .start() .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - let data_stream = interpreter.execute(None, None).await?; + let data_stream = interpreter.execute(None).await?; histogram!( super::mysql_metrics::METRIC_INTERPRETER_USEDTIME, instant.elapsed() diff --git a/query/tests/it/interpreters/interpreter_call.rs b/query/tests/it/interpreters/interpreter_call.rs index 21fb869f7fc1..ed0779861c19 100644 --- a/query/tests/it/interpreters/interpreter_call.rs +++ b/query/tests/it/interpreters/interpreter_call.rs @@ -32,7 +32,7 @@ async fn test_call_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "call system$test()").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); assert_eq!( res.err().unwrap().code(), @@ -52,7 +52,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "call system$fuse_history()").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1028, displayText = Function `FUSE_HISTORY` expect to have 2 arguments, but got 0."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -64,7 +64,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "call system$fuse_history(default, test)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); assert_eq!( res.err().unwrap().code(), @@ -78,7 +78,7 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "call system$fuse_history(system, tables)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CallInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1006, displayText = expecting fuse table, but got table of engine type: SystemTables."; @@ -93,14 +93,14 @@ async fn test_call_fuse_history_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // FuseHistory { let plan = PlanParser::parse(ctx.clone(), "call system$fuse_history(default, a)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) @@ -115,7 +115,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "call admin$bootstrap_tenant()").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1028, displayText = Function `BOOTSTRAP_TENANT` expect to have 5 arguments, but got 0."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -129,7 +129,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1062, displayText = Access denied: 'BOOTSTRAP_TENANT' only used in management-mode."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -148,7 +148,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert_eq!(res.is_err(), true); let expect = "Code: 1063, displayText = Access denied: 'BOOTSTRAP_TENANT' requires user TENANTSETTING option flag."; assert_eq!(expect, res.err().unwrap().to_string()); @@ -168,7 +168,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None, None).await?; + executor.execute(None).await?; let user_mgr = ctx.get_user_manager(); let user_info = user_mgr @@ -193,7 +193,7 @@ async fn test_call_bootstrap_tenant_interpreter() -> Result<()> { ) .await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None, None).await?; + executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_database_create.rs b/query/tests/it/interpreters/interpreter_database_create.rs index 3aa4791e9318..af13ad7d24d8 100644 --- a/query/tests/it/interpreters/interpreter_database_create.rs +++ b/query/tests/it/interpreters/interpreter_database_create.rs @@ -28,7 +28,7 @@ async fn test_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CreateDatabaseInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} Ok(()) diff --git a/query/tests/it/interpreters/interpreter_database_drop.rs b/query/tests/it/interpreters/interpreter_database_drop.rs index e79a148b0b4b..21f727cce86e 100644 --- a/query/tests/it/interpreters/interpreter_database_drop.rs +++ b/query/tests/it/interpreters/interpreter_database_drop.rs @@ -26,7 +26,7 @@ async fn test_drop_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "drop database default").await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "DropDatabaseInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_database_show_create.rs b/query/tests/it/interpreters/interpreter_database_show_create.rs index 9df227e89eb2..e84e0a5fcc7e 100644 --- a/query/tests/it/interpreters/interpreter_database_show_create.rs +++ b/query/tests/it/interpreters/interpreter_database_show_create.rs @@ -28,7 +28,7 @@ async fn test_show_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show create database default").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateDatabaseInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+---------------------------+", @@ -45,7 +45,7 @@ async fn test_show_create_database_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show create database system").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateDatabaseInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+----------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_explain.rs b/query/tests/it/interpreters/interpreter_explain.rs index f42d56744d42..aa29f030c344 100644 --- a/query/tests/it/interpreters/interpreter_explain.rs +++ b/query/tests/it/interpreters/interpreter_explain.rs @@ -32,7 +32,7 @@ async fn test_explain_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx, plan)?; assert_eq!(executor.name(), "ExplainInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); diff --git a/query/tests/it/interpreters/interpreter_factory_interceptor.rs b/query/tests/it/interpreters/interpreter_factory_interceptor.rs index 8a454fbb7c70..9776cd2ac480 100644 --- a/query/tests/it/interpreters/interpreter_factory_interceptor.rs +++ b/query/tests/it/interpreters/interpreter_factory_interceptor.rs @@ -29,7 +29,7 @@ async fn test_interpreter_interceptor() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; interpreter.start().await?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); @@ -59,7 +59,7 @@ async fn test_interpreter_interceptor() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -87,7 +87,7 @@ async fn test_interpreter_interceptor_for_insert() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; interpreter.start().await?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; stream.try_collect::>().await?; interpreter.finish().await?; } @@ -98,7 +98,7 @@ async fn test_interpreter_interceptor_for_insert() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ diff --git a/query/tests/it/interpreters/interpreter_insert.rs b/query/tests/it/interpreters/interpreter_insert.rs index 61b216228ed0..1aa4de8b6d9b 100644 --- a/query/tests/it/interpreters/interpreter_insert.rs +++ b/query/tests/it/interpreters/interpreter_insert.rs @@ -27,7 +27,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.default_value_table(a String, b String DEFAULT 'b') Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Create input table. @@ -35,7 +35,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.input_table(a String, b String, c String, d String, e String) Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Create output table. @@ -43,7 +43,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "create table default.output_table(a UInt8, b Int8, c UInt16, d Int16, e String) Engine = Memory"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Insert into default value table. @@ -53,7 +53,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.default_value_table(a) values('a')"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // insert into select. @@ -61,7 +61,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.default_value_table(a) select a from default.default_value_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // select. @@ -69,7 +69,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "select * from default.default_value_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---+---+", @@ -88,7 +88,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "insert into default.input_table values(1,1,1,1,1), (2,2,2,2,2)"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Insert into output table. @@ -97,7 +97,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // select. @@ -105,7 +105,7 @@ async fn test_insert_into_interpreter() -> Result<()> { let query = "select * from default.output_table"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---+---+---+---+---+", diff --git a/query/tests/it/interpreters/interpreter_privilege_grant.rs b/query/tests/it/interpreters/interpreter_privilege_grant.rs index 2f90335968f4..e14a22bdf938 100644 --- a/query/tests/it/interpreters/interpreter_privilege_grant.rs +++ b/query/tests/it/interpreters/interpreter_privilege_grant.rs @@ -115,7 +115,7 @@ async fn test_grant_privilege_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &tt.query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantPrivilegeInterpreter"); - let r = match executor.execute(None, None).await { + let r = match executor.execute(None).await { Err(err) => Err(err), Ok(mut stream) => { while let Some(_block) = stream.next().await {} diff --git a/query/tests/it/interpreters/interpreter_privilege_revoke.rs b/query/tests/it/interpreters/interpreter_privilege_revoke.rs index 14790faa3d1d..b691e856b6ac 100644 --- a/query/tests/it/interpreters/interpreter_privilege_revoke.rs +++ b/query/tests/it/interpreters/interpreter_privilege_revoke.rs @@ -49,7 +49,7 @@ async fn test_revoke_privilege_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "RevokePrivilegeInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let new_user = user_mgr.get_user(&tenant, user_info.identity()).await?; assert_eq!(new_user.grants, UserGrantSet::empty()); @@ -78,7 +78,7 @@ async fn test_revoke_privilege_interpreter_on_role() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "RevokePrivilegeInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let role = user_mgr.get_role(&tenant, "role1".to_string()).await?; diff --git a/query/tests/it/interpreters/interpreter_role_grant.rs b/query/tests/it/interpreters/interpreter_role_grant.rs index 736ceba58b0b..0e78f1673bbd 100644 --- a/query/tests/it/interpreters/interpreter_role_grant.rs +++ b/query/tests/it/interpreters/interpreter_role_grant.rs @@ -33,7 +33,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -48,7 +48,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownUser("").code()) } @@ -63,7 +63,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let query = "GRANT ROLE 'test' TO 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let user_info = user_mgr.get_user(&tenant, user_info.identity()).await?; let roles = user_info.grants.roles(); @@ -77,7 +77,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "GrantRoleInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -93,7 +93,7 @@ async fn test_grant_role_interpreter() -> Result<()> { let query = "GRANT ROLE 'test' TO ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); diff --git a/query/tests/it/interpreters/interpreter_role_revoke.rs b/query/tests/it/interpreters/interpreter_role_revoke.rs index 4bc9d0c117ed..7ed5f5fcf9dc 100644 --- a/query/tests/it/interpreters/interpreter_role_revoke.rs +++ b/query/tests/it/interpreters/interpreter_role_revoke.rs @@ -34,7 +34,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RevokeRoleInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownUser("").code()) } @@ -50,7 +50,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let user_info = user_mgr.get_user(&tenant, test_user.identity()).await?; let roles = user_info.grants.roles(); @@ -62,7 +62,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM 'test_user'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let user_info = user_mgr .get_user(&tenant, UserIdentity::new("test_user", "%")) @@ -77,7 +77,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RevokeRoleInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); assert_eq!(res.err().unwrap().code(), ErrorCode::UnknownRole("").code()) } @@ -93,7 +93,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); @@ -105,7 +105,7 @@ async fn test_revoke_role_interpreter() -> Result<()> { let query = "REVOKE ROLE 'test' FROM ROLE 'test_role'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; let role_info = user_mgr.get_role(&tenant, test_role.identity()).await?; let roles = role_info.grants.roles(); diff --git a/query/tests/it/interpreters/interpreter_select.rs b/query/tests/it/interpreters/interpreter_select.rs index 73aadd4c6bb2..0ae9a1b94407 100644 --- a/query/tests/it/interpreters/interpreter_select.rs +++ b/query/tests/it/interpreters/interpreter_select.rs @@ -30,7 +30,7 @@ async fn test_select_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SelectInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 1); @@ -60,7 +60,7 @@ async fn test_select_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SelectInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let block = &result[0]; assert_eq!(block.num_columns(), 4); diff --git a/query/tests/it/interpreters/interpreter_setting.rs b/query/tests/it/interpreters/interpreter_setting.rs index 711dcc3a0cd2..c16f34e89a21 100644 --- a/query/tests/it/interpreters/interpreter_setting.rs +++ b/query/tests/it/interpreters/interpreter_setting.rs @@ -27,7 +27,7 @@ async fn test_setting_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; assert_eq!(executor.name(), "SettingInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} Ok(()) @@ -39,7 +39,7 @@ async fn test_setting_interpreter_error() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SET max_block_size=1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan)?; - if let Err(e) = executor.execute(None, None).await { + if let Err(e) = executor.execute(None).await { let expect = "Code: 1020, displayText = Unknown variable: \"xx\"."; assert_eq!(expect, format!("{}", e)); } diff --git a/query/tests/it/interpreters/interpreter_show_databases.rs b/query/tests/it/interpreters/interpreter_show_databases.rs index 9faa34791293..7e2898600821 100644 --- a/query/tests/it/interpreters/interpreter_show_databases.rs +++ b/query/tests/it/interpreters/interpreter_show_databases.rs @@ -28,7 +28,7 @@ async fn test_show_databases_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show databases").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowDatabasesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------------------+", @@ -46,7 +46,7 @@ async fn test_show_databases_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "show databases like '%tem%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+----------+", diff --git a/query/tests/it/interpreters/interpreter_show_engines.rs b/query/tests/it/interpreters/interpreter_show_engines.rs index f6c0633d3abd..77126f024857 100644 --- a/query/tests/it/interpreters/interpreter_show_engines.rs +++ b/query/tests/it/interpreters/interpreter_show_engines.rs @@ -28,7 +28,7 @@ async fn test_show_engines_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show engines").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowEnginesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------+-----------------------------+", diff --git a/query/tests/it/interpreters/interpreter_show_functions.rs b/query/tests/it/interpreters/interpreter_show_functions.rs index a5085b5564f7..d8b9db3eb79a 100644 --- a/query/tests/it/interpreters/interpreter_show_functions.rs +++ b/query/tests/it/interpreters/interpreter_show_functions.rs @@ -26,7 +26,7 @@ async fn test_show_functions_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show functions").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowFunctionsInterpreter"); - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_grant.rs b/query/tests/it/interpreters/interpreter_show_grant.rs index a0fcf1abb7e6..837234cb5a5e 100644 --- a/query/tests/it/interpreters/interpreter_show_grant.rs +++ b/query/tests/it/interpreters/interpreter_show_grant.rs @@ -43,7 +43,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowGrantsInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["+--------+", "| Grants |", "+--------+", "+--------+"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -54,7 +54,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowGrantsInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["+--------+", "| Grants |", "+--------+", "+--------+"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -73,7 +73,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role2'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------+", @@ -98,7 +98,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR 'test'@'localhost'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------------------------------------------------+", @@ -125,7 +125,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR 'test'@'localhost'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------------------------+", @@ -145,7 +145,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role1'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------------------------------------+", @@ -169,7 +169,7 @@ async fn test_show_grant_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "SHOW GRANTS FOR ROLE 'role1'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+--------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_show_metrics.rs b/query/tests/it/interpreters/interpreter_show_metrics.rs index 4357c5862e5c..469f82fc7bb5 100644 --- a/query/tests/it/interpreters/interpreter_show_metrics.rs +++ b/query/tests/it/interpreters/interpreter_show_metrics.rs @@ -28,7 +28,7 @@ async fn test_show_metrics_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show metrics").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowMetricsInterpreter"); - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_processlist.rs b/query/tests/it/interpreters/interpreter_show_processlist.rs index 9bfdc3212076..3d8b301572b6 100644 --- a/query/tests/it/interpreters/interpreter_show_processlist.rs +++ b/query/tests/it/interpreters/interpreter_show_processlist.rs @@ -26,7 +26,7 @@ async fn test_show_processlist_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show processlist").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowProcesslistInterpreter"); - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_roles.rs b/query/tests/it/interpreters/interpreter_show_roles.rs index 39fc374707bb..19e3999a2bd2 100644 --- a/query/tests/it/interpreters/interpreter_show_roles.rs +++ b/query/tests/it/interpreters/interpreter_show_roles.rs @@ -26,7 +26,7 @@ async fn test_show_roles_interpreter() -> Result<()> { let query = "CREATE ROLE test"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // show roles. @@ -35,7 +35,7 @@ async fn test_show_roles_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowRolesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------+-----------------+", diff --git a/query/tests/it/interpreters/interpreter_show_settings.rs b/query/tests/it/interpreters/interpreter_show_settings.rs index 7ac9e4b48124..266ea2e109eb 100644 --- a/query/tests/it/interpreters/interpreter_show_settings.rs +++ b/query/tests/it/interpreters/interpreter_show_settings.rs @@ -28,7 +28,7 @@ async fn test_show_settings_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowSettingsInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; // let expected = vec![ // "+------------------------------------+---------+---------+---------+--------------------------------------------------------------------------------------------------------------------------------------------+--------+", diff --git a/query/tests/it/interpreters/interpreter_show_tab_stat.rs b/query/tests/it/interpreters/interpreter_show_tab_stat.rs index 29351bb799e6..d007197f137a 100644 --- a/query/tests/it/interpreters/interpreter_show_tab_stat.rs +++ b/query/tests/it/interpreters/interpreter_show_tab_stat.rs @@ -29,26 +29,26 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Use database. { let plan = PlanParser::parse(ctx.clone(), "use db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Create table. { let plan = PlanParser::parse(ctx.clone(), "create table data(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } { let plan = PlanParser::parse(ctx.clone(), "create table bend(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } } @@ -57,7 +57,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -75,7 +75,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -92,7 +92,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status where Name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -109,7 +109,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ r"\+------\+--------\+---------\+------------\+------\+----------------\+-------------\+-----------------\+--------------\+-----------\+----------------\+-------------------------------\+-------------\+------------\+-----------\+----------\+---------\+", @@ -127,7 +127,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show table status in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTabStatInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -145,7 +145,7 @@ async fn test_show_tab_stat_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "drop database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_tables.rs b/query/tests/it/interpreters/interpreter_show_tables.rs index 91942b75b5d9..a7ed39b9a4cb 100644 --- a/query/tests/it/interpreters/interpreter_show_tables.rs +++ b/query/tests/it/interpreters/interpreter_show_tables.rs @@ -29,26 +29,26 @@ async fn test_show_tables_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "create database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Use database. { let plan = PlanParser::parse(ctx.clone(), "use db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Create table. { let plan = PlanParser::parse(ctx.clone(), "create table data(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } { let plan = PlanParser::parse(ctx.clone(), "create table bend(a Int)").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } } @@ -57,7 +57,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -75,7 +75,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -93,7 +93,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -110,7 +110,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables like '%da%'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -127,7 +127,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables where table_name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -145,7 +145,7 @@ async fn test_show_tables_interpreter() -> Result<()> { PlanParser::parse(ctx.clone(), "show full tables where table_name != 'data'").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -162,7 +162,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+", @@ -180,7 +180,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables from db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+---------------+------------+", @@ -198,7 +198,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show tables in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -217,7 +217,7 @@ async fn test_show_tables_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "show full tables in db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowTablesInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ @@ -235,7 +235,7 @@ async fn test_show_tables_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), "drop database db1").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_show_users.rs b/query/tests/it/interpreters/interpreter_show_users.rs index 5404890f4e85..dc7107a03e2f 100644 --- a/query/tests/it/interpreters/interpreter_show_users.rs +++ b/query/tests/it/interpreters/interpreter_show_users.rs @@ -26,7 +26,7 @@ async fn test_show_users_interpreter() -> Result<()> { let query = "CREATE USER 'test'@'localhost' IDENTIFIED BY 'password'"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // show users. @@ -35,7 +35,7 @@ async fn test_show_users_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowUsersInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+------+-----------+----------------------+------------------------------------------+", diff --git a/query/tests/it/interpreters/interpreter_table_create.rs b/query/tests/it/interpreters/interpreter_table_create.rs index 177bf4e4f3d8..76b2cabd76c9 100644 --- a/query/tests/it/interpreters/interpreter_table_create.rs +++ b/query/tests/it/interpreters/interpreter_table_create.rs @@ -31,7 +31,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = interpreter.execute(None, None).await?; + let mut stream = interpreter.execute(None).await?; while let Some(_block) = stream.next().await {} let schema = plan.schema(); @@ -58,7 +58,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), TEST_CREATE_QUERY).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } { @@ -67,7 +67,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), TEST_CREATE_QUERY_SELECT).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = interpreter.execute(None, None).await?; + let mut stream = interpreter.execute(None).await?; while let Some(_block) = stream.next().await {} let schema = plan.schema(); @@ -97,7 +97,7 @@ async fn test_create_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - assert!(interpreter.execute(None, None).await.is_err()); + assert!(interpreter.execute(None).await.is_err()); } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_table_describe.rs b/query/tests/it/interpreters/interpreter_table_describe.rs index f39f1eae7cdb..e4be0d46a04d 100644 --- a/query/tests/it/interpreters/interpreter_table_describe.rs +++ b/query/tests/it/interpreters/interpreter_table_describe.rs @@ -33,7 +33,7 @@ async fn interpreter_describe_table_test() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = interpreter.execute(None, None).await?; + let _ = interpreter.execute(None).await?; } // describe table. @@ -42,7 +42,7 @@ async fn interpreter_describe_table_test() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DescribeTableInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-------+----------+------+---------+-------+", diff --git a/query/tests/it/interpreters/interpreter_table_drop.rs b/query/tests/it/interpreters/interpreter_table_drop.rs index d9f4ea2cde17..e44ebeecacf2 100644 --- a/query/tests/it/interpreters/interpreter_table_drop.rs +++ b/query/tests/it/interpreters/interpreter_table_drop.rs @@ -33,7 +33,7 @@ async fn test_drop_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Drop table. @@ -41,7 +41,7 @@ async fn test_drop_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "DROP TABLE a").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropTableInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_table_rename.rs b/query/tests/it/interpreters/interpreter_table_rename.rs index c23ca19ff0fa..60d93c272ec6 100644 --- a/query/tests/it/interpreters/interpreter_table_rename.rs +++ b/query/tests/it/interpreters/interpreter_table_rename.rs @@ -33,7 +33,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // Rename table. @@ -41,7 +41,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "RENAME TABLE a TO b").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "RenameTableInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -52,7 +52,7 @@ async fn test_rename_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "DROP TABLE b").await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropTableInterpreter"); - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_table_show_create.rs b/query/tests/it/interpreters/interpreter_table_show_create.rs index 602fcedf4ab1..fd553116986f 100644 --- a/query/tests/it/interpreters/interpreter_table_show_create.rs +++ b/query/tests/it/interpreters/interpreter_table_show_create.rs @@ -76,13 +76,13 @@ async fn interpreter_show_create_table_test() -> Result<()> { for stmt in case.create_stmt { let plan = PlanParser::parse(ctx.clone(), stmt).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } let plan = PlanParser::parse(ctx.clone(), case.show_stmt).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "ShowCreateTableInterpreter"); let result = executor - .execute(None, None) + .execute(None) .await? .try_collect::>() .await?; diff --git a/query/tests/it/interpreters/interpreter_table_truncate.rs b/query/tests/it/interpreters/interpreter_table_truncate.rs index 122c22704440..a8b94056aa43 100644 --- a/query/tests/it/interpreters/interpreter_table_truncate.rs +++ b/query/tests/it/interpreters/interpreter_table_truncate.rs @@ -33,7 +33,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = interpreter.execute(None, None).await?; + let _ = interpreter.execute(None).await?; } // Insert into. @@ -41,7 +41,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "INSERT INTO default.a VALUES('1,1', '2,2')"; let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let _ = executor.execute(None, None).await?; + let _ = executor.execute(None).await?; } // select. @@ -49,7 +49,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "SELECT * FROM default.a"; let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec![ "+-----+-----+", @@ -68,7 +68,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(interpreter.name(), "TruncateTableInterpreter"); - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); @@ -79,7 +79,7 @@ async fn test_truncate_table_interpreter() -> Result<()> { let query = "SELECT * FROM default.a"; let plan = PlanParser::parse(ctx.clone(), query).await?; let interpreter = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let stream = interpreter.execute(None, None).await?; + let stream = interpreter.execute(None).await?; let result = stream.try_collect::>().await?; let expected = vec!["++", "++"]; common_datablocks::assert_blocks_sorted_eq(expected, result.as_slice()); diff --git a/query/tests/it/interpreters/interpreter_use_database.rs b/query/tests/it/interpreters/interpreter_use_database.rs index 8e9530258697..76f97e0a1945 100644 --- a/query/tests/it/interpreters/interpreter_use_database.rs +++ b/query/tests/it/interpreters/interpreter_use_database.rs @@ -27,7 +27,7 @@ async fn test_use_interpreter() -> Result<()> { let interpreter = InterpreterFactory::get(ctx, plan)?; assert_eq!(interpreter.name(), "UseDatabaseInterpreter"); - let mut stream = interpreter.execute(None, None).await?; + let mut stream = interpreter.execute(None).await?; while let Some(_block) = stream.next().await {} Ok(()) @@ -40,7 +40,7 @@ async fn test_use_database_interpreter_error() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), "USE xx").await?; let interpreter = InterpreterFactory::get(ctx, plan)?; - if let Err(e) = interpreter.execute(None, None).await { + if let Err(e) = interpreter.execute(None).await { let expect = "Code: 1003, displayText = Cannot USE 'xx', because the 'xx' doesn't exist."; assert_eq!(expect, format!("{}", e)); } diff --git a/query/tests/it/interpreters/interpreter_user_alter.rs b/query/tests/it/interpreters/interpreter_user_alter.rs index 563b8942baa4..160f462a3fb4 100644 --- a/query/tests/it/interpreters/interpreter_user_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_alter.rs @@ -57,7 +57,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let new_user = user_mgr.get_user(tenant, user_info.identity()).await?; assert_eq!( @@ -75,7 +75,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - executor.execute(None, None).await?; + executor.execute(None).await?; let user_info = user_mgr.get_user(tenant, user_info.identity()).await?; assert!(user_info.has_option_flag(UserOptionFlag::TenantSetting)); assert_eq!( @@ -89,7 +89,7 @@ async fn test_alter_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), &test_query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserInterpreter"); - executor.execute(None, None).await?; + executor.execute(None).await?; let user_info = user_mgr.get_user(tenant, user_info.identity()).await?; assert!(!user_info.has_option_flag(UserOptionFlag::TenantSetting)); } diff --git a/query/tests/it/interpreters/interpreter_user_create.rs b/query/tests/it/interpreters/interpreter_user_create.rs index e27347656a92..af3530c85db3 100644 --- a/query/tests/it/interpreters/interpreter_user_create.rs +++ b/query/tests/it/interpreters/interpreter_user_create.rs @@ -29,7 +29,7 @@ async fn test_create_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "CreateUserInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} Ok(()) diff --git a/query/tests/it/interpreters/interpreter_user_drop.rs b/query/tests/it/interpreters/interpreter_user_drop.rs index a6f829046f17..8eaf840ff15e 100644 --- a/query/tests/it/interpreters/interpreter_user_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_drop.rs @@ -33,7 +33,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - let ret = executor.execute(None, None).await; + let ret = executor.execute(None).await; assert!(ret.is_err()) } @@ -42,7 +42,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - let ret = executor.execute(None, None).await; + let ret = executor.execute(None).await; assert!(ret.is_ok()) } @@ -69,7 +69,7 @@ async fn test_drop_user_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx, plan.clone())?; assert_eq!(executor.name(), "DropUserInterpreter"); - executor.execute(None, None).await?; + executor.execute(None).await?; } Ok(()) diff --git a/query/tests/it/interpreters/interpreter_user_stage.rs b/query/tests/it/interpreters/interpreter_user_stage.rs index 52d16bfd939c..b30dfc989b2f 100644 --- a/query/tests/it/interpreters/interpreter_user_stage.rs +++ b/query/tests/it/interpreters/interpreter_user_stage.rs @@ -32,7 +32,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserStageInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} } @@ -43,7 +43,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DescribeUserStageInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; let mut blocks = vec![]; while let Some(block) = stream.next().await { @@ -74,7 +74,7 @@ async fn test_user_stage_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserStageInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} } diff --git a/query/tests/it/interpreters/interpreter_user_udf_alter.rs b/query/tests/it/interpreters/interpreter_user_udf_alter.rs index 08d84110a0d6..df90c0aee11e 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_alter.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_alter.rs @@ -32,7 +32,7 @@ async fn test_alter_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -52,7 +52,7 @@ async fn test_alter_udf_interpreter() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "AlterUserUDFInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let udf = ctx diff --git a/query/tests/it/interpreters/interpreter_user_udf_create.rs b/query/tests/it/interpreters/interpreter_user_udf_create.rs index 4cc6510ff3a0..f2ea46485ff3 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_create.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_create.rs @@ -34,7 +34,7 @@ async fn test_create_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -51,7 +51,7 @@ async fn test_create_udf_interpreter() -> Result<()> { // IF NOT EXISTS. let plan = PlanParser::parse(ctx.clone(), query).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - executor.execute(None, None).await?; + executor.execute(None).await?; let udf = ctx .get_user_manager() @@ -69,7 +69,7 @@ async fn test_create_udf_interpreter() -> Result<()> { "CREATE FUNCTION isnotempty AS (p) -> not(is_null(p)) DESC = 'This is a description'"; let plan = PlanParser::parse(ctx.clone(), query1).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let r = executor.execute(None, None).await; + let r = executor.execute(None).await; assert!(r.is_err()); let e = r.err(); assert_eq!(e.unwrap().code(), ErrorCode::udf_already_exists_code()); diff --git a/query/tests/it/interpreters/interpreter_user_udf_drop.rs b/query/tests/it/interpreters/interpreter_user_udf_drop.rs index 4c8587a0f771..0c389a78d301 100644 --- a/query/tests/it/interpreters/interpreter_user_udf_drop.rs +++ b/query/tests/it/interpreters/interpreter_user_udf_drop.rs @@ -35,7 +35,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { { let plan = PlanParser::parse(ctx.clone(), CREATE_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -52,7 +52,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_ok()); } @@ -60,7 +60,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF_IF_EXISTS).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_ok()); } @@ -68,7 +68,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_err()); } @@ -76,7 +76,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), CREATE_UDF).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "CreateUserUDFInterpreter"); - let mut stream = executor.execute(None, None).await?; + let mut stream = executor.execute(None).await?; while let Some(_block) = stream.next().await {} let udf = ctx .get_user_manager() @@ -92,7 +92,7 @@ async fn test_drop_udf_interpreter() -> Result<()> { let plan = PlanParser::parse(ctx.clone(), DROP_UDF_IF_EXISTS).await?; let executor = InterpreterFactory::get(ctx.clone(), plan.clone())?; assert_eq!(executor.name(), "DropUserUDFInterpreter"); - let res = executor.execute(None, None).await; + let res = executor.execute(None).await; assert!(res.is_ok()); } diff --git a/query/tests/it/storages/fuse/operations/read_plan.rs b/query/tests/it/storages/fuse/operations/read_plan.rs index a94a8aa6866e..7f8e08ac940d 100644 --- a/query/tests/it/storages/fuse/operations/read_plan.rs +++ b/query/tests/it/storages/fuse/operations/read_plan.rs @@ -118,7 +118,7 @@ async fn test_fuse_table_exact_statistic() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; let mut table = fixture.latest_default_table().await?; diff --git a/query/tests/it/storages/fuse/pruning.rs b/query/tests/it/storages/fuse/pruning.rs index f6537c50d3b5..bbaff98061a2 100644 --- a/query/tests/it/storages/fuse/pruning.rs +++ b/query/tests/it/storages/fuse/pruning.rs @@ -90,7 +90,7 @@ async fn test_block_pruner() -> Result<()> { }; let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; // get table let catalog = ctx.get_catalog(); @@ -231,7 +231,7 @@ async fn test_block_pruner_monotonic() -> Result<()> { let catalog = ctx.get_catalog(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; // get table let table = catalog diff --git a/query/tests/it/storages/fuse/table.rs b/query/tests/it/storages/fuse/table.rs index 7238ec4b2f3f..d1db318745b3 100644 --- a/query/tests/it/storages/fuse/table.rs +++ b/query/tests/it/storages/fuse/table.rs @@ -38,7 +38,7 @@ async fn test_fuse_table_normal_case() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; let mut table = fixture.latest_default_table().await?; @@ -163,7 +163,7 @@ async fn test_fuse_table_truncate() -> Result<()> { let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; let table = fixture.latest_default_table().await?; let truncate_plan = TruncateTablePlan { @@ -234,7 +234,7 @@ async fn test_fuse_table_optimize() -> Result<()> { let tbl_name = create_table_plan.table.clone(); let db_name = create_table_plan.db.clone(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; // insert 5 times let n = 5; @@ -266,7 +266,7 @@ async fn test_fuse_table_optimize() -> Result<()> { // To avoid flaky test, the value of setting `max_threads` is set to be 1, so that pipeline_builder will // only arrange one worker for the `ReadDataSourcePlan`. ctx.get_settings().set_max_threads(1)?; - let data_stream = interpreter.execute(None, None).await?; + let data_stream = interpreter.execute(None).await?; let _ = data_stream.try_collect::>(); // verify compaction diff --git a/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs b/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs index 3a3df3127eda..3c5bce49fedb 100644 --- a/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs +++ b/query/tests/it/storages/fuse/table_functions/fuse_history_table.rs @@ -80,7 +80,7 @@ async fn test_fuse_history_table_read() -> Result<()> { // test db & table let create_table_plan = fixture.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; // func args let arg_db = Expression::create_literal(DataValue::String(db.as_bytes().to_vec())); diff --git a/query/tests/it/storages/fuse/table_test_fixture.rs b/query/tests/it/storages/fuse/table_test_fixture.rs index c9ea2e04a2e8..0533b2c5df16 100644 --- a/query/tests/it/storages/fuse/table_test_fixture.rs +++ b/query/tests/it/storages/fuse/table_test_fixture.rs @@ -131,7 +131,7 @@ impl TestFixture { pub async fn create_default_table(&self) -> Result<()> { let create_table_plan = self.default_crate_table_plan(); let interpreter = CreateTableInterpreter::try_create(self.ctx.clone(), create_table_plan)?; - interpreter.execute(None, None).await?; + interpreter.execute(None).await?; Ok(()) } @@ -269,7 +269,7 @@ pub async fn expects_ok( pub async fn execute_query(ctx: Arc, query: &str) -> Result { let plan = PlanParser::parse(ctx.clone(), query).await?; InterpreterFactory::get(ctx.clone(), plan)? - .execute(None, None) + .execute(None) .await } diff --git a/query/tests/it/table_functions/numbers_table.rs b/query/tests/it/table_functions/numbers_table.rs index 7a4355ed02d9..187671b0f20b 100644 --- a/query/tests/it/table_functions/numbers_table.rs +++ b/query/tests/it/table_functions/numbers_table.rs @@ -114,7 +114,7 @@ async fn test_limit_push_down() -> Result<()> { let executor = InterpreterFactory::get(ctx.clone(), plan)?; - let stream = executor.execute(None, None).await?; + let stream = executor.execute(None).await?; let result = stream.try_collect::>().await?; let expect = test.result; let actual = result.as_slice(); From 3ce904baf224253a1468cef75690bbf8f7e29181 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 17:31:02 +0800 Subject: [PATCH 06/27] cargo fix + lint --- query/src/interpreters/interpreter.rs | 3 --- query/src/interpreters/interpreter_call.rs | 1 - query/src/interpreters/interpreter_copy.rs | 1 - query/src/interpreters/interpreter_database_create.rs | 1 - query/src/interpreters/interpreter_database_drop.rs | 1 - query/src/interpreters/interpreter_database_show_create.rs | 1 - query/src/interpreters/interpreter_explain.rs | 1 - query/src/interpreters/interpreter_factory_interceptor.rs | 1 - query/src/interpreters/interpreter_insert.rs | 4 +--- query/src/interpreters/interpreter_kill.rs | 1 - query/src/interpreters/interpreter_list.rs | 1 - query/src/interpreters/interpreter_privilege_grant.rs | 1 - query/src/interpreters/interpreter_privilege_revoke.rs | 1 - query/src/interpreters/interpreter_role_create.rs | 1 - query/src/interpreters/interpreter_role_drop.rs | 1 - query/src/interpreters/interpreter_role_grant.rs | 1 - query/src/interpreters/interpreter_role_revoke.rs | 1 - query/src/interpreters/interpreter_select.rs | 1 - query/src/interpreters/interpreter_setting.rs | 1 - query/src/interpreters/interpreter_show_databases.rs | 1 - query/src/interpreters/interpreter_show_engines.rs | 1 - query/src/interpreters/interpreter_show_functions.rs | 1 - query/src/interpreters/interpreter_show_grants.rs | 1 - query/src/interpreters/interpreter_show_metrics.rs | 1 - query/src/interpreters/interpreter_show_processlist.rs | 1 - query/src/interpreters/interpreter_show_roles.rs | 1 - query/src/interpreters/interpreter_show_settings.rs | 1 - query/src/interpreters/interpreter_show_tab_stat.rs | 1 - query/src/interpreters/interpreter_show_tables.rs | 1 - query/src/interpreters/interpreter_show_users.rs | 1 - query/src/interpreters/interpreter_table_create.rs | 1 - query/src/interpreters/interpreter_table_describe.rs | 1 - query/src/interpreters/interpreter_table_drop.rs | 1 - query/src/interpreters/interpreter_table_optimize.rs | 1 - query/src/interpreters/interpreter_table_rename.rs | 1 - query/src/interpreters/interpreter_table_show_create.rs | 1 - query/src/interpreters/interpreter_table_truncate.rs | 1 - query/src/interpreters/interpreter_use_database.rs | 1 - query/src/interpreters/interpreter_user_alter.rs | 1 - query/src/interpreters/interpreter_user_create.rs | 1 - query/src/interpreters/interpreter_user_drop.rs | 1 - query/src/interpreters/interpreter_user_stage_create.rs | 1 - query/src/interpreters/interpreter_user_stage_describe.rs | 1 - query/src/interpreters/interpreter_user_stage_drop.rs | 1 - query/src/interpreters/interpreter_user_udf_alter.rs | 1 - query/src/interpreters/interpreter_user_udf_create.rs | 1 - query/src/interpreters/interpreter_user_udf_drop.rs | 1 - query/src/interpreters/interpreter_view_alter.rs | 1 - query/src/interpreters/interpreter_view_create.rs | 1 - query/src/interpreters/interpreter_view_drop.rs | 1 - query/src/servers/clickhouse/interactive_worker_base.rs | 3 --- 51 files changed, 1 insertion(+), 57 deletions(-) diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index 844e2470b59e..b3d1933d468a 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -13,17 +13,14 @@ // limitations under the License. use std::any::Any; -use std::sync::Arc; use common_datavalues::DataSchemaRef; use common_datavalues::DataSchemaRefExt; use common_exception::ErrorCode; use common_exception::Result; -use common_infallible::Mutex; use common_streams::SendableDataBlockStream; use crate::pipelines::new::NewPipeline; -use crate::pipelines::new::SourcePipeBuilder; #[async_trait::async_trait] /// Interpreter is a trait for different PlanNode diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index 21e59d18b42f..ece2cf293659 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use super::Interpreter; use super::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::procedures::ProcedureFactory; use crate::sessions::QueryContext; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 7f385a029b80..3825cac6bd12 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -34,7 +34,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::pipelines::new::executor::PipelinePullingExecutor; use crate::pipelines::new::NewPipeline; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::StageSource; diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index c78b716da2b2..0517de505bbb 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -26,7 +26,6 @@ use common_tracing::tracing; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index 55ef4e324c21..131b27468340 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -25,7 +25,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct DropDatabaseInterpreter { diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index bbf0045fd0db..62c7f8d54fab 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -25,7 +25,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct ShowCreateDatabaseInterpreter { diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 9fed342b72d0..239dd1419859 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -27,7 +27,6 @@ use crate::interpreters::plan_schedulers; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::processors::PipelineBuilder; use crate::sessions::QueryContext; diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 63ff898f2c66..41e667436868 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -24,7 +24,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::InterpreterQueryLog; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Clone)] diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index afae4e78eb0d..9784f06993c9 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -29,7 +29,6 @@ use common_planners::PlanNode; use common_planners::SelectPlan; use common_streams::DataBlockStream; use common_streams::SendableDataBlockStream; -use common_tracing::tracing; use futures::TryStreamExt; use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; @@ -40,7 +39,6 @@ use crate::interpreters::InterpreterPtr; use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelineCompleteExecutor; use crate::pipelines::new::processors::port::OutputPort; -use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::BlocksSource; use crate::pipelines::new::processors::TransformAddOn; use crate::pipelines::new::processors::TransformCastSchema; @@ -219,7 +217,7 @@ impl Interpreter for InsertInterpreter { ) -> Result { let settings = self.ctx.get_settings(); - /// Use insert in new processor + // Use insert in new processor if settings.get_enable_new_processor_framework()? != 0 && self.ctx.get_cluster().is_empty() { return self.execute_new(input_stream).await; diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index bfe38dd9c216..bda6c9f60f45 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -26,7 +26,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct KillInterpreter { diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index ad42d0e4adb5..b855ac575aea 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -30,7 +30,6 @@ use regex::Regex; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::StageSource; diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index 6c222e0f7c58..3090005853d6 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -27,7 +27,6 @@ use common_tracing::tracing; use crate::interpreters::interpreter_common::validate_grant_object_exists; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index 93289cb2a332..5a1b407bdcd8 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -25,7 +25,6 @@ use common_tracing::tracing; use crate::interpreters::interpreter_common::validate_grant_object_exists; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index fa95e361901c..8552d3f3336c 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -24,7 +24,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index 31de47e41d21..68a749a5490a 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index 61deb483f5c2..01af8d3c9e4b 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -24,7 +24,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index e1aa75c809d6..4c0eff3b1ece 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -24,7 +24,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index f9385d17670b..9ea54363c89c 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -30,7 +30,6 @@ use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelinePullingExecutor; use crate::pipelines::new::NewPipeline; use crate::pipelines::new::QueryPipelineBuilder; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; /// SelectInterpreter struct which interprets SelectPlan diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index 6b519c77fe3e..d67e25900245 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -25,7 +25,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct SettingInterpreter { diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 85fbce98d238..7447c8c82291 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -26,7 +26,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 137db02c849a..9e4f4fe8384b 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index d3504447fff5..0bdb738242ec 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -26,7 +26,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index af2b2ce3c1af..9487e0bdef52 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -25,7 +25,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct ShowGrantsInterpreter { diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index 38528fbc4608..9444bf63156f 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index f59f9c7bbe41..d2540b7c452d 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index a243b17f204f..00bfef33bd1f 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index 38e4521bd7ab..aa3dd13322f6 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index 32f8cfcaa675..6a28e282e4a7 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -27,7 +27,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index e58e57c52349..b1648115dea2 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -27,7 +27,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index ee92f53bb29d..7e4a9b745536 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -25,7 +25,6 @@ use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index 4e3637457e5a..6edd27861f84 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -32,7 +32,6 @@ use super::InsertInterpreter; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct CreateTableInterpreter { diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index 686cea0c3fa1..5efcd38f1c62 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -25,7 +25,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct DescribeTableInterpreter { diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index f1f6109e7b5e..6d7480411f0b 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -26,7 +26,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index 65750d695a3b..76fc06f53982 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -26,7 +26,6 @@ use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterFactory; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::PlanParser; diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index 0111ba714bf2..ff20374eb975 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -24,7 +24,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct RenameTableInterpreter { diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index 7a1af0bb2497..e48e92a01205 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -27,7 +27,6 @@ use common_tracing::tracing; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::sql::is_internal_opt_key; diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index c20c3dbb8c07..4a2de2056a29 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -24,7 +24,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct TruncateTableInterpreter { diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index bf169b99b457..be3bfa0134b7 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -23,7 +23,6 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; pub struct UseDatabaseInterpreter { diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index 2e63ed2c612e..5fced0c1b449 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index e344090c827b..90a2be1de719 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -26,7 +26,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index 4ea5451492cb..d0cc63814e17 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index ef8c415c79d5..2911a3842873 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -24,7 +24,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index 5d4743b3ab56..07112036ee93 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -26,7 +26,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index 4a75388a202e..416ff1842139 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index e834189a0834..00a4d0d21fc3 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index 9f1695211488..29a51bbde341 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index b48f0e005b6a..2b488e310ba9 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -23,7 +23,6 @@ use common_tracing::tracing; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; #[derive(Debug)] diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index b8c30107c686..f8261ba45de2 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -30,7 +30,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 97db6f571423..3cecca13005d 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -29,7 +29,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index cd24aab06b95..c51ea813fe97 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -27,7 +27,6 @@ use common_streams::SendableDataBlockStream; use crate::catalogs::Catalog; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; -use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; use crate::storages::view::view_table::VIEW_ENGINE; diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 2cac22b9466f..17ee350112ff 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::BorrowMut; -use std::ops::Deref; -use std::ops::DerefMut; use std::sync::atomic::AtomicBool; use std::sync::atomic::Ordering; use std::sync::Arc; From 357983bebae703e907e5737db4dd6b2894d66cc6 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 17:50:48 +0800 Subject: [PATCH 07/27] fix stream --- .../clickhouse/interactive_worker_base.rs | 98 +++++++++++-------- 1 file changed, 59 insertions(+), 39 deletions(-) diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 17ee350112ff..6b44ae8bf115 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -102,48 +102,65 @@ impl InteractiveWorkerBase { input: stream, schema: sc.clone(), }; - let output_port = OutputPort::create(); - let sync_receiver_ck_source = SyncReceiverCkSource::create( - ctx.clone(), - ck_stream.input.into_inner(), - output_port.clone(), - sc, - )?; - let mut source_pipe_builder = SourcePipeBuilder::create(); - source_pipe_builder.add_source(output_port, sync_receiver_ck_source); let interpreter = InterpreterFactory::get(ctx.clone(), PlanNode::Insert(insert))?; - // Get the specific `InterceptorInterpreter`, then the inner `InsertInterpreter` - let interceptor_interpreter = match interpreter - .as_any() - .downcast_ref::() - { - Some(interceptor) => interceptor, - None => panic!("Interpreter isn't a InterceptorInterpreter!"), - }; - // In the context of this function, inner must be `InsertInterpreter` - let insert_interpreter = interceptor_interpreter.get_inner(); - // Get the specific `InsertInterpreter` - let insert_interpreter = match insert_interpreter - .as_any() - .downcast_ref::() + let name = interpreter.name().to_string(); + + if ctx.get_settings().get_enable_new_processor_framework()? != 0 + && ctx.get_cluster().is_empty() { - Some(insert) => insert, - None => panic!("Interpreter isn't a InsertInterpreter!"), - }; - let mut insert_interpreter_box = insert_interpreter.clone().get_box(); - // Set `SourcePipeBuilder` to `InsertInterpreter`, used in insert source is `StreamingWithFormat` - insert_interpreter_box - .as_mut() - .set_source_pipe_builder(Some(source_pipe_builder)); - - // Set the newest `InsertInterpreter` to `InterceptorInterpreter` - let mut interceptor_interpreter_box = interceptor_interpreter.clone().get_box(); - interceptor_interpreter_box - .as_mut() - .set_insert_inner(insert_interpreter_box as Box); + let output_port = OutputPort::create(); + let sync_receiver_ck_source = SyncReceiverCkSource::create( + ctx.clone(), + ck_stream.input.into_inner(), + output_port.clone(), + sc, + )?; + let mut source_pipe_builder = SourcePipeBuilder::create(); + source_pipe_builder.add_source(output_port, sync_receiver_ck_source); - let name = interpreter.name().to_string(); + // Get the specific `InterceptorInterpreter`, then the inner `InsertInterpreter` + let interceptor_interpreter = match interpreter + .as_any() + .downcast_ref::() + { + Some(interceptor) => interceptor, + None => panic!("Interpreter isn't a InterceptorInterpreter!"), + }; + // In the context of this function, inner must be `InsertInterpreter` + let insert_interpreter = interceptor_interpreter.get_inner(); + // Get the specific `InsertInterpreter` + let insert_interpreter = match insert_interpreter + .as_any() + .downcast_ref::() + { + Some(insert) => insert, + None => panic!("Interpreter isn't a InsertInterpreter!"), + }; + let mut insert_interpreter_box = insert_interpreter.clone().get_box(); + // Set `SourcePipeBuilder` to `InsertInterpreter`, used in insert source is `StreamingWithFormat` + insert_interpreter_box + .as_mut() + .set_source_pipe_builder(Some(source_pipe_builder)); + + // Set the newest `InsertInterpreter` to `InterceptorInterpreter` + let mut interceptor_interpreter_box = interceptor_interpreter.clone().get_box(); + interceptor_interpreter_box + .as_mut() + .set_insert_inner(insert_interpreter_box as Box); + + let (mut tx, rx) = mpsc::channel(20); + tx.send(BlockItem::InsertSample(sample_block)).await.ok(); + + // the data is comming in async mode + let sent_all_data = ch_ctx.state.sent_all_data.clone(); + ctx.try_spawn(async move { + interceptor_interpreter_box.execute(None).await.unwrap(); + sent_all_data.notify_one(); + })?; + + return Ok(rx); + } let (mut tx, rx) = mpsc::channel(20); tx.send(BlockItem::InsertSample(sample_block)).await.ok(); @@ -152,7 +169,10 @@ impl InteractiveWorkerBase { let sent_all_data = ch_ctx.state.sent_all_data.clone(); let start = Instant::now(); ctx.try_spawn(async move { - interceptor_interpreter_box.execute(None).await.unwrap(); + interpreter + .execute(Some(Box::pin(ck_stream))) + .await + .unwrap(); sent_all_data.notify_one(); })?; histogram!( From cb8a7a0dcd3de1f7701894105e6c5f7b39c9ea22 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 22:06:43 +0800 Subject: [PATCH 08/27] fix comments & make code simple --- query/src/interpreters/interpreter.rs | 10 ++++- query/src/interpreters/interpreter_call.rs | 4 -- query/src/interpreters/interpreter_copy.rs | 4 -- .../interpreter_database_create.rs | 4 -- .../interpreters/interpreter_database_drop.rs | 4 -- .../interpreter_database_show_create.rs | 4 -- query/src/interpreters/interpreter_explain.rs | 4 -- .../interpreter_factory_interceptor.rs | 24 +++++------ query/src/interpreters/interpreter_insert.rs | 43 +++++++------------ query/src/interpreters/interpreter_kill.rs | 4 -- query/src/interpreters/interpreter_list.rs | 4 -- .../interpreter_privilege_grant.rs | 4 -- .../interpreter_privilege_revoke.rs | 4 -- .../interpreters/interpreter_role_create.rs | 4 -- .../src/interpreters/interpreter_role_drop.rs | 4 -- .../interpreters/interpreter_role_grant.rs | 4 -- .../interpreters/interpreter_role_revoke.rs | 4 -- query/src/interpreters/interpreter_select.rs | 4 -- query/src/interpreters/interpreter_setting.rs | 4 -- .../interpreter_show_databases.rs | 4 -- .../interpreters/interpreter_show_engines.rs | 4 -- .../interpreter_show_functions.rs | 4 -- .../interpreters/interpreter_show_grants.rs | 4 -- .../interpreters/interpreter_show_metrics.rs | 4 -- .../interpreter_show_processlist.rs | 4 -- .../interpreters/interpreter_show_roles.rs | 4 -- .../interpreters/interpreter_show_settings.rs | 4 -- .../interpreters/interpreter_show_tab_stat.rs | 4 -- .../interpreters/interpreter_show_tables.rs | 4 -- .../interpreters/interpreter_show_users.rs | 4 -- .../interpreters/interpreter_table_create.rs | 4 -- .../interpreter_table_describe.rs | 4 -- .../interpreters/interpreter_table_drop.rs | 4 -- .../interpreter_table_optimize.rs | 4 -- .../interpreters/interpreter_table_rename.rs | 4 -- .../interpreter_table_show_create.rs | 4 -- .../interpreter_table_truncate.rs | 4 -- .../interpreters/interpreter_use_database.rs | 4 -- .../interpreters/interpreter_user_alter.rs | 4 -- .../interpreters/interpreter_user_create.rs | 4 -- .../src/interpreters/interpreter_user_drop.rs | 4 -- .../interpreter_user_stage_create.rs | 4 -- .../interpreter_user_stage_describe.rs | 4 -- .../interpreter_user_stage_drop.rs | 4 -- .../interpreter_user_udf_alter.rs | 4 -- .../interpreter_user_udf_create.rs | 4 -- .../interpreters/interpreter_user_udf_drop.rs | 4 -- .../interpreters/interpreter_view_alter.rs | 4 -- .../interpreters/interpreter_view_create.rs | 4 -- .../src/interpreters/interpreter_view_drop.rs | 4 -- .../clickhouse/interactive_worker_base.rs | 32 +------------- 51 files changed, 36 insertions(+), 261 deletions(-) diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index b3d1933d468a..fd3cfc4ed482 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -21,6 +21,7 @@ use common_exception::Result; use common_streams::SendableDataBlockStream; use crate::pipelines::new::NewPipeline; +use crate::pipelines::new::SourcePipeBuilder; #[async_trait::async_trait] /// Interpreter is a trait for different PlanNode @@ -29,8 +30,6 @@ pub trait Interpreter: Sync + Send { /// Return the name of Interpreter, such as "CreateDatabaseInterpreter" fn name(&self) -> &str; - fn as_any(&self) -> &dyn Any; - /// Return the schema of Interpreter fn schema(&self) -> DataSchemaRef { DataSchemaRefExt::create(vec![]) @@ -69,6 +68,13 @@ pub trait Interpreter: Sync + Send { self.name() ))) } + + fn set_source_pipe_builder(&self, _builder: Option) -> Result<()> { + Err(ErrorCode::UnImplement(format!( + "UnImplement set_source_pipe_builder method for {:?}", + self.name() + ))) + } } pub type InterpreterPtr = std::sync::Arc; diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index ece2cf293659..dea618924da0 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -43,10 +43,6 @@ impl Interpreter for CallInterpreter { "CallInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", name = "call_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 3825cac6bd12..00a6e27e7dc2 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -138,10 +138,6 @@ impl Interpreter for CopyInterpreter { "CopyInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", name = "copy_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index 0517de505bbb..73f2660fc1f8 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -46,10 +46,6 @@ impl Interpreter for CreateDatabaseInterpreter { "CreateDatabaseInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index 131b27468340..9e20b567c075 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -44,10 +44,6 @@ impl Interpreter for DropDatabaseInterpreter { "DropDatabaseInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index 62c7f8d54fab..5ff7ea8ae95c 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -47,10 +47,6 @@ impl Interpreter for ShowCreateDatabaseInterpreter { "ShowCreateDatabaseInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 239dd1419859..94f0adaff10d 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -41,10 +41,6 @@ impl Interpreter for ExplainInterpreter { "ExplainInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 41e667436868..57e6318a6817 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use std::time::SystemTime; use common_exception::Result; +use common_infallible::Mutex; use common_planners::PlanNode; use common_streams::ProgressStream; use common_streams::SendableDataBlockStream; @@ -24,13 +25,14 @@ use common_streams::SendableDataBlockStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::InterpreterQueryLog; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::QueryContext; -#[derive(Clone)] pub struct InterceptorInterpreter { ctx: Arc, inner: InterpreterPtr, query_log: InterpreterQueryLog, + source_pipe_builder: Mutex>, } impl InterceptorInterpreter { @@ -39,18 +41,14 @@ impl InterceptorInterpreter { ctx: ctx.clone(), inner, query_log: InterpreterQueryLog::create(ctx, plan), + source_pipe_builder: Mutex::new(None), } } - pub fn get_inner(&self) -> &InterpreterPtr { - &self.inner - } - - pub fn set_insert_inner(&mut self, interpreter: Box) { - self.inner = InterpreterPtr::from(interpreter); - } - pub fn get_box(self) -> Box { - Box::new(self) + fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { + let mut guard = self.source_pipe_builder.lock(); + *guard = builder; + Ok(()) } } @@ -60,14 +58,12 @@ impl Interpreter for InterceptorInterpreter { self.inner.name() } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, ) -> Result { + self.inner + .set_source_pipe_builder(((*self.source_pipe_builder.lock()).clone())); let result_stream = self.inner.execute(input_stream).await?; let metric_stream = ProgressStream::try_create(result_stream, self.ctx.get_result_progress())?; diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 9784f06993c9..dca1b9d05727 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -15,6 +15,7 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; +use std::thread::sleep; use common_exception::ErrorCode; use common_exception::Result; @@ -36,6 +37,7 @@ use crate::interpreters::plan_schedulers; use crate::interpreters::plan_schedulers::InsertWithPlan; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::interpreters::SelectInterpreter; use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelineCompleteExecutor; use crate::pipelines::new::processors::port::OutputPort; @@ -43,16 +45,14 @@ use crate::pipelines::new::processors::BlocksSource; use crate::pipelines::new::processors::TransformAddOn; use crate::pipelines::new::processors::TransformCastSchema; use crate::pipelines::new::NewPipeline; -use crate::pipelines::new::QueryPipelineBuilder; use crate::pipelines::new::SourcePipeBuilder; use crate::pipelines::transforms::AddOnStream; use crate::sessions::QueryContext; -#[derive(Clone)] pub struct InsertInterpreter { ctx: Arc, plan: InsertPlan, - source_pipe_builder: Option, + source_pipe_builder: Mutex>, } impl InsertInterpreter { @@ -60,16 +60,14 @@ impl InsertInterpreter { Ok(Arc::new(InsertInterpreter { ctx, plan, - source_pipe_builder: None, + source_pipe_builder: Mutex::new(None), })) } - pub fn set_source_pipe_builder(&mut self, builder: Option) { - self.source_pipe_builder = builder; - } - - pub fn get_box(self) -> Box { - Box::new(self) + fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { + let mut guard = self.source_pipe_builder.lock(); + *guard = builder; + Ok(()) } async fn execute_new( @@ -101,23 +99,18 @@ impl InsertInterpreter { } InsertInputSource::StreamingWithFormat(_) => { pipeline.add_pipe( - self.clone() - .get_box() - .as_mut() - .source_pipe_builder - .take() + ((*self.source_pipe_builder.lock()).clone()) .ok_or_else(|| ErrorCode::EmptyData("empty source pipe builder"))? .finalize(), ); } InsertInputSource::SelectPlan(plan) => { - let builder = QueryPipelineBuilder::create(self.ctx.clone()); + let select_interpreter = + SelectInterpreter::try_create(self.ctx.clone(), SelectPlan { + input: Arc::new((**plan).clone()), + })?; + pipeline = select_interpreter.create_new_pipeline()?; need_cast_schema = self.check_schema_cast(plan)?; - let optimized_plan = self.rewrite_plan(plan)?; - let select_plan = SelectPlan { - input: Arc::new(optimized_plan), - }; - pipeline = builder.finalize(&select_plan)?; } }; @@ -162,7 +155,6 @@ impl InsertInterpreter { let async_runtime = self.ctx.get_storage_runtime(); - pipeline.set_max_threads(settings.get_max_threads()? as usize); let executor = PipelineCompleteExecutor::try_create(async_runtime, pipeline)?; executor.execute()?; drop(executor); @@ -207,10 +199,6 @@ impl Interpreter for InsertInterpreter { "InsertIntoInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, mut input_stream: Option, @@ -300,7 +288,8 @@ impl Interpreter for InsertInterpreter { } fn create_new_pipeline(&self) -> Result { - let new_pipeline = NewPipeline::create(); + let mut new_pipeline = NewPipeline::create(); + new_pipeline.set_max_threads(self.ctx.get_settings().get_max_threads()? as usize); Ok(new_pipeline) } } diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index bda6c9f60f45..baee52723fdc 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -45,10 +45,6 @@ impl Interpreter for KillInterpreter { "KillInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index b855ac575aea..246f904a89fb 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -75,10 +75,6 @@ impl Interpreter for ListInterpreter { "ListInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", name = "list_interpreter_execute", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index 3090005853d6..fe98c0419467 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -47,10 +47,6 @@ impl Interpreter for GrantPrivilegeInterpreter { "GrantPrivilegeInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index 5a1b407bdcd8..89a4ef8eddb3 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -45,10 +45,6 @@ impl Interpreter for RevokePrivilegeInterpreter { "RevokePrivilegeInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index 8552d3f3336c..eace9b4c194f 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -44,10 +44,6 @@ impl Interpreter for CreateRoleInterpreter { "CreateRoleInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index 68a749a5490a..688c6e5c55ac 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -43,10 +43,6 @@ impl Interpreter for DropRoleInterpreter { "DropRoleInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index 01af8d3c9e4b..cb49b5271b7d 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -44,10 +44,6 @@ impl Interpreter for GrantRoleInterpreter { "GrantRoleInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index 4c0eff3b1ece..8be7d61401bc 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -44,10 +44,6 @@ impl Interpreter for RevokeRoleInterpreter { "RevokeRoleInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index 9ea54363c89c..ccccf6bcaed8 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -60,10 +60,6 @@ impl Interpreter for SelectInterpreter { "SelectInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - /// Get the schema of SelectPlan fn schema(&self) -> DataSchemaRef { self.select.schema() diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index d67e25900245..23769a5ca8e4 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -44,10 +44,6 @@ impl Interpreter for SettingInterpreter { "SettingInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 7447c8c82291..942a69c2d0b0 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -62,10 +62,6 @@ impl Interpreter for ShowDatabasesInterpreter { "ShowDatabasesInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 9e4f4fe8384b..2cf1fceebaee 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -50,10 +50,6 @@ impl Interpreter for ShowEnginesInterpreter { "ShowEnginesInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index 0bdb738242ec..ddc4bb0d8b3c 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -63,10 +63,6 @@ impl Interpreter for ShowFunctionsInterpreter { "ShowFunctionsInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index 9487e0bdef52..91f8823e6b5c 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -44,10 +44,6 @@ impl Interpreter for ShowGrantsInterpreter { "ShowGrantsInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index 9444bf63156f..d472fe911c87 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -50,10 +50,6 @@ impl Interpreter for ShowMetricsInterpreter { "ShowMetricsInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index d2540b7c452d..eb22738ce0ac 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -53,10 +53,6 @@ impl Interpreter for ShowProcessListInterpreter { "ShowProcesslistInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index 00bfef33bd1f..755fd143d50f 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -50,10 +50,6 @@ impl Interpreter for ShowRolesInterpreter { "ShowRolesInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index aa3dd13322f6..ef8f11cd74ef 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -50,10 +50,6 @@ impl Interpreter for ShowSettingsInterpreter { "ShowSettingsInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index 6a28e282e4a7..93ff37e0dd01 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -89,10 +89,6 @@ impl Interpreter for ShowTabStatInterpreter { "ShowTabStatInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index b1648115dea2..420e6684c9c9 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -80,10 +80,6 @@ impl Interpreter for ShowTablesInterpreter { "ShowTablesInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index 7e4a9b745536..81944dd99521 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -53,10 +53,6 @@ impl Interpreter for ShowUsersInterpreter { "ShowUsersInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index 6edd27861f84..538ddac57df7 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -51,10 +51,6 @@ impl Interpreter for CreateTableInterpreter { "CreateTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index 5efcd38f1c62..22be1df882a3 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -44,10 +44,6 @@ impl Interpreter for DescribeTableInterpreter { "DescribeTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index 6d7480411f0b..167a30543705 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -46,10 +46,6 @@ impl Interpreter for DropTableInterpreter { "DropTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index 76fc06f53982..ff29ae80352e 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -46,10 +46,6 @@ impl Interpreter for OptimizeTableInterpreter { "OptimizeTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index ff20374eb975..f10be989774f 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -43,10 +43,6 @@ impl Interpreter for RenameTableInterpreter { "RenameTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index e48e92a01205..79d93089fd79 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -47,10 +47,6 @@ impl Interpreter for ShowCreateTableInterpreter { "ShowCreateTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index 4a2de2056a29..2d7c4f76b9fa 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -43,10 +43,6 @@ impl Interpreter for TruncateTableInterpreter { "TruncateTableInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index be3bfa0134b7..262b1c8980a7 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -42,10 +42,6 @@ impl Interpreter for UseDatabaseInterpreter { "UseDatabaseInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index 5fced0c1b449..a036b1237f91 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -43,10 +43,6 @@ impl Interpreter for AlterUserInterpreter { "AlterUserInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index 90a2be1de719..0e13ede27a2e 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -46,10 +46,6 @@ impl Interpreter for CreateUserInterpreter { "CreateUserInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index d0cc63814e17..ef4d8681de58 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -43,10 +43,6 @@ impl Interpreter for DropUserInterpreter { "DropUserInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index 2911a3842873..ea15b551ba0a 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -44,10 +44,6 @@ impl Interpreter for CreateUserStageInterpreter { "CreateUserStageInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index 07112036ee93..d85ab92a7eff 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -49,10 +49,6 @@ impl Interpreter for DescribeUserStageInterpreter { "DescribeUserStageInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index 416ff1842139..5fee33b8b4b5 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -43,10 +43,6 @@ impl Interpreter for DropUserStageInterpreter { "DropUserStageInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index 00a4d0d21fc3..57a9c58206a1 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -43,10 +43,6 @@ impl Interpreter for AlterUserUDFInterpreter { "AlterUserUDFInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "debug", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index 29a51bbde341..8c738fbb62a8 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -43,10 +43,6 @@ impl Interpreter for CreateUserUDFInterpreter { "CreateUserUDFInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index 2b488e310ba9..c8ad8c6172c9 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -43,10 +43,6 @@ impl Interpreter for DropUserUDFInterpreter { "DropUserUDFInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - #[tracing::instrument(level = "info", skip(self, _input_stream), fields(ctx.id = self.ctx.get_id().as_str()))] async fn execute( &self, diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index f8261ba45de2..cb03b16cbc10 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -50,10 +50,6 @@ impl Interpreter for AlterViewInterpreter { "AlterViewInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute(&self, _: Option) -> Result { // check privilige self.ctx diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 3cecca13005d..7210b89d8443 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -49,10 +49,6 @@ impl Interpreter for CreateViewInterpreter { "CreateViewInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute(&self, _: Option) -> Result { // check privilige self.ctx diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index c51ea813fe97..dc1e5000e825 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -47,10 +47,6 @@ impl Interpreter for DropViewInterpreter { "DropViewInterpreter" } - fn as_any(&self) -> &dyn Any { - self - } - async fn execute( &self, _input_stream: Option, diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 6b44ae8bf115..5bc55c22f0b9 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -119,35 +119,7 @@ impl InteractiveWorkerBase { let mut source_pipe_builder = SourcePipeBuilder::create(); source_pipe_builder.add_source(output_port, sync_receiver_ck_source); - // Get the specific `InterceptorInterpreter`, then the inner `InsertInterpreter` - let interceptor_interpreter = match interpreter - .as_any() - .downcast_ref::() - { - Some(interceptor) => interceptor, - None => panic!("Interpreter isn't a InterceptorInterpreter!"), - }; - // In the context of this function, inner must be `InsertInterpreter` - let insert_interpreter = interceptor_interpreter.get_inner(); - // Get the specific `InsertInterpreter` - let insert_interpreter = match insert_interpreter - .as_any() - .downcast_ref::() - { - Some(insert) => insert, - None => panic!("Interpreter isn't a InsertInterpreter!"), - }; - let mut insert_interpreter_box = insert_interpreter.clone().get_box(); - // Set `SourcePipeBuilder` to `InsertInterpreter`, used in insert source is `StreamingWithFormat` - insert_interpreter_box - .as_mut() - .set_source_pipe_builder(Some(source_pipe_builder)); - - // Set the newest `InsertInterpreter` to `InterceptorInterpreter` - let mut interceptor_interpreter_box = interceptor_interpreter.clone().get_box(); - interceptor_interpreter_box - .as_mut() - .set_insert_inner(insert_interpreter_box as Box); + interpreter.set_source_pipe_builder(Option::from(source_pipe_builder)); let (mut tx, rx) = mpsc::channel(20); tx.send(BlockItem::InsertSample(sample_block)).await.ok(); @@ -155,7 +127,7 @@ impl InteractiveWorkerBase { // the data is comming in async mode let sent_all_data = ch_ctx.state.sent_all_data.clone(); ctx.try_spawn(async move { - interceptor_interpreter_box.execute(None).await.unwrap(); + interpreter.execute(None).await.unwrap(); sent_all_data.notify_one(); })?; From 9b751b6fd90c40d4681cd4572a6d18f2e0e36801 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 22:15:28 +0800 Subject: [PATCH 09/27] fix --- query/src/interpreters/interpreter.rs | 2 +- query/src/interpreters/interpreter_call.rs | 2 +- query/src/interpreters/interpreter_copy.rs | 2 +- .../interpreters/interpreter_database_create.rs | 2 +- .../interpreters/interpreter_database_drop.rs | 2 +- .../interpreter_database_show_create.rs | 2 +- query/src/interpreters/interpreter_explain.rs | 2 +- .../interpreter_factory_interceptor.rs | 16 ++++++++-------- query/src/interpreters/interpreter_insert.rs | 16 ++++++++-------- query/src/interpreters/interpreter_kill.rs | 2 +- query/src/interpreters/interpreter_list.rs | 2 +- .../interpreters/interpreter_privilege_grant.rs | 2 +- .../interpreters/interpreter_privilege_revoke.rs | 2 +- .../src/interpreters/interpreter_role_create.rs | 2 +- query/src/interpreters/interpreter_role_drop.rs | 2 +- query/src/interpreters/interpreter_role_grant.rs | 2 +- .../src/interpreters/interpreter_role_revoke.rs | 2 +- query/src/interpreters/interpreter_select.rs | 2 +- query/src/interpreters/interpreter_setting.rs | 2 +- .../interpreters/interpreter_show_databases.rs | 2 +- .../src/interpreters/interpreter_show_engines.rs | 2 +- .../interpreters/interpreter_show_functions.rs | 2 +- .../src/interpreters/interpreter_show_grants.rs | 2 +- .../src/interpreters/interpreter_show_metrics.rs | 2 +- .../interpreters/interpreter_show_processlist.rs | 2 +- query/src/interpreters/interpreter_show_roles.rs | 2 +- .../interpreters/interpreter_show_settings.rs | 2 +- .../interpreters/interpreter_show_tab_stat.rs | 2 +- .../src/interpreters/interpreter_show_tables.rs | 2 +- query/src/interpreters/interpreter_show_users.rs | 2 +- .../src/interpreters/interpreter_table_create.rs | 2 +- .../interpreters/interpreter_table_describe.rs | 2 +- query/src/interpreters/interpreter_table_drop.rs | 2 +- .../interpreters/interpreter_table_optimize.rs | 2 +- .../src/interpreters/interpreter_table_rename.rs | 2 +- .../interpreter_table_show_create.rs | 2 +- .../interpreters/interpreter_table_truncate.rs | 2 +- .../src/interpreters/interpreter_use_database.rs | 2 +- query/src/interpreters/interpreter_user_alter.rs | 2 +- .../src/interpreters/interpreter_user_create.rs | 2 +- query/src/interpreters/interpreter_user_drop.rs | 2 +- .../interpreter_user_stage_create.rs | 2 +- .../interpreter_user_stage_describe.rs | 2 +- .../interpreters/interpreter_user_stage_drop.rs | 2 +- .../interpreters/interpreter_user_udf_alter.rs | 2 +- .../interpreters/interpreter_user_udf_create.rs | 2 +- .../interpreters/interpreter_user_udf_drop.rs | 2 +- query/src/interpreters/interpreter_view_alter.rs | 2 +- .../src/interpreters/interpreter_view_create.rs | 2 +- query/src/interpreters/interpreter_view_drop.rs | 2 +- .../clickhouse/interactive_worker_base.rs | 5 ++--- 51 files changed, 66 insertions(+), 67 deletions(-) diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index fd3cfc4ed482..1a03c306ac07 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use common_datavalues::DataSchemaRef; use common_datavalues::DataSchemaRefExt; diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index dea618924da0..ce5ad2738703 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 00a6e27e7dc2..44f1372ff30c 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::path::Path; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index 73f2660fc1f8..2c39ef5aaae0 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index 9e20b567c075..068ea8980c68 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index 5ff7ea8ae95c..4049d38cd28e 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 94f0adaff10d..4410cbf95e73 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 57e6318a6817..724a2f77a9c4 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use std::time::SystemTime; @@ -44,12 +44,6 @@ impl InterceptorInterpreter { source_pipe_builder: Mutex::new(None), } } - - fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { - let mut guard = self.source_pipe_builder.lock(); - *guard = builder; - Ok(()) - } } #[async_trait::async_trait] @@ -63,7 +57,7 @@ impl Interpreter for InterceptorInterpreter { input_stream: Option, ) -> Result { self.inner - .set_source_pipe_builder(((*self.source_pipe_builder.lock()).clone())); + .set_source_pipe_builder((*self.source_pipe_builder.lock()).clone()); let result_stream = self.inner.execute(input_stream).await?; let metric_stream = ProgressStream::try_create(result_stream, self.ctx.get_result_progress())?; @@ -96,4 +90,10 @@ impl Interpreter for InterceptorInterpreter { } self.query_log.log_finish(now).await } + + fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { + let mut guard = self.source_pipe_builder.lock(); + *guard = builder; + Ok(()) + } } diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index dca1b9d05727..0d95577ced55 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::collections::VecDeque; use std::sync::Arc; -use std::thread::sleep; + use common_exception::ErrorCode; use common_exception::Result; @@ -64,12 +64,6 @@ impl InsertInterpreter { })) } - fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { - let mut guard = self.source_pipe_builder.lock(); - *guard = builder; - Ok(()) - } - async fn execute_new( &self, _input_stream: Option, @@ -292,4 +286,10 @@ impl Interpreter for InsertInterpreter { new_pipeline.set_max_threads(self.ctx.get_settings().get_max_threads()? as usize); Ok(new_pipeline) } + + fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { + let mut guard = self.source_pipe_builder.lock(); + *guard = builder; + Ok(()) + } } diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index baee52723fdc..2b0590ca5dd1 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datavalues::DataSchema; diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index 246f904a89fb..5303faa93907 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index fe98c0419467..23958fc2db01 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index 89a4ef8eddb3..077777e97c32 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index eace9b4c194f..4c90dbae28cd 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index 688c6e5c55ac..e118f0f08b58 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index cb49b5271b7d..40181584b8a1 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index 8be7d61401bc..62de91859e50 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index ccccf6bcaed8..36ff3a4592cc 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datavalues::DataSchemaRef; diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index 23769a5ca8e4..2798db78d96f 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use chrono_tz::Tz; diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 942a69c2d0b0..2706faa80fe6 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 2cf1fceebaee..0ba519bb5a8b 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index ddc4bb0d8b3c..7a02f26348e1 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index 91f8823e6b5c..590a700af1fe 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index d472fe911c87..a04b67db048c 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index eb22738ce0ac..ad3ba8d9191d 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index 755fd143d50f..9d7217588682 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index ef8f11cd74ef..7d6678347c4d 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index 93ff37e0dd01..b520e7fb6a16 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index 420e6684c9c9..d5c6e95be4da 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index 81944dd99521..b2a0a27fc0b0 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index 538ddac57df7..ebe0a7ad6be6 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datavalues::DataField; diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index 22be1df882a3..371b2de7309a 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index 167a30543705..8ffad4de9b81 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index ff29ae80352e..7c4bb526ccc7 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index f10be989774f..ba0b34576344 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index 79d93089fd79..d2bbaabe4caf 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index 2d7c4f76b9fa..d1e6e15fcdbf 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index 262b1c8980a7..28bd527b1947 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datavalues::DataSchema; diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index a036b1237f91..bf7706b7fbd9 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index 0e13ede27a2e..3202fceb1c66 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index ef4d8681de58..bfb9b406af39 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index ea15b551ba0a..77c0ce1f16b1 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index d85ab92a7eff..6d7f2cbb14ae 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index 5fee33b8b4b5..420d966a062c 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index 57a9c58206a1..8d16c1a8a1c1 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index 8c738fbb62a8..6111aeb49848 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index c8ad8c6172c9..8cdf5d0ac65f 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -12,7 +12,7 @@ // 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; diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index cb03b16cbc10..cd901004bbf6 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::collections::BTreeMap; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 7210b89d8443..2c29af1c9f8e 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; + use std::collections::BTreeMap; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index dc1e5000e825..5b471a31ef14 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -12,7 +12,7 @@ // 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::ErrorCode; diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 5bc55c22f0b9..d268a9c3d330 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -41,9 +41,8 @@ use tokio_stream::wrappers::IntervalStream; use tokio_stream::wrappers::ReceiverStream; use super::writers::from_clickhouse_block; -use crate::interpreters::InsertInterpreter; -use crate::interpreters::InterceptorInterpreter; -use crate::interpreters::Interpreter; + + use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::SyncReceiverCkSource; From bcf024b5cc88f1177a517fbfdb623b8b433833ce Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 22:28:50 +0800 Subject: [PATCH 10/27] fix lint --- query/src/interpreters/interpreter.rs | 2 -- query/src/interpreters/interpreter_call.rs | 1 - query/src/interpreters/interpreter_copy.rs | 1 - query/src/interpreters/interpreter_database_create.rs | 1 - query/src/interpreters/interpreter_database_drop.rs | 1 - .../src/interpreters/interpreter_database_show_create.rs | 1 - query/src/interpreters/interpreter_explain.rs | 1 - .../src/interpreters/interpreter_factory_interceptor.rs | 4 ++-- query/src/interpreters/interpreter_insert.rs | 9 --------- query/src/interpreters/interpreter_kill.rs | 1 - query/src/interpreters/interpreter_list.rs | 1 - query/src/interpreters/interpreter_privilege_grant.rs | 1 - query/src/interpreters/interpreter_privilege_revoke.rs | 1 - query/src/interpreters/interpreter_role_create.rs | 1 - query/src/interpreters/interpreter_role_drop.rs | 1 - query/src/interpreters/interpreter_role_grant.rs | 1 - query/src/interpreters/interpreter_role_revoke.rs | 1 - query/src/interpreters/interpreter_select.rs | 1 - query/src/interpreters/interpreter_setting.rs | 1 - query/src/interpreters/interpreter_show_databases.rs | 1 - query/src/interpreters/interpreter_show_engines.rs | 1 - query/src/interpreters/interpreter_show_functions.rs | 1 - query/src/interpreters/interpreter_show_grants.rs | 1 - query/src/interpreters/interpreter_show_metrics.rs | 1 - query/src/interpreters/interpreter_show_processlist.rs | 1 - query/src/interpreters/interpreter_show_roles.rs | 1 - query/src/interpreters/interpreter_show_settings.rs | 1 - query/src/interpreters/interpreter_show_tab_stat.rs | 1 - query/src/interpreters/interpreter_show_tables.rs | 1 - query/src/interpreters/interpreter_show_users.rs | 1 - query/src/interpreters/interpreter_table_create.rs | 1 - query/src/interpreters/interpreter_table_describe.rs | 1 - query/src/interpreters/interpreter_table_drop.rs | 1 - query/src/interpreters/interpreter_table_optimize.rs | 1 - query/src/interpreters/interpreter_table_rename.rs | 1 - query/src/interpreters/interpreter_table_show_create.rs | 1 - query/src/interpreters/interpreter_table_truncate.rs | 1 - query/src/interpreters/interpreter_use_database.rs | 1 - query/src/interpreters/interpreter_user_alter.rs | 1 - query/src/interpreters/interpreter_user_create.rs | 1 - query/src/interpreters/interpreter_user_drop.rs | 1 - query/src/interpreters/interpreter_user_stage_create.rs | 1 - .../src/interpreters/interpreter_user_stage_describe.rs | 1 - query/src/interpreters/interpreter_user_stage_drop.rs | 1 - query/src/interpreters/interpreter_user_udf_alter.rs | 1 - query/src/interpreters/interpreter_user_udf_create.rs | 1 - query/src/interpreters/interpreter_user_udf_drop.rs | 1 - query/src/interpreters/interpreter_view_alter.rs | 1 - query/src/interpreters/interpreter_view_create.rs | 1 - query/src/interpreters/interpreter_view_drop.rs | 1 - query/src/servers/clickhouse/interactive_worker_base.rs | 6 +++--- 51 files changed, 5 insertions(+), 63 deletions(-) diff --git a/query/src/interpreters/interpreter.rs b/query/src/interpreters/interpreter.rs index 1a03c306ac07..70e33338527c 100644 --- a/query/src/interpreters/interpreter.rs +++ b/query/src/interpreters/interpreter.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - - use common_datavalues::DataSchemaRef; use common_datavalues::DataSchemaRefExt; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_call.rs b/query/src/interpreters/interpreter_call.rs index ce5ad2738703..bf6d8b3bb962 100644 --- a/query/src/interpreters/interpreter_call.rs +++ b/query/src/interpreters/interpreter_call.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 44f1372ff30c..9a191f61a8e4 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::path::Path; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_database_create.rs b/query/src/interpreters/interpreter_database_create.rs index 2c39ef5aaae0..e451b7036d89 100644 --- a/query/src/interpreters/interpreter_database_create.rs +++ b/query/src/interpreters/interpreter_database_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_database_drop.rs b/query/src/interpreters/interpreter_database_drop.rs index 068ea8980c68..721a290a76bc 100644 --- a/query/src/interpreters/interpreter_database_drop.rs +++ b/query/src/interpreters/interpreter_database_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_database_show_create.rs b/query/src/interpreters/interpreter_database_show_create.rs index 4049d38cd28e..640cbadfa931 100644 --- a/query/src/interpreters/interpreter_database_show_create.rs +++ b/query/src/interpreters/interpreter_database_show_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_explain.rs b/query/src/interpreters/interpreter_explain.rs index 4410cbf95e73..54e7a9451025 100644 --- a/query/src/interpreters/interpreter_explain.rs +++ b/query/src/interpreters/interpreter_explain.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_factory_interceptor.rs b/query/src/interpreters/interpreter_factory_interceptor.rs index 724a2f77a9c4..daaac9b43710 100644 --- a/query/src/interpreters/interpreter_factory_interceptor.rs +++ b/query/src/interpreters/interpreter_factory_interceptor.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use std::time::SystemTime; @@ -56,7 +55,8 @@ impl Interpreter for InterceptorInterpreter { &self, input_stream: Option, ) -> Result { - self.inner + let _ = self + .inner .set_source_pipe_builder((*self.source_pipe_builder.lock()).clone()); let result_stream = self.inner.execute(input_stream).await?; let metric_stream = diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 0d95577ced55..b59e768f6cae 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -12,11 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::collections::VecDeque; use std::sync::Arc; - use common_exception::ErrorCode; use common_exception::Result; use common_functions::scalars::CastFunction; @@ -33,12 +31,10 @@ use common_streams::SendableDataBlockStream; use futures::TryStreamExt; use crate::interpreters::interpreter_insert_with_stream::InsertWithStream; -use crate::interpreters::plan_schedulers; use crate::interpreters::plan_schedulers::InsertWithPlan; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; use crate::interpreters::SelectInterpreter; -use crate::optimizers::Optimizers; use crate::pipelines::new::executor::PipelineCompleteExecutor; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::BlocksSource; @@ -165,11 +161,6 @@ impl InsertInterpreter { ))) } - /// Call this method to optimize the logical plan before executing - fn rewrite_plan(&self, select_plan: &PlanNode) -> Result { - plan_schedulers::apply_plan_rewrite(Optimizers::create(self.ctx.clone()), select_plan) - } - fn check_schema_cast(&self, plan_node: &PlanNode) -> common_exception::Result { let output_schema = &self.plan.schema; let select_schema = plan_node.schema(); diff --git a/query/src/interpreters/interpreter_kill.rs b/query/src/interpreters/interpreter_kill.rs index 2b0590ca5dd1..d4de47ce7c01 100644 --- a/query/src/interpreters/interpreter_kill.rs +++ b/query/src/interpreters/interpreter_kill.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datavalues::DataSchema; diff --git a/query/src/interpreters/interpreter_list.rs b/query/src/interpreters/interpreter_list.rs index 5303faa93907..cfbac9613639 100644 --- a/query/src/interpreters/interpreter_list.rs +++ b/query/src/interpreters/interpreter_list.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_privilege_grant.rs b/query/src/interpreters/interpreter_privilege_grant.rs index 23958fc2db01..bb76097f36e5 100644 --- a/query/src/interpreters/interpreter_privilege_grant.rs +++ b/query/src/interpreters/interpreter_privilege_grant.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_privilege_revoke.rs b/query/src/interpreters/interpreter_privilege_revoke.rs index 077777e97c32..48538395e468 100644 --- a/query/src/interpreters/interpreter_privilege_revoke.rs +++ b/query/src/interpreters/interpreter_privilege_revoke.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_role_create.rs b/query/src/interpreters/interpreter_role_create.rs index 4c90dbae28cd..a1828cc81a22 100644 --- a/query/src/interpreters/interpreter_role_create.rs +++ b/query/src/interpreters/interpreter_role_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_role_drop.rs b/query/src/interpreters/interpreter_role_drop.rs index e118f0f08b58..f12c2e29b76c 100644 --- a/query/src/interpreters/interpreter_role_drop.rs +++ b/query/src/interpreters/interpreter_role_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_role_grant.rs b/query/src/interpreters/interpreter_role_grant.rs index 40181584b8a1..cc8c9dcf03c5 100644 --- a/query/src/interpreters/interpreter_role_grant.rs +++ b/query/src/interpreters/interpreter_role_grant.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_role_revoke.rs b/query/src/interpreters/interpreter_role_revoke.rs index 62de91859e50..7f784eb4e7f0 100644 --- a/query/src/interpreters/interpreter_role_revoke.rs +++ b/query/src/interpreters/interpreter_role_revoke.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_select.rs b/query/src/interpreters/interpreter_select.rs index 36ff3a4592cc..af5e3b9b4a14 100644 --- a/query/src/interpreters/interpreter_select.rs +++ b/query/src/interpreters/interpreter_select.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datavalues::DataSchemaRef; diff --git a/query/src/interpreters/interpreter_setting.rs b/query/src/interpreters/interpreter_setting.rs index 2798db78d96f..996cda81cfe0 100644 --- a/query/src/interpreters/interpreter_setting.rs +++ b/query/src/interpreters/interpreter_setting.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use chrono_tz::Tz; diff --git a/query/src/interpreters/interpreter_show_databases.rs b/query/src/interpreters/interpreter_show_databases.rs index 2706faa80fe6..42431cc52500 100644 --- a/query/src/interpreters/interpreter_show_databases.rs +++ b/query/src/interpreters/interpreter_show_databases.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_engines.rs b/query/src/interpreters/interpreter_show_engines.rs index 0ba519bb5a8b..26d0a82febd9 100644 --- a/query/src/interpreters/interpreter_show_engines.rs +++ b/query/src/interpreters/interpreter_show_engines.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_functions.rs b/query/src/interpreters/interpreter_show_functions.rs index 7a02f26348e1..d5f0c30ee827 100644 --- a/query/src/interpreters/interpreter_show_functions.rs +++ b/query/src/interpreters/interpreter_show_functions.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_grants.rs b/query/src/interpreters/interpreter_show_grants.rs index 590a700af1fe..7706e2ca397c 100644 --- a/query/src/interpreters/interpreter_show_grants.rs +++ b/query/src/interpreters/interpreter_show_grants.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_show_metrics.rs b/query/src/interpreters/interpreter_show_metrics.rs index a04b67db048c..e60251abb446 100644 --- a/query/src/interpreters/interpreter_show_metrics.rs +++ b/query/src/interpreters/interpreter_show_metrics.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_processlist.rs b/query/src/interpreters/interpreter_show_processlist.rs index ad3ba8d9191d..03f689f63fc2 100644 --- a/query/src/interpreters/interpreter_show_processlist.rs +++ b/query/src/interpreters/interpreter_show_processlist.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_roles.rs b/query/src/interpreters/interpreter_show_roles.rs index 9d7217588682..eb360225bec0 100644 --- a/query/src/interpreters/interpreter_show_roles.rs +++ b/query/src/interpreters/interpreter_show_roles.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_settings.rs b/query/src/interpreters/interpreter_show_settings.rs index 7d6678347c4d..343276035949 100644 --- a/query/src/interpreters/interpreter_show_settings.rs +++ b/query/src/interpreters/interpreter_show_settings.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_tab_stat.rs b/query/src/interpreters/interpreter_show_tab_stat.rs index b520e7fb6a16..d07b786ad612 100644 --- a/query/src/interpreters/interpreter_show_tab_stat.rs +++ b/query/src/interpreters/interpreter_show_tab_stat.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_tables.rs b/query/src/interpreters/interpreter_show_tables.rs index d5c6e95be4da..8dcf1c257f78 100644 --- a/query/src/interpreters/interpreter_show_tables.rs +++ b/query/src/interpreters/interpreter_show_tables.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_show_users.rs b/query/src/interpreters/interpreter_show_users.rs index b2a0a27fc0b0..95681955aa40 100644 --- a/query/src/interpreters/interpreter_show_users.rs +++ b/query/src/interpreters/interpreter_show_users.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_table_create.rs b/query/src/interpreters/interpreter_table_create.rs index ebe0a7ad6be6..48f8067efe43 100644 --- a/query/src/interpreters/interpreter_table_create.rs +++ b/query/src/interpreters/interpreter_table_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datavalues::DataField; diff --git a/query/src/interpreters/interpreter_table_describe.rs b/query/src/interpreters/interpreter_table_describe.rs index 371b2de7309a..6d1a4f6664f9 100644 --- a/query/src/interpreters/interpreter_table_describe.rs +++ b/query/src/interpreters/interpreter_table_describe.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_table_drop.rs b/query/src/interpreters/interpreter_table_drop.rs index 8ffad4de9b81..8aa9f99fec55 100644 --- a/query/src/interpreters/interpreter_table_drop.rs +++ b/query/src/interpreters/interpreter_table_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/interpreters/interpreter_table_optimize.rs b/query/src/interpreters/interpreter_table_optimize.rs index 7c4bb526ccc7..e958589445a0 100644 --- a/query/src/interpreters/interpreter_table_optimize.rs +++ b/query/src/interpreters/interpreter_table_optimize.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_table_rename.rs b/query/src/interpreters/interpreter_table_rename.rs index ba0b34576344..ec4bc6bbade9 100644 --- a/query/src/interpreters/interpreter_table_rename.rs +++ b/query/src/interpreters/interpreter_table_rename.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_table_show_create.rs b/query/src/interpreters/interpreter_table_show_create.rs index d2bbaabe4caf..299367a0c2b6 100644 --- a/query/src/interpreters/interpreter_table_show_create.rs +++ b/query/src/interpreters/interpreter_table_show_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_table_truncate.rs b/query/src/interpreters/interpreter_table_truncate.rs index d1e6e15fcdbf..24c4fa4ad548 100644 --- a/query/src/interpreters/interpreter_table_truncate.rs +++ b/query/src/interpreters/interpreter_table_truncate.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_use_database.rs b/query/src/interpreters/interpreter_use_database.rs index 28bd527b1947..c7d63d0c9137 100644 --- a/query/src/interpreters/interpreter_use_database.rs +++ b/query/src/interpreters/interpreter_use_database.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datavalues::DataSchema; diff --git a/query/src/interpreters/interpreter_user_alter.rs b/query/src/interpreters/interpreter_user_alter.rs index bf7706b7fbd9..d06e7bd66353 100644 --- a/query/src/interpreters/interpreter_user_alter.rs +++ b/query/src/interpreters/interpreter_user_alter.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_create.rs b/query/src/interpreters/interpreter_user_create.rs index 3202fceb1c66..a6f2f7515692 100644 --- a/query/src/interpreters/interpreter_user_create.rs +++ b/query/src/interpreters/interpreter_user_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_drop.rs b/query/src/interpreters/interpreter_user_drop.rs index bfb9b406af39..7a506af93e4c 100644 --- a/query/src/interpreters/interpreter_user_drop.rs +++ b/query/src/interpreters/interpreter_user_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_stage_create.rs b/query/src/interpreters/interpreter_user_stage_create.rs index 77c0ce1f16b1..3981911161e5 100644 --- a/query/src/interpreters/interpreter_user_stage_create.rs +++ b/query/src/interpreters/interpreter_user_stage_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_stage_describe.rs b/query/src/interpreters/interpreter_user_stage_describe.rs index 6d7f2cbb14ae..4cfed88d87dc 100644 --- a/query/src/interpreters/interpreter_user_stage_describe.rs +++ b/query/src/interpreters/interpreter_user_stage_describe.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/interpreters/interpreter_user_stage_drop.rs b/query/src/interpreters/interpreter_user_stage_drop.rs index 420d966a062c..c294c6ddd494 100644 --- a/query/src/interpreters/interpreter_user_stage_drop.rs +++ b/query/src/interpreters/interpreter_user_stage_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_udf_alter.rs b/query/src/interpreters/interpreter_user_udf_alter.rs index 8d16c1a8a1c1..dfb435b081a5 100644 --- a/query/src/interpreters/interpreter_user_udf_alter.rs +++ b/query/src/interpreters/interpreter_user_udf_alter.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_udf_create.rs b/query/src/interpreters/interpreter_user_udf_create.rs index 6111aeb49848..40f82a027d32 100644 --- a/query/src/interpreters/interpreter_user_udf_create.rs +++ b/query/src/interpreters/interpreter_user_udf_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_user_udf_drop.rs b/query/src/interpreters/interpreter_user_udf_drop.rs index 8cdf5d0ac65f..9b3a1831a465 100644 --- a/query/src/interpreters/interpreter_user_udf_drop.rs +++ b/query/src/interpreters/interpreter_user_udf_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::Result; diff --git a/query/src/interpreters/interpreter_view_alter.rs b/query/src/interpreters/interpreter_view_alter.rs index cd901004bbf6..2b910944f7b6 100644 --- a/query/src/interpreters/interpreter_view_alter.rs +++ b/query/src/interpreters/interpreter_view_alter.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::collections::BTreeMap; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_view_create.rs b/query/src/interpreters/interpreter_view_create.rs index 2c29af1c9f8e..209bef29bf3d 100644 --- a/query/src/interpreters/interpreter_view_create.rs +++ b/query/src/interpreters/interpreter_view_create.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::collections::BTreeMap; use std::sync::Arc; diff --git a/query/src/interpreters/interpreter_view_drop.rs b/query/src/interpreters/interpreter_view_drop.rs index 5b471a31ef14..5efc82e89fc1 100644 --- a/query/src/interpreters/interpreter_view_drop.rs +++ b/query/src/interpreters/interpreter_view_drop.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::Arc; use common_exception::ErrorCode; diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index d268a9c3d330..454c17300ac4 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -41,8 +41,6 @@ use tokio_stream::wrappers::IntervalStream; use tokio_stream::wrappers::ReceiverStream; use super::writers::from_clickhouse_block; - - use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::SyncReceiverCkSource; @@ -118,7 +116,9 @@ impl InteractiveWorkerBase { let mut source_pipe_builder = SourcePipeBuilder::create(); source_pipe_builder.add_source(output_port, sync_receiver_ck_source); - interpreter.set_source_pipe_builder(Option::from(source_pipe_builder)); + let _ = interpreter + .set_source_pipe_builder(Option::from(source_pipe_builder)) + .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); let (mut tx, rx) = mpsc::channel(20); tx.send(BlockItem::InsertSample(sample_block)).await.ok(); From 45211ec449f0da62789ab7e3c120e42db1f699a4 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 18 Apr 2022 22:56:15 +0800 Subject: [PATCH 11/27] fix bug --- query/src/interpreters/interpreter_insert.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index b59e768f6cae..36331f7f18b2 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -95,12 +95,12 @@ impl InsertInterpreter { ); } InsertInputSource::SelectPlan(plan) => { + need_cast_schema = self.check_schema_cast(plan)?; let select_interpreter = SelectInterpreter::try_create(self.ctx.clone(), SelectPlan { input: Arc::new((**plan).clone()), })?; pipeline = select_interpreter.create_new_pipeline()?; - need_cast_schema = self.check_schema_cast(plan)?; } }; @@ -145,6 +145,7 @@ impl InsertInterpreter { let async_runtime = self.ctx.get_storage_runtime(); + pipeline.set_max_threads(self.ctx.get_settings().get_max_threads()? as usize); let executor = PipelineCompleteExecutor::try_create(async_runtime, pipeline)?; executor.execute()?; drop(executor); @@ -273,8 +274,7 @@ impl Interpreter for InsertInterpreter { } fn create_new_pipeline(&self) -> Result { - let mut new_pipeline = NewPipeline::create(); - new_pipeline.set_max_threads(self.ctx.get_settings().get_max_threads()? as usize); + let new_pipeline = NewPipeline::create(); Ok(new_pipeline) } From 13ffc5f1c8249c3745c322df60a8e07b70c024a0 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 19 Apr 2022 14:18:34 +0800 Subject: [PATCH 12/27] make test happy --- common/datavalues/src/columns/series.rs | 1 + query/src/interpreters/interpreter_insert.rs | 26 ++++++++++---------- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/common/datavalues/src/columns/series.rs b/common/datavalues/src/columns/series.rs index 503f418e89ee..c431cabf770b 100644 --- a/common/datavalues/src/columns/series.rs +++ b/common/datavalues/src/columns/series.rs @@ -55,6 +55,7 @@ impl Series { } pub fn check_get(column: &ColumnRef) -> Result<&T> { + dbg!("xxx: {}", column.clone()); let arr = column.as_any().downcast_ref::().ok_or_else(|| { ErrorCode::UnknownColumn(format!( "downcast column error, column type: {:?}, expected column: {:?}", diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 36331f7f18b2..b4d9261d67bd 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -104,19 +104,6 @@ impl InsertInterpreter { } }; - let need_fill_missing_columns = table.schema() != plan.schema(); - if need_fill_missing_columns { - pipeline.add_transform(|transform_input_port, transform_output_port| { - TransformAddOn::try_create( - transform_input_port, - transform_output_port, - self.plan.schema(), - self.plan.schema(), - self.ctx.clone(), - ) - })?; - } - // cast schema if need_cast_schema { let mut functions = Vec::with_capacity(self.plan.schema().fields().len()); @@ -141,6 +128,19 @@ impl InsertInterpreter { })?; } + let need_fill_missing_columns = table.schema() != plan.schema(); + if need_fill_missing_columns { + pipeline.add_transform(|transform_input_port, transform_output_port| { + TransformAddOn::try_create( + transform_input_port, + transform_output_port, + self.plan.schema(), + table.schema(), + self.ctx.clone(), + ) + })?; + } + table.append2(self.ctx.clone(), &mut pipeline)?; let async_runtime = self.ctx.get_storage_runtime(); From 63e151db8cd38c4d0697c0d41128d43a9dc4bfd0 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 19 Apr 2022 15:22:04 +0800 Subject: [PATCH 13/27] delete dbg! & fix ci --- common/datavalues/src/columns/series.rs | 1 - query/src/pipelines/new/processors/transforms/transform_addon.rs | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/common/datavalues/src/columns/series.rs b/common/datavalues/src/columns/series.rs index c431cabf770b..503f418e89ee 100644 --- a/common/datavalues/src/columns/series.rs +++ b/common/datavalues/src/columns/series.rs @@ -55,7 +55,6 @@ impl Series { } pub fn check_get(column: &ColumnRef) -> Result<&T> { - dbg!("xxx: {}", column.clone()); let arr = column.as_any().downcast_ref::().ok_or_else(|| { ErrorCode::UnknownColumn(format!( "downcast column error, column type: {:?}, expected column: {:?}", diff --git a/query/src/pipelines/new/processors/transforms/transform_addon.rs b/query/src/pipelines/new/processors/transforms/transform_addon.rs index 1cc1f6f7235c..7b7d8509f9c8 100644 --- a/query/src/pipelines/new/processors/transforms/transform_addon.rs +++ b/query/src/pipelines/new/processors/transforms/transform_addon.rs @@ -60,6 +60,7 @@ where Self: Transform Box::new(Expression::Cast { expr: Box::new(expression), data_type: f.data_type().clone(), + pg_style: false, }), ); From d6d8e58101fd0359ba0817f24a6a3a6ac2c0d587 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 19 Apr 2022 21:53:03 +0800 Subject: [PATCH 14/27] fix bug in clickhouse handler& add streamsource --- query/src/pipelines/new/processors/mod.rs | 1 + .../pipelines/new/processors/sources/mod.rs | 2 + .../new/processors/sources/stream_source.rs | 66 +++++++++++++++++++ .../clickhouse/interactive_worker_base.rs | 11 +++- query/src/servers/http/clickhouse_handler.rs | 23 ++++++- 5 files changed, 98 insertions(+), 5 deletions(-) create mode 100644 query/src/pipelines/new/processors/sources/stream_source.rs diff --git a/query/src/pipelines/new/processors/mod.rs b/query/src/pipelines/new/processors/mod.rs index 868f3791234c..c01f0c20bc77 100644 --- a/query/src/pipelines/new/processors/mod.rs +++ b/query/src/pipelines/new/processors/mod.rs @@ -38,6 +38,7 @@ pub use sources::AsyncSource; pub use sources::AsyncSourcer; pub use sources::BlocksSource; pub use sources::EmptySource; +pub use sources::StreamSource; pub use sources::SyncReceiverCkSource; pub use sources::SyncReceiverSource; pub use sources::SyncSource; diff --git a/query/src/pipelines/new/processors/sources/mod.rs b/query/src/pipelines/new/processors/sources/mod.rs index 8ee836f76299..ce570288ed3f 100644 --- a/query/src/pipelines/new/processors/sources/mod.rs +++ b/query/src/pipelines/new/processors/sources/mod.rs @@ -15,6 +15,7 @@ mod async_source; mod blocks_source; mod empty_source; +mod stream_source; mod sync_ck_source_receiver; mod sync_source; mod sync_source_receiver; @@ -23,6 +24,7 @@ pub use async_source::AsyncSource; pub use async_source::AsyncSourcer; pub use blocks_source::BlocksSource; pub use empty_source::EmptySource; +pub use stream_source::StreamSource; pub use sync_ck_source_receiver::SyncReceiverCkSource; pub use sync_source::SyncSource; pub use sync_source::SyncSourcer; diff --git a/query/src/pipelines/new/processors/sources/stream_source.rs b/query/src/pipelines/new/processors/sources/stream_source.rs new file mode 100644 index 000000000000..ffbdb43578ad --- /dev/null +++ b/query/src/pipelines/new/processors/sources/stream_source.rs @@ -0,0 +1,66 @@ +// Copyright 2022 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::borrow::Borrow; +use std::future::Future; +use std::ops::Deref; +use std::rc::Rc; +use std::sync::Arc; + +use common_datablocks::DataBlock; +use common_exception::ErrorCode; +use common_exception::Result; +use common_streams::SendableDataBlockStream; +use futures::StreamExt; + +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::AsyncSource; +use crate::pipelines::new::processors::AsyncSourcer; +use crate::sessions::QueryContext; + +pub struct StreamSource { + stream: Option, +} + +impl StreamSource { + pub fn create( + ctx: Arc, + stream: Option, + out: Arc, + ) -> Result { + AsyncSourcer::create(ctx, out, StreamSource { stream }) + } +} + +impl AsyncSource for StreamSource { + const NAME: &'static str = "stream source"; + type BlockFuture<'a> = impl Future>> where Self: 'a; + + fn generate(&mut self) -> Self::BlockFuture<'_> { + async move { + match self + .stream + .take() + .ok_or_else(|| ErrorCode::EmptyData("input stream not exist or consumed"))? + .next() + .await + { + Some(Ok(block)) => Ok(Some(block)), + Some(Err(e)) => Err(e), + None => Ok(None), + } + } + } +} diff --git a/query/src/servers/clickhouse/interactive_worker_base.rs b/query/src/servers/clickhouse/interactive_worker_base.rs index 454c17300ac4..00bf9937442f 100644 --- a/query/src/servers/clickhouse/interactive_worker_base.rs +++ b/query/src/servers/clickhouse/interactive_worker_base.rs @@ -123,20 +123,25 @@ impl InteractiveWorkerBase { let (mut tx, rx) = mpsc::channel(20); tx.send(BlockItem::InsertSample(sample_block)).await.ok(); - // the data is comming in async mode + // the data is coming in async mode let sent_all_data = ch_ctx.state.sent_all_data.clone(); + let start = Instant::now(); ctx.try_spawn(async move { interpreter.execute(None).await.unwrap(); sent_all_data.notify_one(); })?; - + histogram!( + super::clickhouse_metrics::METRIC_INTERPRETER_USEDTIME, + start.elapsed(), + "interpreter" => name + ); return Ok(rx); } let (mut tx, rx) = mpsc::channel(20); tx.send(BlockItem::InsertSample(sample_block)).await.ok(); - // the data is comming in async mode + // the data is coming in async mode let sent_all_data = ch_ctx.state.sent_all_data.clone(); let start = Instant::now(); ctx.try_spawn(async move { diff --git a/query/src/servers/http/clickhouse_handler.rs b/query/src/servers/http/clickhouse_handler.rs index 29610e7ffc56..6935b741f4af 100644 --- a/query/src/servers/http/clickhouse_handler.rs +++ b/query/src/servers/http/clickhouse_handler.rs @@ -24,6 +24,7 @@ use common_streams::NDJsonSourceBuilder; use common_streams::SendableDataBlockStream; use common_streams::SourceStream; use common_tracing::tracing; +use futures::FutureExt; use futures::StreamExt; use poem::error::BadRequest; use poem::error::InternalServerError; @@ -38,6 +39,12 @@ use poem::Route; use serde::Deserialize; use crate::interpreters::InterpreterFactory; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::Event::Async; +use crate::pipelines::new::processors::AsyncSourcer; +use crate::pipelines::new::processors::StreamSource; +use crate::pipelines::new::processors::SyncReceiverCkSource; +use crate::pipelines::new::SourcePipeBuilder; use crate::servers::http::formats::tsv_output::block_to_tsv; use crate::servers::http::formats::Format; use crate::sessions::QueryContext; @@ -70,8 +77,20 @@ async fn execute( .start() .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - - let data_stream = interpreter.execute(input_stream).await?; + let data_stream: SendableDataBlockStream; + if ctx.get_settings().get_enable_new_processor_framework()? != 0 && ctx.get_cluster().is_empty() + { + let output_port = OutputPort::create(); + let stream_source = StreamSource::create(ctx.clone(), input_stream, output_port.clone())?; + let mut source_pipe_builder = SourcePipeBuilder::create(); + source_pipe_builder.add_source(output_port, stream_source); + let _ = interpreter + .set_source_pipe_builder(Option::from(source_pipe_builder)) + .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); + data_stream = interpreter.execute(None).await?; + } else { + data_stream = interpreter.execute(input_stream).await?; + } let mut data_stream = ctx.try_create_abortable(data_stream)?; let stream = stream! { From 3aba2262b1be2037ad8caebceba55a132577ffaa Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 19 Apr 2022 22:10:57 +0800 Subject: [PATCH 15/27] cargo fix --- .../new/processors/sources/stream_source.rs | 3 -- query/src/servers/http/clickhouse_handler.rs | 34 +++++++++---------- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/query/src/pipelines/new/processors/sources/stream_source.rs b/query/src/pipelines/new/processors/sources/stream_source.rs index ffbdb43578ad..12029d228740 100644 --- a/query/src/pipelines/new/processors/sources/stream_source.rs +++ b/query/src/pipelines/new/processors/sources/stream_source.rs @@ -12,10 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::Borrow; use std::future::Future; -use std::ops::Deref; -use std::rc::Rc; use std::sync::Arc; use common_datablocks::DataBlock; diff --git a/query/src/servers/http/clickhouse_handler.rs b/query/src/servers/http/clickhouse_handler.rs index 6935b741f4af..046ea0b3bc03 100644 --- a/query/src/servers/http/clickhouse_handler.rs +++ b/query/src/servers/http/clickhouse_handler.rs @@ -24,7 +24,6 @@ use common_streams::NDJsonSourceBuilder; use common_streams::SendableDataBlockStream; use common_streams::SourceStream; use common_tracing::tracing; -use futures::FutureExt; use futures::StreamExt; use poem::error::BadRequest; use poem::error::InternalServerError; @@ -40,10 +39,7 @@ use serde::Deserialize; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; -use crate::pipelines::new::processors::processor::Event::Async; -use crate::pipelines::new::processors::AsyncSourcer; use crate::pipelines::new::processors::StreamSource; -use crate::pipelines::new::processors::SyncReceiverCkSource; use crate::pipelines::new::SourcePipeBuilder; use crate::servers::http::formats::tsv_output::block_to_tsv; use crate::servers::http::formats::Format; @@ -77,20 +73,22 @@ async fn execute( .start() .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - let data_stream: SendableDataBlockStream; - if ctx.get_settings().get_enable_new_processor_framework()? != 0 && ctx.get_cluster().is_empty() - { - let output_port = OutputPort::create(); - let stream_source = StreamSource::create(ctx.clone(), input_stream, output_port.clone())?; - let mut source_pipe_builder = SourcePipeBuilder::create(); - source_pipe_builder.add_source(output_port, stream_source); - let _ = interpreter - .set_source_pipe_builder(Option::from(source_pipe_builder)) - .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); - data_stream = interpreter.execute(None).await?; - } else { - data_stream = interpreter.execute(input_stream).await?; - } + let data_stream: SendableDataBlockStream = + if ctx.get_settings().get_enable_new_processor_framework()? != 0 + && ctx.get_cluster().is_empty() + { + let output_port = OutputPort::create(); + let stream_source = + StreamSource::create(ctx.clone(), input_stream, output_port.clone())?; + let mut source_pipe_builder = SourcePipeBuilder::create(); + source_pipe_builder.add_source(output_port, stream_source); + let _ = interpreter + .set_source_pipe_builder(Option::from(source_pipe_builder)) + .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); + interpreter.execute(None).await? + } else { + interpreter.execute(input_stream).await? + }; let mut data_stream = ctx.try_create_abortable(data_stream)?; let stream = stream! { From d586c0f6f7d51765421b56a1ba34910ada81d86a Mon Sep 17 00:00:00 2001 From: xudong963 Date: Tue, 19 Apr 2022 23:15:22 +0800 Subject: [PATCH 16/27] fix comment --- query/src/pipelines/new/processors/sources/stream_source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/query/src/pipelines/new/processors/sources/stream_source.rs b/query/src/pipelines/new/processors/sources/stream_source.rs index 12029d228740..4aa540c68474 100644 --- a/query/src/pipelines/new/processors/sources/stream_source.rs +++ b/query/src/pipelines/new/processors/sources/stream_source.rs @@ -49,7 +49,7 @@ impl AsyncSource for StreamSource { async move { match self .stream - .take() + .as_mut() .ok_or_else(|| ErrorCode::EmptyData("input stream not exist or consumed"))? .next() .await From 30b39a09ccb74f6af0ad7bea5cf7e3cd70d67636 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 08:20:10 +0800 Subject: [PATCH 17/27] fix streaming load --- query/src/servers/http/v1/load.rs | 36 ++++++++++++++++++++++++++----- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 92d1f7bc3068..83b9c5d1bf61 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -43,6 +43,9 @@ use serde::Deserialize; use serde::Serialize; use crate::interpreters::InterpreterFactory; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::StreamSource; +use crate::pipelines::new::SourcePipeBuilder; use crate::sessions::SessionManager; use crate::sessions::SessionType; use crate::sql::PlanParser; @@ -152,11 +155,34 @@ pub async fn streaming_load( .await .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); - // this runs inside the runtime of poem, load is not cpu densive so it's ok - let mut data_stream = interpreter - .execute(Some(source_stream)) - .await - .map_err(InternalServerError)?; + // this runs inside the runtime of poem, load is not cpu defensive so it's ok + let mut data_stream = if context + .get_settings() + .get_enable_new_processor_framework() + .unwrap() + != 0 + && context.get_cluster().is_empty() + { + let output_port = OutputPort::create(); + let stream_source = + StreamSource::create(context.clone(), Some(source_stream), output_port.clone()) + .unwrap(); + let mut source_pipe_builder = SourcePipeBuilder::create(); + source_pipe_builder.add_source(output_port, stream_source); + let _ = interpreter + .set_source_pipe_builder(Option::from(source_pipe_builder)) + .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); + interpreter + .execute(None) + .await + .map_err(InternalServerError)? + } else { + interpreter + .execute(Some(source_stream)) + .await + .map_err(InternalServerError)? + }; + while let Some(_block) = data_stream.next().await {} // Write Finish to query log table. From 035c815300fb47811fbc0bbc34372d247ce539c4 Mon Sep 17 00:00:00 2001 From: sundyli <543950155@qq.com> Date: Wed, 20 Apr 2022 09:07:25 +0800 Subject: [PATCH 18/27] feat(sources): add StreamSourceV2 --- .../processors/sources/stream_source_v2.rs | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 query/src/pipelines/new/processors/sources/stream_source_v2.rs diff --git a/query/src/pipelines/new/processors/sources/stream_source_v2.rs b/query/src/pipelines/new/processors/sources/stream_source_v2.rs new file mode 100644 index 000000000000..2303974e5165 --- /dev/null +++ b/query/src/pipelines/new/processors/sources/stream_source_v2.rs @@ -0,0 +1,52 @@ +// Copyright 2022 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::future::Future; +use std::sync::Arc; + +use common_datablocks::DataBlock; +use common_exception::ErrorCode; +use common_exception::Result; +use common_streams::SendableDataBlockStream; +use common_streams::Source; +use futures::StreamExt; + +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::ProcessorPtr; +use crate::pipelines::new::processors::AsyncSource; +use crate::pipelines::new::processors::AsyncSourcer; +use crate::sessions::QueryContext; + +pub struct StreamSourceV2 { + s: Box, +} + +impl StreamSourceV2 { + pub fn create( + ctx: Arc, + s: Box, + out: Arc, + ) -> Result { + AsyncSourcer::create(ctx, out, StreamSourceV2 { s }) + } +} + +impl AsyncSource for StreamSourceV2 { + const NAME: &'static str = "stream source"; + type BlockFuture<'a> = impl Future>> where Self: 'a; + + fn generate(&mut self) -> Self::BlockFuture<'_> { + async move { self.s.read().await } + } +} From 5d61038c078697b6b9499ef353c66f9615f92d07 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 14:41:46 +0800 Subject: [PATCH 19/27] try to debug --- .../0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index 5231564f4fc7..4fad3cf7581b 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -10,7 +10,7 @@ insert into t values (5); insert into t values (6); insert into t values (7); -optimize table t compact; +-- optimize table t compact; -- optimize exact explain select count(1) from t; From 103e27721b27c1e0961d74307f7b8563e6313b35 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 15:28:57 +0800 Subject: [PATCH 20/27] process copy and test correctness by ci --- query/src/interpreters/interpreter_copy.rs | 22 ++++++++++++++++--- .../09_0008_fuse_optimize_table.sql | 2 +- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/query/src/interpreters/interpreter_copy.rs b/query/src/interpreters/interpreter_copy.rs index 9a191f61a8e4..8e32f66f9b1f 100644 --- a/query/src/interpreters/interpreter_copy.rs +++ b/query/src/interpreters/interpreter_copy.rs @@ -31,6 +31,7 @@ use regex::Regex; use crate::interpreters::stream::ProcessorExecutorStream; use crate::interpreters::Interpreter; use crate::interpreters::InterpreterPtr; +use crate::pipelines::new::executor::PipelineCompleteExecutor; use crate::pipelines::new::executor::PipelinePullingExecutor; use crate::pipelines::new::NewPipeline; use crate::sessions::QueryContext; @@ -112,15 +113,30 @@ impl CopyInterpreter { if let Err(e) = res { return Err(e); } + + let table = ctx + .get_table(&self.plan.db_name, &self.plan.tbl_name) + .await?; + + if ctx.get_settings().get_enable_new_processor_framework()? != 0 + && self.ctx.get_cluster().is_empty() + { + table.append2(ctx.clone(), &mut pipeline)?; + pipeline.set_max_threads(settings.get_max_threads()? as usize); + + let async_runtime = ctx.get_storage_runtime(); + let executor = PipelineCompleteExecutor::try_create(async_runtime, pipeline)?; + executor.execute()?; + + return Ok(ctx.consume_precommit_blocks()); + } + pipeline.set_max_threads(settings.get_max_threads()? as usize); let async_runtime = ctx.get_storage_runtime(); let executor = PipelinePullingExecutor::try_create(async_runtime, pipeline)?; let source_stream = Box::pin(ProcessorExecutorStream::create(executor)?); - let table = ctx - .get_table(&self.plan.db_name, &self.plan.tbl_name) - .await?; let operations = table .append_data(ctx.clone(), source_stream) .await? diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index 4fad3cf7581b..5231564f4fc7 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -10,7 +10,7 @@ insert into t values (5); insert into t values (6); insert into t values (7); --- optimize table t compact; +optimize table t compact; -- optimize exact explain select count(1) from t; From 811b40ac84eb4eea6a583664f8434af3a5c485cc Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 16:29:05 +0800 Subject: [PATCH 21/27] add more test for optimize table --- .../09_fuse_engine/09_0008_fuse_optimize_table.result | 3 +++ .../0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql | 1 + 2 files changed, 4 insertions(+) diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result index 46086c759843..65822740cd37 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result @@ -1,3 +1,6 @@ +5 +6 +7 Projection: count():UInt64 Projection: 3 as count():UInt64 Expression: 3:UInt64 (Exact Statistics) diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index 5231564f4fc7..4d654e554596 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -11,6 +11,7 @@ insert into t values (6); insert into t values (7); optimize table t compact; +select * from t order by a; -- optimize exact explain select count(1) from t; From 04b5c68f41c4498a5c4cd45cd9f20d69019013e4 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 16:35:20 +0800 Subject: [PATCH 22/27] add more test for optimize table --- .../09_fuse_engine/09_0008_fuse_optimize_table.result | 9 +++++++++ .../09_fuse_engine/09_0008_fuse_optimize_table.sql | 3 ++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result index 65822740cd37..342cb99089b9 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.result @@ -1,6 +1,9 @@ 5 6 7 +5 +6 +7 Projection: count():UInt64 Projection: 3 as count():UInt64 Expression: 3:UInt64 (Exact Statistics) @@ -13,6 +16,12 @@ Projection: count():UInt64 5 6 7 +5 +6 +7 +8 +9 +10 1 5 6 diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index 4d654e554596..3b17e62c9d79 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -9,7 +9,7 @@ create table t(a uint64); insert into t values (5); insert into t values (6); insert into t values (7); - +select * from t order by a; optimize table t compact; select * from t order by a; @@ -36,6 +36,7 @@ select * from t order by a; insert into t values (8); insert into t values (9); insert into t values (10); +select * from t order by a; -- purge and compact optimize table 't' all; -- expect 1 snapshot left From 00588cb0636339d6e51644e426b16ae65e5c3d28 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 18:46:04 +0800 Subject: [PATCH 23/27] add more test for insert --- tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.result | 2 ++ tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.sql | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.result b/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.result index ee070b75edb7..61e666289ba3 100644 --- a/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.result +++ b/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.result @@ -1,5 +1,7 @@ 1 1 1 origin 2 2 2 origin +1 1 1 origin +2 2 2 origin 3 3 3 change 4 4 4 change 5 5 5 change2 diff --git a/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.sql b/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.sql index 79b923055ed7..068d9076c4b7 100644 --- a/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.sql +++ b/tests/suites/0_stateless/03_dml/03_0017_insert_overwrite.sql @@ -8,6 +8,8 @@ CREATE TABLE IF NOT EXISTS t2(a Int8, b UInt32, c UInt64, d String) Engine = MEM INSERT INTO t1 (a,b,c,d) VALUES(1, 1, 1, 'origin'), (2, 2, 2, 'origin'); INSERT INTO t2 (a,b,c,d) VALUES(3, 3, 3, 'change'), (4, 4, 4, 'change'); +select * from t1; +INSERT OVERWRITE t1 select * from t1; select * from t1; INSERT OVERWRITE t1 select * from t2; select * from t1; From aaced353af5c1d7a0c461090dd29713b9069f807 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Wed, 20 Apr 2022 20:16:15 +0800 Subject: [PATCH 24/27] set max_threads = 16 --- .../0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index 3b17e62c9d79..ab73afb72c11 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -1,3 +1,4 @@ +set max_threads = 16; DROP DATABASE IF EXISTS db_09_0008; CREATE DATABASE db_09_0008; USE db_09_0008; From 9fdd9a9b6159fce2eaa169026bad21c5cba4a327 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 21 Apr 2022 03:04:07 +0800 Subject: [PATCH 25/27] fix test fail --- query/src/pipelines/new/executor/executor_graph.rs | 1 - query/src/pipelines/new/executor/pipeline_executor.rs | 1 - .../new/processors/transforms/transform_block_compact.rs | 2 +- .../0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) diff --git a/query/src/pipelines/new/executor/executor_graph.rs b/query/src/pipelines/new/executor/executor_graph.rs index dcc1180fb69c..aebf34b9d971 100644 --- a/query/src/pipelines/new/executor/executor_graph.rs +++ b/query/src/pipelines/new/executor/executor_graph.rs @@ -291,7 +291,6 @@ impl ScheduleQueue { while let Some(processor) = self.async_queue.pop_front() { tasks.push_back(ExecutorTask::Async(processor)); } - global.push_tasks(ctx, tasks) } diff --git a/query/src/pipelines/new/executor/pipeline_executor.rs b/query/src/pipelines/new/executor/pipeline_executor.rs index b7b7c37e3adc..97f4bcac0e5d 100644 --- a/query/src/pipelines/new/executor/pipeline_executor.rs +++ b/query/src/pipelines/new/executor/pipeline_executor.rs @@ -50,7 +50,6 @@ impl PipelineExecutor { while let Some(task) = init_schedule_queue.pop_task() { tasks.push_back(task); } - global_tasks_queue.init_tasks(tasks); Ok(Arc::new(PipelineExecutor { diff --git a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs index 50eff0a4a75a..0f5ddaa97b4c 100644 --- a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs +++ b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs @@ -120,7 +120,7 @@ impl Compactor for BlockCompactor { )); } } else { - blocks.push(block); + res.push(block); } } diff --git a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql index ab73afb72c11..3b17e62c9d79 100644 --- a/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql +++ b/tests/suites/0_stateless/09_fuse_engine/09_0008_fuse_optimize_table.sql @@ -1,4 +1,3 @@ -set max_threads = 16; DROP DATABASE IF EXISTS db_09_0008; CREATE DATABASE db_09_0008; USE db_09_0008; From 500cd4e89b9c103d57f33b6e4242b0fd584d2c57 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 21 Apr 2022 10:59:32 +0800 Subject: [PATCH 26/27] bug free:> --- .../new/processors/transforms/transform_block_compact.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs index 0f5ddaa97b4c..2808c3fd9a11 100644 --- a/query/src/pipelines/new/processors/transforms/transform_block_compact.rs +++ b/query/src/pipelines/new/processors/transforms/transform_block_compact.rs @@ -107,20 +107,21 @@ impl Compactor for BlockCompactor { blocks.remove(size - 1); } else { let accumulated_rows: usize = blocks.iter_mut().map(|b| b.num_rows()).sum(); + let merged = DataBlock::concat_blocks(blocks)?; blocks.clear(); if accumulated_rows >= self.max_row_per_block { - let cut = block.slice(0, self.max_row_per_block); + let cut = merged.slice(0, self.max_row_per_block); res.push(cut); if accumulated_rows != self.max_row_per_block { - blocks.push(block.slice( + blocks.push(merged.slice( self.max_row_per_block, accumulated_rows - self.max_row_per_block, )); } } else { - res.push(block); + blocks.push(merged); } } From f6dba1381524978c0b5e6b368910bdb93caa4493 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Apr 2022 09:53:54 +0800 Subject: [PATCH 27/27] fix streaming load block --- common/streams/src/sources/mod.rs | 3 + query/src/pipelines/new/processors/mod.rs | 1 + .../pipelines/new/processors/sources/mod.rs | 2 + .../processors/sources/stream_source_v2.rs | 3 - query/src/servers/http/v1/load.rs | 183 +++++++++++++++--- 5 files changed, 162 insertions(+), 30 deletions(-) diff --git a/common/streams/src/sources/mod.rs b/common/streams/src/sources/mod.rs index 8d914d327e0f..a87a44667be2 100644 --- a/common/streams/src/sources/mod.rs +++ b/common/streams/src/sources/mod.rs @@ -18,6 +18,9 @@ mod source_ndjson; mod source_parquet; pub use source::Source; +pub use source_csv::CsvSource; pub use source_csv::CsvSourceBuilder; +pub use source_ndjson::NDJsonSource; pub use source_ndjson::NDJsonSourceBuilder; +pub use source_parquet::ParquetSource; pub use source_parquet::ParquetSourceBuilder; diff --git a/query/src/pipelines/new/processors/mod.rs b/query/src/pipelines/new/processors/mod.rs index c01f0c20bc77..9c76bfaf69bb 100644 --- a/query/src/pipelines/new/processors/mod.rs +++ b/query/src/pipelines/new/processors/mod.rs @@ -39,6 +39,7 @@ pub use sources::AsyncSourcer; pub use sources::BlocksSource; pub use sources::EmptySource; pub use sources::StreamSource; +pub use sources::StreamSourceV2; pub use sources::SyncReceiverCkSource; pub use sources::SyncReceiverSource; pub use sources::SyncSource; diff --git a/query/src/pipelines/new/processors/sources/mod.rs b/query/src/pipelines/new/processors/sources/mod.rs index ce570288ed3f..56f6880531ea 100644 --- a/query/src/pipelines/new/processors/sources/mod.rs +++ b/query/src/pipelines/new/processors/sources/mod.rs @@ -16,6 +16,7 @@ mod async_source; mod blocks_source; mod empty_source; mod stream_source; +mod stream_source_v2; mod sync_ck_source_receiver; mod sync_source; mod sync_source_receiver; @@ -25,6 +26,7 @@ pub use async_source::AsyncSourcer; pub use blocks_source::BlocksSource; pub use empty_source::EmptySource; pub use stream_source::StreamSource; +pub use stream_source_v2::StreamSourceV2; pub use sync_ck_source_receiver::SyncReceiverCkSource; pub use sync_source::SyncSource; pub use sync_source::SyncSourcer; diff --git a/query/src/pipelines/new/processors/sources/stream_source_v2.rs b/query/src/pipelines/new/processors/sources/stream_source_v2.rs index 2303974e5165..5d1ca63cd139 100644 --- a/query/src/pipelines/new/processors/sources/stream_source_v2.rs +++ b/query/src/pipelines/new/processors/sources/stream_source_v2.rs @@ -16,11 +16,8 @@ use std::future::Future; use std::sync::Arc; use common_datablocks::DataBlock; -use common_exception::ErrorCode; use common_exception::Result; -use common_streams::SendableDataBlockStream; use common_streams::Source; -use futures::StreamExt; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::ProcessorPtr; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 83b9c5d1bf61..dc5bb94e7713 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -44,8 +44,9 @@ use serde::Serialize; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; -use crate::pipelines::new::processors::StreamSource; +use crate::pipelines::new::processors::StreamSourceV2; use crate::pipelines::new::SourcePipeBuilder; +use crate::sessions::QueryContext; use crate::sessions::SessionManager; use crate::sessions::SessionType; use crate::sql::PlanParser; @@ -110,6 +111,82 @@ pub async fn streaming_load( let format_settings = context.get_format_settings().map_err(InternalServerError)?; + if context + .get_settings() + .get_enable_new_processor_framework() + .unwrap() + != 0 + && context.get_cluster().is_empty() + { + let source_pipe_builder = match &plan { + PlanNode::Insert(insert) => match &insert.source { + InsertInputSource::StreamingWithFormat(format) => { + if format.to_lowercase().as_str() == "csv" { + csv_source_pipe_builder( + context.clone(), + &plan, + &format_settings, + multipart, + max_block_size, + ) + .await + } else if format.to_lowercase().as_str() == "parquet" { + parquet_source_pipe_builder(context.clone(), &plan, multipart).await + } else if format.to_lowercase().as_str() == "ndjson" + || format.to_lowercase().as_str() == "jsoneachrow" + { + ndjson_source_pipe_builder(context.clone(), &plan, multipart).await + } else { + Err(poem::Error::from_string( + format!( + "Streaming load only supports csv format, but got {}", + format + ), + StatusCode::BAD_REQUEST, + )) + } + } + _non_supported_source => Err(poem::Error::from_string( + "Only supports streaming upload. e.g. INSERT INTO $table FORMAT CSV", + StatusCode::BAD_REQUEST, + )), + }, + non_insert_plan => Err(poem::Error::from_string( + format!( + "Only supports INSERT statement in streaming load, but got {}", + non_insert_plan.name() + ), + StatusCode::BAD_REQUEST, + )), + }?; + let interpreter = + InterpreterFactory::get(context.clone(), plan.clone()).map_err(InternalServerError)?; + let _ = interpreter + .set_source_pipe_builder(Option::from(source_pipe_builder)) + .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); + let mut data_stream = interpreter + .execute(None) + .await + .map_err(InternalServerError)?; + while let Some(_block) = data_stream.next().await {} + // Write Finish to query log table. + let _ = interpreter + .finish() + .await + .map_err(|e| tracing::error!("interpreter.finish error: {:?}", e)); + + // TODO generate id + // TODO duplicate by insert_label + let mut id = uuid::Uuid::new_v4().to_string(); + return Ok(Json(LoadResponse { + id, + state: "SUCCESS".to_string(), + stats: context.get_scan_progress_value(), + error: None, + })); + }; + + // After new processor is ready, the following code can directly delete // validate plan let source_stream = match &plan { PlanNode::Insert(insert) => match &insert.source { @@ -156,32 +233,10 @@ pub async fn streaming_load( .map_err(|e| tracing::error!("interpreter.start.error: {:?}", e)); // this runs inside the runtime of poem, load is not cpu defensive so it's ok - let mut data_stream = if context - .get_settings() - .get_enable_new_processor_framework() - .unwrap() - != 0 - && context.get_cluster().is_empty() - { - let output_port = OutputPort::create(); - let stream_source = - StreamSource::create(context.clone(), Some(source_stream), output_port.clone()) - .unwrap(); - let mut source_pipe_builder = SourcePipeBuilder::create(); - source_pipe_builder.add_source(output_port, stream_source); - let _ = interpreter - .set_source_pipe_builder(Option::from(source_pipe_builder)) - .map_err(|e| tracing::error!("interpreter.set_source_pipe_builder.error: {:?}", e)); - interpreter - .execute(None) - .await - .map_err(InternalServerError)? - } else { - interpreter - .execute(Some(source_stream)) - .await - .map_err(InternalServerError)? - }; + let mut data_stream = interpreter + .execute(Some(source_stream)) + .await + .map_err(InternalServerError)?; while let Some(_block) = data_stream.next().await {} @@ -280,3 +335,77 @@ fn build_csv_stream( Ok(Box::pin(stream)) } + +async fn csv_source_pipe_builder( + ctx: Arc, + plan: &PlanNode, + format_settings: &FormatSettings, + mut multipart: Multipart, + block_size: usize, +) -> PoemResult { + let mut builder = CsvSourceBuilder::create(plan.schema(), format_settings.clone()); + builder.block_size(block_size); + let mut source_pipe_builder = SourcePipeBuilder::create(); + while let Ok(Some(field)) = multipart.next_field().await { + let bytes = field + .bytes() + .await + .map_err_to_code(ErrorCode::BadBytes, || "Read part to field bytes error") + .unwrap(); + let cursor = Cursor::new(bytes); + let csv_source = builder.build(cursor).unwrap(); + let output_port = OutputPort::create(); + let source = + StreamSourceV2::create(ctx.clone(), Box::new(csv_source), output_port.clone()).unwrap(); + source_pipe_builder.add_source(output_port, source); + } + Ok(source_pipe_builder) +} + +async fn parquet_source_pipe_builder( + ctx: Arc, + plan: &PlanNode, + mut multipart: Multipart, +) -> PoemResult { + let builder = ParquetSourceBuilder::create(plan.schema()); + let mut source_pipe_builder = SourcePipeBuilder::create(); + while let Ok(Some(field)) = multipart.next_field().await { + let bytes = field + .bytes() + .await + .map_err_to_code(ErrorCode::BadBytes, || "Read part to field bytes error") + .unwrap(); + let cursor = Cursor::new(bytes); + let parquet_source = builder.build(cursor).unwrap(); + let output_port = OutputPort::create(); + let source = + StreamSourceV2::create(ctx.clone(), Box::new(parquet_source), output_port.clone()) + .unwrap(); + source_pipe_builder.add_source(output_port, source); + } + Ok(source_pipe_builder) +} + +async fn ndjson_source_pipe_builder( + ctx: Arc, + plan: &PlanNode, + mut multipart: Multipart, +) -> PoemResult { + let builder = NDJsonSourceBuilder::create(plan.schema()); + let mut source_pipe_builder = SourcePipeBuilder::create(); + while let Ok(Some(field)) = multipart.next_field().await { + let bytes = field + .bytes() + .await + .map_err_to_code(ErrorCode::BadBytes, || "Read part to field bytes error") + .unwrap(); + let cursor = Cursor::new(bytes); + let ndjson_source = builder.build(cursor).unwrap(); + let output_port = OutputPort::create(); + let source = + StreamSourceV2::create(ctx.clone(), Box::new(ndjson_source), output_port.clone()) + .unwrap(); + source_pipe_builder.add_source(output_port, source); + } + Ok(source_pipe_builder) +}