risingwave_frontend/handler/
alter_parallelism.rs1use pgwire::pg_response::StatementType;
16use risingwave_pb::meta::table_parallelism::{
17 AdaptiveParallelism, FixedParallelism, PbParallelism,
18};
19use risingwave_pb::meta::{PbTableParallelism, TableParallelism};
20use risingwave_sqlparser::ast::{ObjectName, SetVariableValue, SetVariableValueSingle, Value};
21use risingwave_sqlparser::keywords::Keyword;
22use thiserror_ext::AsReport;
23
24use super::alter_utils::resolve_streaming_job_id_for_alter_parallelism;
25use super::{HandlerArgs, RwPgResponse};
26use crate::catalog::FragmentId;
27use crate::error::{ErrorCode, Result};
28use crate::handler::util::{LongRunningNotificationAction, execute_with_long_running_notification};
29
30pub async fn handle_alter_parallelism(
31 handler_args: HandlerArgs,
32 obj_name: ObjectName,
33 parallelism: SetVariableValue,
34 stmt_type: StatementType,
35 deferred: bool,
36) -> Result<RwPgResponse> {
37 let session = handler_args.session;
38
39 let job_id = resolve_streaming_job_id_for_alter_parallelism(
40 &session,
41 obj_name,
42 stmt_type,
43 "parallelism",
44 )?;
45
46 let target_parallelism = extract_table_parallelism(parallelism)?;
47
48 let mut builder = RwPgResponse::builder(stmt_type);
49
50 let catalog_writer = session.catalog_writer()?;
51 execute_with_long_running_notification(
52 catalog_writer.alter_parallelism(job_id, target_parallelism, deferred),
53 &session,
54 "ALTER PARALLELISM",
55 LongRunningNotificationAction::SuggestRecover,
56 )
57 .await?;
58
59 if deferred {
60 builder = builder.notice("DEFERRED is used, please ensure that automatic parallelism control is enabled on the meta, otherwise, the alter will not take effect.".to_owned());
61 }
62
63 Ok(builder.into())
64}
65
66pub async fn handle_alter_fragment_parallelism(
67 handler_args: HandlerArgs,
68 fragment_ids: Vec<FragmentId>,
69 parallelism: SetVariableValue,
70) -> Result<RwPgResponse> {
71 let session = handler_args.session;
72 let target_parallelism = extract_fragment_parallelism(parallelism)?;
73
74 session
75 .env()
76 .meta_client()
77 .alter_fragment_parallelism(fragment_ids, target_parallelism)
78 .await?;
79
80 Ok(RwPgResponse::builder(StatementType::ALTER_FRAGMENT).into())
81}
82
83fn extract_table_parallelism(parallelism: SetVariableValue) -> Result<TableParallelism> {
84 let adaptive_parallelism = PbTableParallelism {
85 parallelism: Some(PbParallelism::Adaptive(AdaptiveParallelism {})),
86 };
87
88 let target_parallelism = match parallelism {
90 SetVariableValue::Single(SetVariableValueSingle::Ident(ident))
91 if ident
92 .real_value()
93 .eq_ignore_ascii_case(&Keyword::ADAPTIVE.to_string()) =>
94 {
95 adaptive_parallelism
96 }
97
98 SetVariableValue::Default => adaptive_parallelism,
99 SetVariableValue::Single(SetVariableValueSingle::Literal(Value::Number(v))) => {
100 let fixed_parallelism = v.parse::<u32>().map_err(|e| {
101 ErrorCode::InvalidInputSyntax(format!(
102 "target parallelism must be a valid number or adaptive: {}",
103 e.as_report()
104 ))
105 })?;
106
107 if fixed_parallelism == 0 {
108 adaptive_parallelism
109 } else {
110 PbTableParallelism {
111 parallelism: Some(PbParallelism::Fixed(FixedParallelism {
112 parallelism: fixed_parallelism,
113 })),
114 }
115 }
116 }
117
118 _ => {
119 return Err(ErrorCode::InvalidInputSyntax(
120 "target parallelism must be a valid number or adaptive".to_owned(),
121 )
122 .into());
123 }
124 };
125
126 Ok(target_parallelism)
127}
128
129fn extract_fragment_parallelism(parallelism: SetVariableValue) -> Result<Option<TableParallelism>> {
130 match parallelism {
131 SetVariableValue::Default => Ok(None),
132 other => extract_table_parallelism(other).map(Some),
133 }
134}