Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix(frontend): only schedule dml to cn which contain table source writer #4329

Merged
merged 5 commits into from
Aug 1, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion src/frontend/src/binder/insert.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@

use itertools::Itertools;
use risingwave_common::error::{ErrorCode, Result};
use risingwave_common::types::DataType;
use risingwave_common::types::{DataType, ParallelUnitId};
use risingwave_sqlparser::ast::{Ident, ObjectName, Query, SetExpr};

use super::{BoundQuery, BoundSetExpr};
Expand All @@ -26,6 +26,9 @@ pub struct BoundInsert {
/// Used for injecting deletion chunks to the source.
pub table_source: BoundTableSource,

/// Used for scheduling.
pub vnode_mapping: Option<Vec<ParallelUnitId>>,

pub source: BoundQuery,

/// Used as part of an extra `Project` when the column types of `source` query does not match
Expand All @@ -40,6 +43,7 @@ impl Binder {
_columns: Vec<Ident>,
source: Query,
) -> Result<BoundInsert> {
let (schema_name, table_name) = Self::resolve_table_name(source_name.clone())?;
let table_source = self.bind_table_source(source_name)?;

let expected_types = table_source
Expand All @@ -48,6 +52,11 @@ impl Binder {
.map(|c| c.data_type.clone())
.collect();

let vnode_mapping = self
.bind_table(&schema_name, &table_name, None)?
.table_catalog
.vnode_mapping;
yezizp2012 marked this conversation as resolved.
Show resolved Hide resolved

// When the column types of `source` query does not match `expected_types`, casting is
// needed.
//
Expand Down Expand Up @@ -112,6 +121,7 @@ impl Binder {

let insert = BoundInsert {
table_source,
vnode_mapping,
source,
cast_exprs,
};
Expand Down
22 changes: 15 additions & 7 deletions src/frontend/src/binder/update.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ use std::collections::HashMap;
use itertools::Itertools;
use risingwave_common::ensure;
use risingwave_common::error::{ErrorCode, Result};
use risingwave_common::types::ParallelUnitId;
use risingwave_sqlparser::ast::{Assignment, Expr, TableFactor, TableWithJoins};

use super::{Binder, BoundTableSource, Relation};
Expand All @@ -29,6 +30,9 @@ pub struct BoundUpdate {
/// Used for injecting new chunks to the source.
pub table_source: BoundTableSource,

/// Used for scheduling.
pub vnode_mapping: Option<Vec<ParallelUnitId>>,

/// Used for scanning the records to update with the `selection`.
pub table: Relation,

Expand All @@ -46,21 +50,24 @@ impl Binder {
assignments: Vec<Assignment>,
selection: Option<Expr>,
) -> Result<BoundUpdate> {
let table_source = {
ensure!(table.joins.is_empty());
let name = match &table.relation {
TableFactor::Table { name, .. } => name.clone(),
_ => unreachable!(),
};
self.bind_table_source(name)?
ensure!(table.joins.is_empty());
let source_name = match &table.relation {
TableFactor::Table { name, .. } => name.clone(),
_ => unreachable!(),
};
let (schema_name, table_name) = Self::resolve_table_name(source_name.clone())?;
let table_source = self.bind_table_source(source_name)?;

if table_source.append_only {
return Err(ErrorCode::BindError(
"Append-only table source doesn't support update".to_string(),
)
.into());
}
let vnode_mapping = self
.bind_table(&schema_name, &table_name, None)?
.table_catalog
.vnode_mapping;

let table = self.bind_vec_table_with_joins(vec![table])?.unwrap();
assert_matches!(table, Relation::BaseTable(_));
Expand Down Expand Up @@ -123,6 +130,7 @@ impl Binder {

Ok(BoundUpdate {
table_source,
vnode_mapping,
table,
selection,
exprs,
Expand Down
11 changes: 9 additions & 2 deletions src/frontend/src/handler/dml.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::error::Result;
use risingwave_sqlparser::ast::Statement;

use crate::binder::Binder;
use crate::binder::{Binder, BoundStatement};
use crate::handler::privilege::{check_privileges, resolve_privileges};
use crate::handler::util::{to_pg_field, to_pg_rows};
use crate::planner::Planner;
Expand All @@ -39,6 +39,13 @@ pub async fn handle_dml(context: OptimizerContext, stmt: Statement) -> Result<Pg
let check_items = resolve_privileges(&bound);
check_privileges(&session, &check_items)?;

let vnodes = match &bound {
BoundStatement::Insert(insert) => insert.vnode_mapping.clone(),
BoundStatement::Update(update) => update.vnode_mapping.clone(),
BoundStatement::Delete(delete) => delete.table.table_catalog.vnode_mapping.clone(),
BoundStatement::Query(_) => unreachable!(),
};

let (plan, pg_descs) = {
// Subblock to make sure PlanRef (an Rc) is dropped before `await` below.
let root = Planner::new(context.into()).plan(bound)?;
Expand All @@ -54,7 +61,7 @@ pub async fn handle_dml(context: OptimizerContext, stmt: Statement) -> Result<Pg
let mut rows = vec![];
#[for_await]
for chunk in query_manager
.schedule_single(execution_context, plan)
.schedule_single(execution_context, plan, vnodes)
.await?
{
rows.extend(to_pg_rows(chunk?, false));
Expand Down
21 changes: 20 additions & 1 deletion src/frontend/src/scheduler/distributed/query_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,10 @@ use std::fmt::{Debug, Formatter};
use futures::StreamExt;
use futures_async_stream::try_stream;
use log::debug;
use rand::distributions::{Distribution as RandDistribution, Uniform};
use risingwave_common::array::DataChunk;
use risingwave_common::error::RwError;
use risingwave_common::types::ParallelUnitId;
use risingwave_pb::batch_plan::exchange_info::DistributionMode;
use risingwave_pb::batch_plan::{
ExchangeInfo, PlanFragment, PlanNode as BatchPlanProst, TaskId, TaskOutputId,
Expand Down Expand Up @@ -72,8 +74,25 @@ impl QueryManager {
&self,
_context: ExecutionContextRef,
plan: BatchPlanProst,
vnodes: Option<Vec<ParallelUnitId>>,
) -> SchedulerResult<impl DataChunkStream> {
let worker_node_addr = self.worker_node_manager.next_random()?.host.unwrap();
let worker_node_addr = match vnodes {
Some(mut parallel_unit_ids) => {
parallel_unit_ids.dedup();
let candidates = self
.worker_node_manager
.get_workers_by_parallel_unit_ids(&parallel_unit_ids)?;
let mut rng = rand::thread_rng();
let die = Uniform::from(0..candidates.len());
candidates
.get(die.sample(&mut rng))
.unwrap()
.clone()
.host
.unwrap()
yezizp2012 marked this conversation as resolved.
Show resolved Hide resolved
}
None => self.worker_node_manager.next_random()?.host.unwrap(),
};

let compute_client = self
.compute_client_pool
Expand Down