Skip to content

Commit 6deb026

Browse files
committed
refactor: rewrite select plan into insert-select plan.
1 parent ec4ca08 commit 6deb026

File tree

2 files changed

+50
-28
lines changed

2 files changed

+50
-28
lines changed

src/query/service/src/interpreters/interpreter_insert_v2.rs

Lines changed: 47 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ use common_datavalues::DataType;
2020
use common_exception::ErrorCode;
2121
use common_exception::Result;
2222
use common_functions::scalars::CastFunction;
23-
use common_planners::StageKind;
2423
use common_streams::DataBlockStream;
2524
use common_streams::SendableDataBlockStream;
2625
use futures_util::StreamExt;
@@ -45,7 +44,6 @@ use crate::pipelines::SourcePipeBuilder;
4544
use crate::sessions::QueryContext;
4645
use crate::sessions::TableContext;
4746
use crate::sql::executor::DistributedInsertSelect;
48-
use crate::sql::executor::Exchange;
4947
use crate::sql::executor::PhysicalPlan;
5048
use crate::sql::executor::PhysicalPlanBuilder;
5149
use crate::sql::plans::Insert;
@@ -115,9 +113,13 @@ impl InsertInterpreterV2 {
115113
InsertInputSource::SelectPlan(plan) => {
116114
if !self.ctx.get_cluster().is_empty() {
117115
// distributed insert select
118-
return self
116+
if let Some(stream) = self
119117
.schedule_insert_select(plan, self.plan.catalog.clone(), table.clone())
120-
.await;
118+
.await?
119+
{
120+
return Ok(stream);
121+
}
122+
// else the plan cannot be executed in cluster mode, fallback to standalone mode
121123
}
122124

123125
let select_interpreter = match &**plan {
@@ -202,8 +204,9 @@ impl InsertInterpreterV2 {
202204
plan: &Plan,
203205
catalog: String,
204206
table: Arc<dyn Table>,
205-
) -> Result<SendableDataBlockStream> {
206-
let (inner_plan, select_column_bindings) = match plan {
207+
) -> Result<Option<SendableDataBlockStream>> {
208+
// select_plan is already distributed optimized
209+
let (mut select_plan, select_column_bindings) = match plan {
207210
Plan::Query {
208211
s_expr,
209212
metadata,
@@ -216,26 +219,44 @@ impl InsertInterpreterV2 {
216219
_ => unreachable!(),
217220
};
218221

222+
if !select_plan.is_distributed_plan() {
223+
return Ok(None);
224+
}
225+
219226
table.get_table_info();
220227

221-
let insert_select_plan =
222-
PhysicalPlan::DistributedInsertSelect(Box::new(DistributedInsertSelect {
223-
input: Box::new(inner_plan),
224-
catalog,
225-
table_info: table.get_table_info().clone(),
226-
select_schema: plan.schema(),
227-
select_column_bindings,
228-
insert_schema: self.plan.schema(),
229-
cast_needed: self.check_schema_cast(plan)?,
230-
}));
231-
232-
let final_plan = PhysicalPlan::Exchange(Exchange {
233-
input: Box::new(insert_select_plan),
234-
kind: StageKind::Merge,
235-
keys: vec![],
236-
});
237-
238-
let mut build_res = build_schedule_pipepline(self.ctx.clone(), &final_plan).await?;
228+
let insert_select_plan = match select_plan {
229+
PhysicalPlan::Exchange(ref mut exchange) => {
230+
// insert can be dispatched to different nodes
231+
let input = exchange.input.clone();
232+
exchange.input = Box::new(PhysicalPlan::DistributedInsertSelect(Box::new(
233+
DistributedInsertSelect {
234+
input,
235+
catalog,
236+
table_info: table.get_table_info().clone(),
237+
select_schema: plan.schema(),
238+
select_column_bindings,
239+
insert_schema: self.plan.schema(),
240+
cast_needed: self.check_schema_cast(plan)?,
241+
},
242+
)));
243+
select_plan
244+
}
245+
other_plan => {
246+
// insert should wait until all nodes finished
247+
PhysicalPlan::DistributedInsertSelect(Box::new(DistributedInsertSelect {
248+
input: Box::new(other_plan),
249+
catalog,
250+
table_info: table.get_table_info().clone(),
251+
select_schema: plan.schema(),
252+
select_column_bindings,
253+
insert_schema: self.plan.schema(),
254+
cast_needed: self.check_schema_cast(plan)?,
255+
}))
256+
}
257+
};
258+
259+
let mut build_res = build_schedule_pipepline(self.ctx.clone(), &insert_select_plan).await?;
239260

240261
let settings = self.ctx.get_settings();
241262
let query_need_abort = self.ctx.query_need_abort();
@@ -256,11 +277,11 @@ impl InsertInterpreterV2 {
256277

257278
commit2table(self.ctx.clone(), table.clone(), self.plan.overwrite).await?;
258279

259-
Ok(Box::pin(DataBlockStream::create(
280+
Ok(Some(Box::pin(DataBlockStream::create(
260281
self.plan.schema(),
261282
None,
262283
vec![],
263-
)))
284+
))))
264285
}
265286
}
266287

src/query/service/src/sql/planner/binder/insert.rs

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -125,8 +125,9 @@ impl<'a> Binder {
125125
InsertSource::Select { query } => {
126126
let statement = Statement::Query(query);
127127
let select_plan = self.bind_statement(bind_context, &statement).await?;
128-
// Don't enable distributed optimization for `INSERT INTO ... SELECT ...` for now
129-
let opt_ctx = Arc::new(OptimizerContext::new(OptimizerConfig::default()));
128+
let opt_ctx = Arc::new(OptimizerContext::new(OptimizerConfig {
129+
enable_distributed_optimization: true,
130+
}));
130131
let optimized_plan = optimize(self.ctx.clone(), opt_ctx, select_plan)?;
131132
Ok(InsertInputSource::SelectPlan(Box::new(optimized_plan)))
132133
}

0 commit comments

Comments
 (0)