From 8221d3a16cec7122504e49423ae74a400d902ff1 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 25 Oct 2023 11:55:34 +0800 Subject: [PATCH] feat: add `WAIT` command (#13027) --- proto/ddl_service.proto | 5 +++++ src/frontend/src/handler/mod.rs | 2 ++ src/frontend/src/handler/wait.rs | 31 +++++++++++++++++++++++++++++ src/frontend/src/meta_client.rs | 6 ++++++ src/frontend/src/test_utils.rs | 4 ++++ src/meta/service/src/ddl_service.rs | 5 +++++ src/meta/src/rpc/ddl_controller.rs | 16 +++++++++++++++ src/rpc_client/src/meta_client.rs | 7 +++++++ src/sqlparser/src/ast/mod.rs | 6 ++++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 1 + src/utils/pgwire/src/pg_response.rs | 2 ++ 12 files changed, 86 insertions(+) create mode 100644 src/frontend/src/handler/wait.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 27c9f2ee82f83..1efc933a7d033 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -314,6 +314,10 @@ message GetTablesResponse { map tables = 1; } +message WaitRequest {} + +message WaitResponse {} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -343,4 +347,5 @@ service DdlService { rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); + rpc Wait(WaitRequest) returns (WaitResponse); } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 149f39bead330..174ed23e03ec5 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -73,6 +73,7 @@ mod show; mod transaction; pub mod util; pub mod variable; +mod wait; /// The [`PgResponseBuilder`] used by RisingWave. pub type RwPgResponseBuilder = PgResponseBuilder; @@ -419,6 +420,7 @@ pub async fn handle( } } Statement::Flush => flush::handle_flush(handler_args).await, + Statement::Wait => wait::handle_wait(handler_args).await, Statement::SetVariable { local: _, variable, diff --git a/src/frontend/src/handler/wait.rs b/src/frontend/src/handler/wait.rs new file mode 100644 index 0000000000000..83f2784ec8c17 --- /dev/null +++ b/src/frontend/src/handler/wait.rs @@ -0,0 +1,31 @@ +// 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. + +use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::Result; + +use super::RwPgResponse; +use crate::handler::HandlerArgs; +use crate::session::SessionImpl; + +pub(super) async fn handle_wait(handler_args: HandlerArgs) -> Result { + do_wait(&handler_args.session).await?; + Ok(PgResponse::empty_result(StatementType::WAIT)) +} + +pub(crate) async fn do_wait(session: &SessionImpl) -> Result<()> { + let client = session.env().meta_client(); + client.wait().await?; + Ok(()) +} diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index ae90c2e345f9f..d37c5dec127f1 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -43,6 +43,8 @@ pub trait FrontendMetaClient: Send + Sync { async fn flush(&self, checkpoint: bool) -> Result; + async fn wait(&self) -> Result<()>; + async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result>; async fn list_table_fragments( @@ -111,6 +113,10 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0.flush(checkpoint).await } + async fn wait(&self) -> Result<()> { + self.0.wait().await + } + async fn cancel_creating_jobs(&self, infos: PbJobs) -> Result> { self.0.cancel_creating_jobs(infos).await } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 20eb252fc5053..cf915ae35713d 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -773,6 +773,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { }) } + async fn wait(&self) -> RpcResult<()> { + Ok(()) + } + async fn cancel_creating_jobs(&self, _infos: PbJobs) -> RpcResult> { Ok(vec![]) } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 2fa5f50e15666..061ff93589163 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -732,6 +732,11 @@ impl DdlService for DdlServiceImpl { } Ok(Response::new(GetTablesResponse { tables })) } + + async fn wait(&self, _request: Request) -> Result, Status> { + self.ddl_controller.wait().await; + Ok(Response::new(WaitResponse {})) + } } impl DdlServiceImpl { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 04b9729c5a5b8..36615bd93b757 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -15,6 +15,7 @@ use std::cmp::Ordering; use std::num::NonZeroUsize; use std::sync::Arc; +use std::time::Duration; use itertools::Itertools; use risingwave_common::config::DefaultParallelism; @@ -29,6 +30,7 @@ use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; use tokio::sync::Semaphore; +use tokio::time::sleep; use tracing::log::warn; use tracing::Instrument; @@ -1094,4 +1096,18 @@ impl DdlController { } } } + + pub async fn wait(&self) { + for _ in 0..30 * 60 { + if self + .catalog_manager + .list_creating_background_mvs() + .await + .is_empty() + { + break; + } + sleep(Duration::from_secs(1)).await; + } + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 827860d1af7b3..95b746ea33e6c 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -698,6 +698,12 @@ impl MetaClient { Ok(resp.snapshot.unwrap()) } + pub async fn wait(&self) -> Result<()> { + let request = WaitRequest {}; + self.inner.wait(request).await?; + Ok(()) + } + pub async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result> { let request = CancelCreatingJobsRequest { jobs: Some(jobs) }; let resp = self.inner.cancel_creating_jobs(request).await?; @@ -1719,6 +1725,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } + ,{ ddl_client, wait, WaitRequest, WaitResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index ecae5a9663a88..5d802bae99cdc 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1294,6 +1294,9 @@ pub enum Statement { /// /// Note: RisingWave specific statement. Flush, + /// WAIT for ALL running stream jobs to finish. + /// It will block the current session the condition is met. + Wait, } impl fmt::Display for Statement { @@ -1787,6 +1790,9 @@ impl fmt::Display for Statement { Statement::Flush => { write!(f, "FLUSH") } + Statement::Wait => { + write!(f, "WAIT") + } Statement::Begin { modes } => { write!(f, "BEGIN")?; if !modes.is_empty() { diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 5c2fedb0ea547..4188f06f76ae3 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -540,6 +540,7 @@ define_keywords!( VIEWS, VIRTUAL, VOLATILE, + WAIT, WATERMARK, WHEN, WHENEVER, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index ee054f7d17031..5cc094a204268 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -259,6 +259,7 @@ impl Parser { Keyword::PREPARE => Ok(self.parse_prepare()?), Keyword::COMMENT => Ok(self.parse_comment()?), Keyword::FLUSH => Ok(Statement::Flush), + Keyword::WAIT => Ok(Statement::Wait), _ => self.expected( "an SQL statement", Token::Word(w).with_location(token.location), diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 29ea77f83b71b..eeec929732f50 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -92,6 +92,7 @@ pub enum StatementType { ROLLBACK, SET_TRANSACTION, CANCEL_COMMAND, + WAIT, } impl std::fmt::Display for StatementType { @@ -278,6 +279,7 @@ impl StatementType { }, Statement::Explain { .. } => Ok(StatementType::EXPLAIN), Statement::Flush => Ok(StatementType::FLUSH), + Statement::Wait => Ok(StatementType::WAIT), _ => Err("unsupported statement type".to_string()), } }