-
Notifications
You must be signed in to change notification settings - Fork 525
/
local.rs
493 lines (460 loc) · 21.6 KB
/
local.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
// Copyright 2023 RisingWave 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.
//! Local execution for batch query.
use std::collections::HashMap;
use std::sync::Arc;
use anyhow::Context;
use futures::StreamExt;
use futures_async_stream::try_stream;
use itertools::Itertools;
use pgwire::pg_server::BoxedError;
use rand::seq::SliceRandom;
use risingwave_batch::executor::{BoxedDataChunkStream, ExecutorBuilder};
use risingwave_batch::task::TaskId;
use risingwave_common::array::DataChunk;
use risingwave_common::bail;
use risingwave_common::error::RwError;
use risingwave_common::hash::ParallelUnitMapping;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_common::util::stream_cancel::{cancellable_stream, Tripwire};
use risingwave_connector::source::SplitMetaData;
use risingwave_pb::batch_plan::exchange_info::DistributionMode;
use risingwave_pb::batch_plan::exchange_source::LocalExecutePlan::Plan;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::{
ExchangeInfo, ExchangeSource, LocalExecutePlan, PlanFragment, PlanNode as PlanNodeProst,
TaskId as ProstTaskId, TaskOutputId,
};
use risingwave_pb::common::WorkerNode;
use tokio::sync::mpsc;
use tokio_stream::wrappers::ReceiverStream;
use tracing::debug;
use uuid::Uuid;
use super::plan_fragmenter::{PartitionInfo, QueryStage, QueryStageRef};
use crate::catalog::TableId;
use crate::optimizer::plan_node::PlanNodeType;
use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId};
use crate::scheduler::task_context::FrontendBatchTaskContext;
use crate::scheduler::{PinnedHummockSnapshot, SchedulerResult};
use crate::session::{AuthContext, FrontendEnv};
pub type LocalQueryStream = ReceiverStream<Result<DataChunk, BoxedError>>;
pub struct LocalQueryExecution {
sql: String,
query: Query,
front_env: FrontendEnv,
// The snapshot will be released when LocalQueryExecution is dropped.
snapshot: PinnedHummockSnapshot,
auth_context: Arc<AuthContext>,
cancel_flag: Option<Tripwire<Result<DataChunk, BoxedError>>>,
}
impl LocalQueryExecution {
pub fn new<S: Into<String>>(
query: Query,
front_env: FrontendEnv,
sql: S,
snapshot: PinnedHummockSnapshot,
auth_context: Arc<AuthContext>,
cancel_flag: Tripwire<Result<DataChunk, BoxedError>>,
) -> Self {
Self {
sql: sql.into(),
query,
front_env,
snapshot,
auth_context,
cancel_flag: Some(cancel_flag),
}
}
#[try_stream(ok = DataChunk, error = RwError)]
pub async fn run_inner(self) {
debug!(
"Starting to run query: {:?}, sql: '{}'",
self.query.query_id, self.sql
);
let context =
FrontendBatchTaskContext::new(self.front_env.clone(), self.auth_context.clone());
let task_id = TaskId {
query_id: self.query.query_id.id.clone(),
stage_id: 0,
task_id: 0,
};
let plan_fragment = self.create_plan_fragment()?;
let plan_node = plan_fragment.root.unwrap();
let executor = ExecutorBuilder::new(
&plan_node,
&task_id,
context,
self.snapshot.get_batch_query_epoch(),
);
let executor = executor.build().await?;
#[for_await]
for chunk in executor.execute() {
yield chunk?;
}
}
pub fn run(self) -> BoxedDataChunkStream {
Box::pin(self.run_inner())
}
pub fn stream_rows(mut self) -> LocalQueryStream {
let (sender, receiver) = mpsc::channel(10);
let tripwire = self.cancel_flag.take().unwrap();
let mut data_stream = {
let s = self.run().map(|r| r.map_err(|e| Box::new(e) as BoxedError));
Box::pin(cancellable_stream(s, tripwire))
};
let future = async move {
while let Some(r) = data_stream.next().await {
if (sender.send(r).await).is_err() {
tracing::info!("Receiver closed.");
}
}
};
#[cfg(madsim)]
tokio::spawn(future);
#[cfg(not(madsim))]
tokio::task::spawn_blocking(move || futures::executor::block_on(future));
ReceiverStream::new(receiver)
}
/// Convert query to plan fragment.
///
/// We can convert a query to plan fragment since in local execution mode, there are at most
/// two layers, e.g. root stage and its optional input stage. If it does have input stage, it
/// will be embedded in exchange source, so we can always convert a query into a plan fragment.
///
/// We remark that the boundary to determine which part should be executed on the frontend and
/// which part should be executed on the backend is `the first exchange operator` when looking
/// from the the root of the plan to the leaves. The first exchange operator contains
/// the pushed-down plan fragment.
fn create_plan_fragment(&self) -> SchedulerResult<PlanFragment> {
let root_stage_id = self.query.root_stage_id();
let root_stage = self.query.stage_graph.stages.get(&root_stage_id).unwrap();
assert_eq!(root_stage.parallelism.unwrap(), 1);
let second_stage_id = self.query.stage_graph.get_child_stages(&root_stage_id);
let plan_node_prost = match second_stage_id {
None => {
debug!("Local execution mode converts a plan with a single stage");
self.convert_plan_node(&root_stage.root, &mut None, None)?
}
Some(second_stage_ids) => {
debug!("Local execution mode converts a plan with two stages");
if second_stage_ids.is_empty() {
// This branch is defensive programming. The semantics should be the same as
// `None`.
self.convert_plan_node(&root_stage.root, &mut None, None)?
} else {
let mut second_stages = HashMap::new();
for second_stage_id in second_stage_ids {
let second_stage =
self.query.stage_graph.stages.get(second_stage_id).unwrap();
second_stages.insert(*second_stage_id, second_stage.clone());
}
let mut stage_id_to_plan = Some(second_stages);
let res =
self.convert_plan_node(&root_stage.root, &mut stage_id_to_plan, None)?;
assert!(
stage_id_to_plan.as_ref().unwrap().is_empty(),
"We expect that all the child stage plan fragments have been used"
);
res
}
}
};
Ok(PlanFragment {
root: Some(plan_node_prost),
// Intentionally leave this as `None` as this is the last stage for the frontend
// to really get the output of computation, which is single distribution
// but we do not need to explicitly specify this.
exchange_info: None,
})
}
fn convert_plan_node(
&self,
execution_plan_node: &ExecutionPlanNode,
second_stages: &mut Option<HashMap<StageId, QueryStageRef>>,
partition: Option<PartitionInfo>,
) -> SchedulerResult<PlanNodeProst> {
match execution_plan_node.plan_node_type {
PlanNodeType::BatchExchange => {
let exchange_source_stage_id = execution_plan_node
.source_stage_id
.expect("We expect stage id for Exchange Operator");
let Some(second_stages) = second_stages.as_mut() else {
bail!("Unexpected exchange detected. We are either converting a single stage plan or converting the second stage of the plan.")
};
let second_stage = second_stages.remove(&exchange_source_stage_id).expect(
"We expect child stage fragment for Exchange Operator running in the frontend",
);
let mut node_body = execution_plan_node.node.clone();
let sources = match &mut node_body {
NodeBody::Exchange(exchange_node) => &mut exchange_node.sources,
NodeBody::MergeSortExchange(merge_sort_exchange_node) => {
&mut merge_sort_exchange_node
.exchange
.as_mut()
.expect("MergeSortExchangeNode must have a exchange node")
.sources
}
_ => unreachable!(),
};
assert!(sources.is_empty());
if let Some(table_scan_info) = second_stage.table_scan_info.clone() && let Some(vnode_bitmaps) = table_scan_info.partitions() {
// Similar to the distributed case (StageRunner::schedule_tasks).
// Set `vnode_ranges` of the scan node in `local_execute_plan` of each
// `exchange_source`.
let (parallel_unit_ids, vnode_bitmaps): (Vec<_>, Vec<_>) =
vnode_bitmaps.clone().into_iter().unzip();
let workers = self.front_env.worker_node_manager().get_workers_by_parallel_unit_ids(¶llel_unit_ids)?;
for (idx, (worker_node, partition)) in
(workers.into_iter().zip_eq_fast(vnode_bitmaps.into_iter())).enumerate()
{
let second_stage_plan_node = self.convert_plan_node(
&second_stage.root,
&mut None,
Some(PartitionInfo::Table(partition)),
)?;
let second_stage_plan_fragment = PlanFragment {
root: Some(second_stage_plan_node),
exchange_info: Some(ExchangeInfo {
mode: DistributionMode::Single as i32,
..Default::default()
}),
};
let local_execute_plan = LocalExecutePlan {
plan: Some(second_stage_plan_fragment),
epoch: Some(self.snapshot.get_batch_query_epoch()),
};
let exchange_source = ExchangeSource {
task_output_id: Some(TaskOutputId {
task_id: Some(ProstTaskId {
task_id: idx as u32,
stage_id: exchange_source_stage_id,
query_id: self.query.query_id.id.clone(),
}),
output_id: 0,
}),
host: Some(worker_node.host.as_ref().unwrap().clone()),
local_execute_plan: Some(Plan(local_execute_plan)),
};
sources.push(exchange_source);
}
} else if let Some(source_info) = &second_stage.source_info {
for (id,split) in source_info.split_info().unwrap().iter().enumerate() {
let second_stage_plan_node = self.convert_plan_node(
&second_stage.root,
&mut None,
Some(PartitionInfo::Source(split.clone())),
)?;
let second_stage_plan_fragment = PlanFragment {
root: Some(second_stage_plan_node),
exchange_info: Some(ExchangeInfo {
mode: DistributionMode::Single as i32,
..Default::default()
}),
};
let local_execute_plan = LocalExecutePlan {
plan: Some(second_stage_plan_fragment),
epoch: Some(self.snapshot.get_batch_query_epoch()),
};
// NOTE: select a random work node here.
let worker_node = self.front_env.worker_node_manager().next_random()?;
let exchange_source = ExchangeSource {
task_output_id: Some(TaskOutputId {
task_id: Some(ProstTaskId {
task_id: id as u32,
stage_id: exchange_source_stage_id,
query_id: self.query.query_id.id.clone(),
}),
output_id: 0,
}),
host: Some(worker_node.host.as_ref().unwrap().clone()),
local_execute_plan: Some(Plan(local_execute_plan)),
};
sources.push(exchange_source);
}
}
else {
let second_stage_plan_node =
self.convert_plan_node(&second_stage.root, &mut None, None)?;
let second_stage_plan_fragment = PlanFragment {
root: Some(second_stage_plan_node),
exchange_info: Some(ExchangeInfo {
mode: DistributionMode::Single as i32,
..Default::default()
}),
};
let local_execute_plan = LocalExecutePlan {
plan: Some(second_stage_plan_fragment),
epoch: Some(self.snapshot.get_batch_query_epoch()),
};
let workers = self.choose_worker(&second_stage)?;
*sources = workers
.iter()
.enumerate()
.map(|(idx, worker_node)| {
let exchange_source = ExchangeSource {
task_output_id: Some(TaskOutputId {
task_id: Some(ProstTaskId {
task_id: idx as u32,
stage_id: exchange_source_stage_id,
query_id: self.query.query_id.id.clone(),
}),
output_id: 0,
}),
host: Some(worker_node.host.as_ref().unwrap().clone()),
local_execute_plan: Some(Plan(local_execute_plan.clone())),
};
exchange_source
})
.collect();
}
Ok(PlanNodeProst {
/// Since all the rest plan is embedded into the exchange node,
/// there is no children any more.
children: vec![],
identity: Uuid::new_v4().to_string(),
node_body: Some(node_body),
})
}
PlanNodeType::BatchSeqScan => {
let mut node_body = execution_plan_node.node.clone();
match &mut node_body {
NodeBody::RowSeqScan(ref mut scan_node) => {
if let Some(partition) = partition {
let partition = partition
.into_table()
.expect("PartitionInfo should be TablePartitionInfo here");
scan_node.vnode_bitmap = Some(partition.vnode_bitmap);
scan_node.scan_ranges = partition.scan_ranges;
}
}
NodeBody::SysRowSeqScan(_) => {}
_ => unreachable!(),
}
Ok(PlanNodeProst {
children: vec![],
// TODO: Generate meaningful identify
identity: Uuid::new_v4().to_string(),
node_body: Some(node_body),
})
}
PlanNodeType::BatchSource => {
let mut node_body = execution_plan_node.node.clone();
match &mut node_body {
NodeBody::Source(ref mut source_node) => {
if let Some(partition) = partition {
let partition = partition
.into_source()
.expect("PartitionInfo should be SourcePartitionInfo here");
source_node.split = partition.encode_to_bytes().into();
}
}
_ => unreachable!(),
}
Ok(PlanNodeProst {
children: vec![],
// TODO: Generate meaningful identify
identity: Uuid::new_v4().to_string(),
node_body: Some(node_body),
})
}
PlanNodeType::BatchLookupJoin => {
let mut node_body = execution_plan_node.node.clone();
match &mut node_body {
NodeBody::LocalLookupJoin(node) => {
let side_table_desc = node
.inner_side_table_desc
.as_ref()
.expect("no side table desc");
let table = self
.front_env
.catalog_reader()
.read_guard()
.get_table_by_id(&side_table_desc.table_id.into())
.context("side table not found")?;
let mapping = self
.front_env
.worker_node_manager()
.get_fragment_mapping(&table.fragment_id)
.context("fragment mapping not found")?;
// TODO: should we use `pb::ParallelUnitMapping` here?
node.inner_side_vnode_mapping = mapping.to_expanded();
node.worker_nodes =
self.front_env.worker_node_manager().list_worker_nodes();
}
_ => unreachable!(),
}
let left_child = self.convert_plan_node(
&execution_plan_node.children[0],
second_stages,
partition,
)?;
Ok(PlanNodeProst {
children: vec![left_child],
identity: Uuid::new_v4().to_string(),
node_body: Some(node_body),
})
}
_ => {
let children = execution_plan_node
.children
.iter()
.map(|e| self.convert_plan_node(e, second_stages, partition.clone()))
.collect::<SchedulerResult<Vec<PlanNodeProst>>>()?;
Ok(PlanNodeProst {
children,
// TODO: Generate meaningful identify
identity: Uuid::new_v4().to_string(),
node_body: Some(execution_plan_node.node.clone()),
})
}
}
}
#[inline(always)]
fn get_vnode_mapping(&self, table_id: &TableId) -> Option<ParallelUnitMapping> {
self.front_env
.catalog_reader()
.read_guard()
.get_table_by_id(table_id)
.map(|table| {
self.front_env
.worker_node_manager()
.get_fragment_mapping(&table.fragment_id)
})
.ok()
.flatten()
}
fn choose_worker(&self, stage: &Arc<QueryStage>) -> SchedulerResult<Vec<WorkerNode>> {
if stage.parallelism.unwrap() == 1 {
if let NodeBody::Insert(insert_node) = &stage.root.node
&& let Some(vnode_mapping) = self.get_vnode_mapping(&insert_node.table_id.into()) {
let worker_node = {
let parallel_unit_ids = vnode_mapping.iter_unique().collect_vec();
let candidates = self.front_env
.worker_node_manager()
.get_workers_by_parallel_unit_ids(¶llel_unit_ids)?;
candidates.choose(&mut rand::thread_rng()).unwrap().clone()
};
Ok(vec![worker_node])
} else {
Ok(vec![self.front_env.worker_node_manager().next_random()?])
}
} else {
let mut workers = Vec::with_capacity(stage.parallelism.unwrap() as usize);
for _ in 0..stage.parallelism.unwrap() {
workers.push(self.front_env.worker_node_manager().next_random()?);
}
Ok(workers)
}
}
}