From a06cf13548857bf26330278ea1df16d7f4c006a3 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 17 Dec 2024 11:16:25 -0500 Subject: [PATCH 01/27] draft version --- proto/batch_plan.proto | 19 ++ proto/task_service.proto | 11 + src/batch/executors/src/executor.rs | 2 + .../executors/src/executor/fast_insert.rs | 289 ++++++++++++++++++ src/batch/src/rpc/service/task_service.rs | 52 +++- src/dml/src/table.rs | 53 +++- src/dml/src/txn_channel.rs | 18 +- src/frontend/src/handler/create_table.rs | 10 +- src/frontend/src/handler/fast_insert.rs | 95 ++++++ src/frontend/src/handler/mod.rs | 1 + src/frontend/src/handler/query.rs | 2 + .../src/scheduler/distributed/fast_insert.rs | 247 +++++++++++++++ .../src/scheduler/distributed/query.rs | 5 + src/frontend/src/scheduler/fast_insert.rs | 134 ++++++++ src/frontend/src/scheduler/local.rs | 4 + src/frontend/src/scheduler/mod.rs | 5 +- src/frontend/src/scheduler/plan_fragmenter.rs | 1 + src/frontend/src/webhook/mod.rs | 165 ++++++---- src/rpc_client/src/compute_client.rs | 14 +- src/stream/src/executor/dml.rs | 29 +- 20 files changed, 1064 insertions(+), 92 deletions(-) create mode 100644 src/batch/executors/src/executor/fast_insert.rs create mode 100644 src/frontend/src/handler/fast_insert.rs create mode 100644 src/frontend/src/scheduler/distributed/fast_insert.rs create mode 100644 src/frontend/src/scheduler/fast_insert.rs diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 6e07ceae4d5d4..bfde254f4c770 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -195,6 +195,25 @@ message InsertNode { uint32 session_id = 7; } +message FastInsertNode { + // Id of the table to perform inserting. + uint32 table_id = 1; + // Version of the table. + uint64 table_version_id = 2; + repeated uint32 column_indices = 3; + data.DataChunk data_chunk = 4; + + plan_common.DefaultColumns default_columns = 5; + // An optional field and will be `None` for tables without user-defined pk. + // The `BatchInsertExecutor` should add a column with NULL value which will + // be filled in streaming. + optional uint32 row_id_index = 6; + + // Session id is used to ensure that dml data from the same session should be + // sent to a fixed worker node and channel. + uint32 session_id = 7; +} + message DeleteNode { // Id of the table to perform deleting. uint32 table_id = 1; diff --git a/proto/task_service.proto b/proto/task_service.proto index cb14ee809d943..1d98865e2269a 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -44,6 +44,16 @@ message CreateTaskRequest { plan_common.ExprContext expr_context = 5; } +message FastInsertRequest { + batch_plan.FastInsertNode fast_insert_node = 2; +} + + +message FastInsertResponse { + // Optional error message for failed task. + string error_message = 1; +} + message CancelTaskRequest { batch_plan.TaskId task_id = 1; } @@ -73,6 +83,7 @@ service TaskService { // Cancel an already-died (self execution-failure, previous aborted, completed) task will still succeed. rpc CancelTask(CancelTaskRequest) returns (CancelTaskResponse); rpc Execute(ExecuteRequest) returns (stream GetDataResponse); + rpc FastInsert(FastInsertRequest) returns (FastInsertResponse); } message GetDataRequest { diff --git a/src/batch/executors/src/executor.rs b/src/batch/executors/src/executor.rs index a2b67f06924ae..80f29ecfd35b5 100644 --- a/src/batch/executors/src/executor.rs +++ b/src/batch/executors/src/executor.rs @@ -18,6 +18,7 @@ pub mod aggregation; mod azblob_file_scan; mod delete; mod expand; +mod fast_insert; mod filter; mod gcs_file_scan; mod generic_exchange; @@ -58,6 +59,7 @@ pub use filter::*; use gcs_file_scan::GcsFileScanExecutorBuilder; pub use generic_exchange::*; pub use group_top_n::*; +pub use fast_insert::*; pub use hash_agg::*; pub use hop_window::*; pub use iceberg_scan::*; diff --git a/src/batch/executors/src/executor/fast_insert.rs b/src/batch/executors/src/executor/fast_insert.rs new file mode 100644 index 0000000000000..b48c3df6b7140 --- /dev/null +++ b/src/batch/executors/src/executor/fast_insert.rs @@ -0,0 +1,289 @@ +// Copyright 2024 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 std::iter::repeat; +use std::sync::Arc; + +use anyhow::Context; +use itertools::Itertools; +use risingwave_common::array::{DataChunk, Op, SerialArray, StreamChunk}; +use risingwave_common::catalog::{Field, Schema, TableId, TableVersionId}; +use risingwave_common::transaction::transaction_id::TxnId; +use risingwave_common::types::DataType; +use risingwave_common::util::epoch::Epoch; +use risingwave_dml::dml_manager::DmlManagerRef; +use risingwave_expr::expr::{build_from_prost, BoxedExpression}; +use risingwave_pb::batch_plan::FastInsertNode; +use risingwave_pb::plan_common::IndexAndExpr; + +use crate::error::Result; + +pub struct FastInsertExecutor { + /// Target table id. + table_id: TableId, + table_version_id: TableVersionId, + dml_manager: DmlManagerRef, + // TODO(Kexiang): get rid of it? + input_schema: Schema, + column_indices: Vec, + + // TODO(Kexiang): get rid of it? + sorted_default_columns: Vec<(usize, BoxedExpression)>, + + row_id_index: Option, + txn_id: TxnId, + session_id: u32, +} + +impl FastInsertExecutor { + pub fn build( + dml_manager: DmlManagerRef, + insert_node: FastInsertNode, + ) -> Result<(FastInsertExecutor, DataChunk)> { + let table_id = TableId::new(insert_node.table_id); + let column_indices = insert_node + .column_indices + .iter() + .map(|&i| i as usize) + .collect(); + let sorted_default_columns = if let Some(default_columns) = &insert_node.default_columns { + let mut default_columns = default_columns + .get_default_columns() + .iter() + .cloned() + .map(|IndexAndExpr { index: i, expr: e }| { + Ok(( + i as usize, + build_from_prost(&e.context("expression is None")?) + .context("failed to build expression")?, + )) + }) + .collect::>>()?; + default_columns.sort_unstable_by_key(|(i, _)| *i); + default_columns + } else { + vec![] + }; + let mut schema = Schema::new(vec![Field::unnamed(DataType::Jsonb)]); + schema.fields.push(Field::unnamed(DataType::Serial)); // row_id column + let data_chunk_pb = insert_node + .data_chunk + .expect("no data_chunk found in fast insert node"); + + Ok(( + FastInsertExecutor::new( + table_id, + insert_node.table_version_id, + dml_manager, + schema, + column_indices, + sorted_default_columns, + insert_node.row_id_index.as_ref().map(|index| *index as _), + insert_node.session_id, + ), + DataChunk::from_protobuf(&data_chunk_pb)?, + )) + } + + #[allow(clippy::too_many_arguments)] + pub fn new( + table_id: TableId, + table_version_id: TableVersionId, + dml_manager: DmlManagerRef, + input_schema: Schema, + column_indices: Vec, + sorted_default_columns: Vec<(usize, BoxedExpression)>, + row_id_index: Option, + session_id: u32, + ) -> Self { + let txn_id = dml_manager.gen_txn_id(); + Self { + table_id, + table_version_id, + dml_manager, + input_schema, + column_indices, + sorted_default_columns, + row_id_index, + txn_id, + session_id, + } + } +} + +impl FastInsertExecutor { + pub async fn do_execute(self, data_chunk_to_insert: DataChunk) -> Result { + let table_dml_handle = self + .dml_manager + .table_dml_handle(self.table_id, self.table_version_id)?; + let mut write_handle = table_dml_handle.write_handle(self.session_id, self.txn_id)?; + + write_handle.begin()?; + + // Transform the data chunk to a stream chunk, then write to the source. + // Return the returning chunk. + let write_txn_data = |chunk: DataChunk| async { + let cap = chunk.capacity(); + let (mut columns, vis) = chunk.into_parts(); + + let mut ordered_columns = self + .column_indices + .iter() + .enumerate() + .map(|(i, idx)| (*idx, columns[i].clone())) + .collect_vec(); + ordered_columns.reserve(ordered_columns.len() + self.sorted_default_columns.len()); + + ordered_columns.sort_unstable_by_key(|(idx, _)| *idx); + columns = ordered_columns + .into_iter() + .map(|(_, column)| column) + .collect_vec(); + + // If the user does not specify the primary key, then we need to add a column as the + // primary key. + if let Some(row_id_index) = self.row_id_index { + let row_id_col = SerialArray::from_iter(repeat(None).take(cap)); + columns.insert(row_id_index, Arc::new(row_id_col.into())) + } + + let stream_chunk = StreamChunk::with_visibility(vec![Op::Insert; cap], columns, vis); + + #[cfg(debug_assertions)] + table_dml_handle.check_chunk_schema(&stream_chunk); + + write_handle.write_chunk(stream_chunk).await?; + + Result::Ok(()) + }; + write_txn_data(data_chunk_to_insert).await?; + let epoch = write_handle.end_returning_epoch().await?; + return Ok(epoch); + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::ops::Bound; + + use assert_matches::assert_matches; + use foyer::CacheHint; + use futures::StreamExt; + use risingwave_common::array::{Array, JsonbArrayBuilder}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, INITIAL_TABLE_VERSION_ID}; + use risingwave_common::transaction::transaction_message::TxnMsg; + use risingwave_common::types::JsonbVal; + use risingwave_dml::dml_manager::DmlManager; + use risingwave_storage::hummock::CachePolicy; + use risingwave_storage::memory::MemoryStateStore; + use risingwave_storage::store::{ReadOptions, StateStoreReadExt}; + use serde_json::json; + + use super::*; + use crate::risingwave_common::types::Scalar; + use crate::*; + + #[tokio::test] + async fn test_insert_executor() -> Result<()> { + let dml_manager = Arc::new(DmlManager::for_test()); + let store = MemoryStateStore::new(); + // Schema of the table + let mut schema = Schema::new(vec![Field::unnamed(DataType::Jsonb)]); + schema.fields.push(Field::unnamed(DataType::Serial)); // row_id column + + let row_id_index = Some(1); + + let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); + + let mut header_map = HashMap::new(); + header_map.insert("data".to_string(), "value1".to_string()); + + let json_value = json!(header_map); + let jsonb_val = JsonbVal::from(json_value); + // Add 4 ListValues to ArrayBuilder + builder.append(Some(jsonb_val.as_scalar_ref())); + + // Use builder to obtain a single (List) column DataChunk + let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); + + // Create the table. + let table_id = TableId::new(0); + + // Create reader + let column_descs = schema + .fields + .iter() + .enumerate() + .map(|(i, field)| ColumnDesc::unnamed(ColumnId::new(i as _), field.data_type.clone())) + .collect_vec(); + // We must create a variable to hold this `Arc` here, or it will be dropped + // due to the `Weak` reference in `DmlManager`. + let reader = dml_manager + .register_reader(table_id, INITIAL_TABLE_VERSION_ID, &column_descs) + .unwrap(); + let mut reader = reader.stream_reader().into_stream(); + + // Insert + let insert_executor = Box::new(FastInsertExecutor::new( + table_id, + INITIAL_TABLE_VERSION_ID, + dml_manager, + schema, + vec![0], // Ignoring insertion order + vec![], + row_id_index, + 0, + )); + let handle = tokio::spawn(async move { + insert_executor.do_execute(data_chunk).await.unwrap(); + }); + + // Read + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), _)); + + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Data(_, chunk),_) => { + // assert_eq!( + // chunk.columns()[0].as_int32().iter().collect::>(), + // vec![Some(1), Some(3), Some(5), Some(7), Some(9)] + // ); + + assert_eq!(chunk.columns().len(),2); + let array = chunk.columns()[0].as_jsonb().iter().collect::>(); + println!("WKXLOG arrar: {:?}",array); + // assert_eq!(*array,2); + // assert_eq!(*chunk.columns()[2], array); + }); + + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::End(_), _)); + let epoch = u64::MAX; + let full_range = (Bound::Unbounded, Bound::Unbounded); + let store_content = store + .scan( + full_range, + epoch, + None, + ReadOptions { + cache_policy: CachePolicy::Fill(CacheHint::Normal), + ..Default::default() + }, + ) + .await?; + assert!(store_content.is_empty()); + + handle.await.unwrap(); + + Ok(()) + } +} diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index 7c601cf1cb031..bb28f57377d4e 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -15,16 +15,19 @@ use std::sync::Arc; use risingwave_common::util::tracing::TracingContext; -use risingwave_pb::batch_plan::TaskOutputId; +use risingwave_pb::batch_plan::{FastInsertNode, TaskOutputId}; use risingwave_pb::task_service::task_service_server::TaskService; use risingwave_pb::task_service::{ - CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, GetDataResponse, - TaskInfoResponse, + CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, FastInsertRequest, + FastInsertResponse, GetDataResponse, TaskInfoResponse, }; +use risingwave_storage::dispatch_state_store; use thiserror_ext::AsReport; use tokio_stream::wrappers::ReceiverStream; use tonic::{Request, Response, Status}; +use crate::error::BatchError; +use crate::executor::FastInsertExecutor; use crate::rpc::service::exchange::GrpcExchangeWriter; use crate::task::{ BatchEnvironment, BatchManager, BatchTaskExecution, ComputeNodeContext, StateReporter, @@ -118,6 +121,24 @@ impl TaskService for BatchServiceImpl { let mgr = self.mgr.clone(); BatchServiceImpl::get_execute_stream(env, mgr, req).await } + + #[cfg_attr(coverage, coverage(off))] + async fn fast_insert( + &self, + request: Request, + ) -> Result, Status> { + let insert_node = request + .into_inner() + .fast_insert_node + .expect("no fast insert node found"); + let res = self.do_fast_insert(insert_node).await; + match res { + Ok(_) => Ok(Response::new(FastInsertResponse { + error_message: String::from("success"), + })), + Err(e) => Err(e.into()), + } + } } impl BatchServiceImpl { @@ -185,4 +206,29 @@ impl BatchServiceImpl { }); Ok(Response::new(ReceiverStream::new(rx))) } + + async fn do_fast_insert(&self, insert_node: FastInsertNode) -> Result<(), BatchError> { + let table_id = insert_node.table_id; + let (executor, data_chunk) = + FastInsertExecutor::build(self.env.dml_manager_ref(), insert_node)?; + let epoch = executor.do_execute(data_chunk).await?; + + dispatch_state_store!(self.env.state_store(), store, { + use risingwave_common::catalog::TableId; + use risingwave_hummock_sdk::HummockReadEpoch; + use risingwave_storage::store::TryWaitEpochOptions; + use risingwave_storage::StateStore; + + store + .try_wait_epoch( + HummockReadEpoch::Committed(epoch.0), + TryWaitEpochOptions { + table_id: TableId::new(table_id), + }, + ) + .await + .map_err(BatchError::from)?; + }); + Ok(()) + } } diff --git a/src/dml/src/table.rs b/src/dml/src/table.rs index 0b9c9d398296c..20d6754d4d80b 100644 --- a/src/dml/src/table.rs +++ b/src/dml/src/table.rs @@ -20,6 +20,7 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnDesc; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; +use risingwave_common::util::epoch::Epoch; use tokio::sync::oneshot; use crate::error::{DmlError, Result}; @@ -194,6 +195,17 @@ impl WriteHandle { Ok(()) } + pub async fn end_returning_epoch(mut self) -> Result { + assert_eq!(self.txn_state, TxnState::Begin); + self.txn_state = TxnState::Committed; + // Await the notifier. + let (notifier, epoch_notifier) = + self.write_txn_control_msg_returning_epoch(TxnMsg::End(self.txn_id))?; + notifier.await.map_err(|_| DmlError::ReaderClosed)?; + let epoch = epoch_notifier.await.map_err(|_| DmlError::ReaderClosed)?; + Ok(epoch) + } + pub fn rollback(mut self) -> Result> { self.rollback_inner() } @@ -212,7 +224,7 @@ impl WriteHandle { async fn write_txn_data_msg(&self, txn_msg: TxnMsg) -> Result> { assert_eq!(self.txn_id, txn_msg.txn_id()); let (notifier_tx, notifier_rx) = oneshot::channel(); - match self.tx.send(txn_msg, notifier_tx).await { + match self.tx.send(txn_msg, notifier_tx, None).await { Ok(_) => Ok(notifier_rx), // It's possible that the source executor is scaled in or migrated, so the channel @@ -226,7 +238,7 @@ impl WriteHandle { fn write_txn_control_msg(&self, txn_msg: TxnMsg) -> Result> { assert_eq!(self.txn_id, txn_msg.txn_id()); let (notifier_tx, notifier_rx) = oneshot::channel(); - match self.tx.send_immediate(txn_msg, notifier_tx) { + match self.tx.send_immediate(txn_msg, notifier_tx, None) { Ok(_) => Ok(notifier_rx), // It's possible that the source executor is scaled in or migrated, so the channel @@ -234,6 +246,25 @@ impl WriteHandle { Err(_) => Err(DmlError::ReaderClosed), } } + + fn write_txn_control_msg_returning_epoch( + &self, + txn_msg: TxnMsg, + ) -> Result<(oneshot::Receiver, oneshot::Receiver)> { + assert_eq!(self.txn_id, txn_msg.txn_id()); + let (notifier_tx, notifier_rx) = oneshot::channel(); + let (epoch_notifier_tx, epoch_notifier_rx) = oneshot::channel(); + match self + .tx + .send_immediate(txn_msg, notifier_tx, Some(epoch_notifier_tx)) + { + Ok(_) => Ok((notifier_rx, epoch_notifier_rx)), + + // It's possible that the source executor is scaled in or migrated, so the channel + // is closed. To guarantee the transactional atomicity, bail out. + Err(_) => Err(DmlError::ReaderClosed), + } + } } /// [`TableStreamReader`] reads changes from a certain table continuously. @@ -249,7 +280,7 @@ pub struct TableStreamReader { impl TableStreamReader { #[try_stream(boxed, ok = StreamChunk, error = DmlError)] pub async fn into_data_stream_for_test(mut self) { - while let Some((txn_msg, notifier)) = self.rx.recv().await { + while let Some((txn_msg, notifier, _epoch_notifier)) = self.rx.recv().await { // Notify about that we've taken the chunk. match txn_msg { TxnMsg::Begin(_) | TxnMsg::End(_) | TxnMsg::Rollback(_) => { @@ -263,18 +294,18 @@ impl TableStreamReader { } } - #[try_stream(boxed, ok = TxnMsg, error = DmlError)] + #[try_stream(boxed, ok = (TxnMsg, Option>), error = DmlError)] pub async fn into_stream(mut self) { - while let Some((txn_msg, notifier)) = self.rx.recv().await { + while let Some((txn_msg, notifier, epoch_notifier)) = self.rx.recv().await { // Notify about that we've taken the chunk. match &txn_msg { TxnMsg::Begin(_) | TxnMsg::End(_) | TxnMsg::Rollback(_) => { _ = notifier.send(0); - yield txn_msg; + yield (txn_msg, epoch_notifier); } TxnMsg::Data(_, chunk) => { _ = notifier.send(chunk.cardinality()); - yield txn_msg; + yield (txn_msg, epoch_notifier); } } } @@ -311,7 +342,7 @@ mod tests { .unwrap(); write_handle.begin().unwrap(); - assert_matches!(reader.next().await.unwrap()?, TxnMsg::Begin(_)); + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), None)); macro_rules! write_chunk { ($i:expr) => {{ @@ -343,7 +374,7 @@ mod tests { write_handle.end().await.unwrap(); }); - assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(_)); + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::End(_), None)); Ok(()) } @@ -357,7 +388,7 @@ mod tests { .unwrap(); write_handle.begin().unwrap(); - assert_matches!(reader.next().await.unwrap()?, TxnMsg::Begin(_)); + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), None)); let chunk = StreamChunk::new(vec![Op::Insert], vec![I64Array::from_iter([1]).into_ref()]); write_handle.write_chunk(chunk).await.unwrap(); @@ -369,7 +400,7 @@ mod tests { // Rollback on drop drop(write_handle); - assert_matches!(reader.next().await.unwrap()?, TxnMsg::Rollback(_)); + assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Rollback(_), None)); Ok(()) } diff --git a/src/dml/src/txn_channel.rs b/src/dml/src/txn_channel.rs index 35b8abf73e170..fbcb3da067ec5 100644 --- a/src/dml/src/txn_channel.rs +++ b/src/dml/src/txn_channel.rs @@ -16,6 +16,7 @@ use std::sync::Arc; use futures::FutureExt; use risingwave_common::transaction::transaction_message::TxnMsg; +use risingwave_common::util::epoch::Epoch; use tokio::sync::{mpsc, oneshot, Semaphore}; pub struct PermitValue(u32); @@ -23,6 +24,7 @@ pub struct PermitValue(u32); pub struct TxnMsgWithPermits { pub txn_msg: TxnMsg, pub notificator: oneshot::Sender, + pub epoch_notificator: Option>, pub permit_value: Option, } @@ -80,6 +82,7 @@ impl Sender { &self, txn_msg: TxnMsg, notificator: oneshot::Sender, + epoch_notificator: Option>, ) -> Result<(), mpsc::error::SendError> { // The semaphores should never be closed. let permits = match &txn_msg { @@ -106,6 +109,7 @@ impl Sender { .send(TxnMsgWithPermits { txn_msg, notificator, + epoch_notificator, permit_value: permits, }) .map_err(|e| mpsc::error::SendError(e.0.txn_msg)) @@ -119,8 +123,9 @@ impl Sender { &self, txn_msg: TxnMsg, notificator: oneshot::Sender, + epoch_notificator: Option>, ) -> Result<(), mpsc::error::SendError> { - self.send(txn_msg, notificator) + self.send(txn_msg, notificator, epoch_notificator) .now_or_never() .expect("cannot send immediately") } @@ -141,10 +146,17 @@ impl Receiver { /// Receive the next message for this receiver, with the permits of this message added back. /// /// Returns `None` if the channel has been closed. - pub async fn recv(&mut self) -> Option<(TxnMsg, oneshot::Sender)> { + pub async fn recv( + &mut self, + ) -> Option<( + TxnMsg, + oneshot::Sender, + Option>, + )> { let TxnMsgWithPermits { txn_msg, notificator, + epoch_notificator, permit_value: permits, } = self.rx.recv().await?; @@ -152,6 +164,6 @@ impl Receiver { self.permits.add_permits(permits); } - Some((txn_msg, notificator)) + Some((txn_msg, notificator, epoch_notificator)) } } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 4c08583deb39b..eefe43ff41a86 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -2027,14 +2027,10 @@ fn bind_webhook_info( let db_name = &session.database(); let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, secret_ref.secret_name.clone())?; - let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?; + // let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?; let pb_secret_ref = PbSecretRef { - secret_id: secret_catalog.id.secret_id(), - ref_as: match secret_ref.ref_as { - SecretRefAsType::Text => PbRefAsType::Text, - SecretRefAsType::File => PbRefAsType::File, - } - .into(), + secret_id: 0, + ref_as: PbRefAsType::Text.into(), }; let secure_compare_context = SecureCompareContext { diff --git a/src/frontend/src/handler/fast_insert.rs b/src/frontend/src/handler/fast_insert.rs new file mode 100644 index 0000000000000..c8df8c40a2ba3 --- /dev/null +++ b/src/frontend/src/handler/fast_insert.rs @@ -0,0 +1,95 @@ +// Copyright 2024 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 std::collections::HashSet; +use std::sync::Arc; + +use pgwire::pg_response::StatementType; +use risingwave_common::error::BoxedError; +use risingwave_pb::batch_plan::{FastInsertNode, PlanFragment}; +use risingwave_sqlparser::ast::Statement; +use tokio::sync::mpsc; + +use crate::catalog::TableId; +use crate::error::Result; +use crate::handler::query::{ + gen_batch_plan_by_statement, gen_batch_plan_fragmenter, BatchPlanFragmenterResult, +}; +use crate::handler::HandlerArgs; +use crate::optimizer::OptimizerContext; +use crate::scheduler::plan_fragmenter::Query; +use crate::scheduler::FastInsertExecution; +use crate::session::SessionImpl; + +// pub async fn handle_fast_insert( +// handler_args: HandlerArgs, +// node: FastInsertNode, +// ) -> Result { +// let session = handler_args.session.clone(); + +// // Acquire the write guard for DML statements. +// // session.txn_write_guard()?; + +// gen_fast_insert_plan_inner(session, query, &read_storage_tables) +// } + +// fn gen_fast_insert_plan_inner( +// session: Arc, +// query: Query, +// read_storage_tables: &HashSet, +// ) -> Result { +// let front_env = session.env(); +// let snapshot = session.pinned_snapshot(); + +// // TODO: Passing sql here +// let execution = FastInsertExecution::new( +// query, +// front_env.clone(), +// snapshot.support_barrier_read(), +// snapshot.batch_query_epoch(read_storage_tables)?, +// session, +// ); + +// // Ok(execution.gen_plan()?) +// Ok(execution) +// } + +// pub async fn run_fast_insert(plan: PlanFragment) -> Result<()> { +// let compute_runtime = self.front_env.compute_runtime(); +// let (sender, mut receiver) = mpsc::channel(10); +// // let shutdown_rx = self.shutdown_rx().clone(); +// let sender1 = sender.clone(); +// let exec = async move { +// let mut data_stream = self.run().map(|r| r.map_err(|e| Box::new(e) as BoxedError)); +// while let Some(r) = data_stream.next().await { +// // append a query cancelled error if the query is cancelled. +// // if r.is_err() && shutdown_rx.is_cancelled() { +// // r = Err(Box::new(SchedulerError::QueryCancelled( +// // "Cancelled by user".to_string(), +// // )) as BoxedError); +// // } +// if sender1.send(r).await.is_err() { +// tracing::info!("Receiver closed."); +// return; +// } +// } +// }; + +// compute_runtime.spawn(exec); + +// while let Some(result) = receiver.recv().await { +// result?; +// } +// Ok(()) +// } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 37de2e604f864..1eb5ec49a1778 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -91,6 +91,7 @@ pub mod drop_user; mod drop_view; pub mod explain; pub mod extended_handle; +pub mod fast_insert; pub mod fetch_cursor; mod flush; pub mod handle_privilege; diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 2a26306575275..7cde172ad3f0d 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -286,6 +286,8 @@ fn gen_batch_query_plan( QueryMode::Distributed => logical.gen_batch_distributed_plan()?, }; + // println!("WKXLOG Generated query plan: {:?}", physical); + Ok(BatchQueryPlanResult { plan: physical, query_mode, diff --git a/src/frontend/src/scheduler/distributed/fast_insert.rs b/src/frontend/src/scheduler/distributed/fast_insert.rs new file mode 100644 index 0000000000000..0a3fcfd62ee8f --- /dev/null +++ b/src/frontend/src/scheduler/distributed/fast_insert.rs @@ -0,0 +1,247 @@ +// Copyright 2024 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 std::collections::HashMap; +use std::fmt::{Debug, Formatter}; +use std::mem; +use std::sync::Arc; + +use anyhow::Context; +use futures::executor::block_on; +use petgraph::dot::{Config, Dot}; +use petgraph::Graph; +use pgwire::pg_server::SessionId; +use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; +use risingwave_common::array::DataChunk; +use risingwave_pb::batch_plan::{TaskId as PbTaskId, TaskOutputId as PbTaskOutputId}; +use risingwave_pb::common::{BatchQueryEpoch, HostAddress}; +use risingwave_rpc_client::ComputeClientPoolRef; +use thiserror_ext::AsReport; +use tokio::sync::mpsc::{channel, Receiver, Sender}; +use tokio::sync::{oneshot, RwLock}; +use tokio::task::JoinHandle; +use tracing::{debug, error, info, warn, Instrument}; + +use super::{DistributedQueryMetrics, QueryExecutionInfoRef, QueryResultFetcher, StageEvent}; +use crate::catalog::catalog_service::CatalogReader; +use crate::scheduler::distributed::query::QueryMessage; +use crate::scheduler::distributed::stage::StageEvent::ScheduledRoot; +use crate::scheduler::distributed::StageEvent::Scheduled; +use crate::scheduler::distributed::StageExecution; +use crate::scheduler::plan_fragmenter::{Query, StageId, ROOT_TASK_ID, ROOT_TASK_OUTPUT_ID}; +use crate::scheduler::{ExecutionContextRef, SchedulerError, SchedulerResult}; + +pub struct FastInsertExecution { + query: Arc, + shutdown_tx: Sender, + /// Identified by `process_id`, `secret_key`. Query in the same session should have same key. + pub session_id: SessionId, + /// Permit to execute the query. Once query finishes execution, this is dropped. + pub permit: Option, +} + +impl FastInsertExecution { + #[allow(clippy::too_many_arguments)] + pub fn new( + query: Query, + session_id: SessionId, + permit: Option, + ) -> Self { + let query = Arc::new(query); + let (sender, receiver) = channel(100); + + Self { + query, + shutdown_tx: sender, + session_id, + permit, + } + } + + /// Start execution of this query. + /// Note the two shutdown channel sender and receivers are not dual. + /// One is used for propagate error to `QueryResultFetcher`, one is used for listening on + /// cancel request (from ctrl-c, cli, ui etc). + #[allow(clippy::too_many_arguments)] + pub async fn execute( + self: Arc, + context: ExecutionContextRef, + worker_node_manager: WorkerNodeSelector, + batch_query_epoch: BatchQueryEpoch, + compute_client_pool: ComputeClientPoolRef, + catalog_reader: CatalogReader, + query_execution_info: QueryExecutionInfoRef, + query_metrics: Arc, + ) -> SchedulerResult { + // Start a timer to cancel the query + // let mut timeout_abort_task_handle: Option> = None; + // if let Some(timeout) = context.timeout() { + // let this = self.clone(); + // timeout_abort_task_handle = Some(tokio::spawn(async move { + // tokio::time::sleep(timeout).await; + // warn!( + // "Query {:?} timeout after {} seconds, sending cancel message.", + // this.query.query_id, + // timeout.as_secs(), + // ); + // this.abort(format!("timeout after {} seconds", timeout.as_secs())) + // .await; + // })); + // } + + // Create a oneshot channel for QueryResultFetcher to get failed event. + // let (root_stage_sender, root_stage_receiver) = + // oneshot::channel::>(); + + let runner = QueryRunner { + query: self.query.clone(), + stage_executions, + msg_receiver, + root_stage_sender: Some(root_stage_sender), + scheduled_stages_count: 0, + query_execution_info, + query_metrics, + timeout_abort_task_handle, + }; + + let span = tracing::info_span!( + "distributed_execute", + query_id = self.query.query_id.id, + epoch = ?batch_query_epoch, + ); + + tracing::trace!("Starting query: {:?}", self.query.query_id); + + // Not trace the error here, it will be processed in scheduler. + // tokio::spawn(async move { runner.run().instrument(span).await }); + run_inner(); + // let root_stage = root_stage_receiver + // .await + // .context("Starting query execution failed")??; + + tracing::trace!( + "Received root stage query result fetcher: {:?}, query id: {:?}", + root_stage, + self.query.query_id + ); + + tracing::trace!("Query {:?} started.", self.query.query_id); + Ok(root_stage) + } + + fn run_inner() { + let runner = StageRunner { + epoch: self.epoch, + stage: self.stage.clone(), + worker_node_manager: self.worker_node_manager.clone(), + tasks: self.tasks.clone(), + msg_sender, + children: self.children.clone(), + state: self.state.clone(), + compute_client_pool: self.compute_client_pool.clone(), + catalog_reader: self.catalog_reader.clone(), + ctx: self.ctx.clone(), + }; + + // The channel used for shutdown signal messaging. + let (sender, receiver) = ShutdownToken::new(); + // Fill the shutdown sender. + let mut holder = self.shutdown_tx.write().await; + *holder = Some(sender); + + // Change state before spawn runner. + *s = StageState::Started; + + let span = tracing::info_span!( + "stage", + "otel.name" = format!("Stage {}-{}", self.stage.query_id.id, self.stage.id), + query_id = self.stage.query_id.id, + stage_id = self.stage.id, + ); + self.ctx + .session() + .env() + .compute_runtime() + .spawn(async move { runner.run(receiver).instrument(span).await }); + + tracing::trace!( + "Stage {:?}-{:?} started.", + self.stage.query_id.id, + self.stage.id + ) + } + + fn schedule() { + for id in 0..self.stage.parallelism.unwrap() { + let task_id = PbTaskId { + query_id: self.stage.query_id.id.clone(), + stage_id: self.stage.id, + task_id: id as u64, + }; + let plan_fragment = self.create_plan_fragment(id as u64, None); + let worker = self.choose_worker(&plan_fragment, id, self.stage.dml_table_id)?; + self.call_rpc( + epoch, + worker_node_manager, + task_id, + plan_fragment, + compute_client_pool, + worker, + ); + // futures.push(self.schedule_task(task_id, plan_fragment, worker, expr_context.clone())); + } + } + + fn call_rpc( + epoch: BatchQueryEpoch, + worker_node_manager: WorkerNodeSelector, + task_id: PbTaskId, + plan_fragment: PlanFragment, + compute_client_pool: ComputeClientPoolRef, + worker: Option, + ) { + let mut worker = worker.unwrap_or(worker_node_manager.next_random_worker()?); + let worker_node_addr = worker.host.take().unwrap(); + let compute_client = compute_client_pool + .get_by_addr((&worker_node_addr).into()) + // .inspect_err(|_| self.mask_failed_serving_worker(&worker)) + .map_err(|e| anyhow!(e))?; + + let t_id = task_id.task_id; + + let stream_status: TaskInfoResponse = compute_client + .fast_insert(task_id, plan_fragment, epoch, expr_context) + // .inspect_err(|_| self.mask_failed_serving_worker(&worker)) + .map_err(|e| anyhow!(e))?; + + Ok(stream_status) + } + + // fn mask_failed_serving_worker(&self, worker: &WorkerNode) { + // if !worker.property.as_ref().map_or(false, |p| p.is_serving) { + // return; + // } + // let duration = Duration::from_secs(std::cmp::max( + // self.ctx + // .session + // .env() + // .batch_config() + // .mask_worker_temporary_secs as u64, + // 1, + // )); + // self.worker_node_manager + // .manager + // .mask_worker_node(worker.id, duration); + // } +} diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 62cf4cca08905..5184ddeed333c 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -232,6 +232,11 @@ impl QueryExecution { compute_client_pool: ComputeClientPoolRef, catalog_reader: CatalogReader, ) -> HashMap> { + // println!( + // "WKXLOG self.query.stage_graph.stages.len(): {}, self.query.stage_graph.stages: {:?}", + // self.query.stage_graph.stages.len(), + // self.query.stage_graph.stages + // ); let mut stage_executions: HashMap> = HashMap::with_capacity(self.query.stage_graph.stages.len()); diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs new file mode 100644 index 0000000000000..1389da80dffc6 --- /dev/null +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -0,0 +1,134 @@ +// Copyright 2024 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::atomic::{AtomicU32, Ordering}; +use std::sync::Arc; + +use anyhow::anyhow; +use futures::stream::BoxStream; +use futures::{FutureExt, StreamExt}; +use futures_async_stream::try_stream; +use itertools::Itertools; +use pgwire::pg_server::SessionId; +use risingwave_batch::error::BatchError; +use risingwave_batch::executor::ExecutorBuilder; +use risingwave_batch::task::{ShutdownToken, TaskId}; +use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; +use risingwave_common::array::DataChunk; +use risingwave_common::bail; +use risingwave_common::error::BoxedError; +use risingwave_common::hash::WorkerSlotMapping; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::tracing::{InstrumentStream, TracingContext}; +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, FastInsertNode, LocalExecutePlan, PbTaskId, PlanFragment, + PlanNode as PbPlanNode, TaskOutputId, +}; +use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; +use risingwave_pb::task_service::FastInsertRequest; +use tracing::debug; + +use super::plan_fragmenter::{PartitionInfo, QueryStage, QueryStageRef}; +use crate::catalog::{FragmentId, TableId}; +use crate::error::RwError; +use crate::optimizer::plan_node::PlanNodeType; +use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; +use crate::scheduler::task_context::FrontendBatchTaskContext; +use crate::scheduler::{SchedulerError, SchedulerResult}; +use crate::session::{FrontendEnv, SessionImpl}; + +pub struct FastInsertExecution { + fast_insert_node: FastInsertNode, + front_env: FrontendEnv, + session: Arc, + worker_node_manager: WorkerNodeSelector, +} + +impl FastInsertExecution { + pub fn new( + fast_insert_node: FastInsertNode, + front_env: FrontendEnv, + session: Arc, + ) -> Self { + let worker_node_manager = + WorkerNodeSelector::new(front_env.worker_node_manager_ref(), false); + + Self { + fast_insert_node, + front_env, + session, + worker_node_manager, + } + } + + pub async fn my_execute(self) -> SchedulerResult<()> { + let workers = self.choose_worker( + &TableId::new(self.fast_insert_node.table_id), + self.fast_insert_node.session_id, + )?; + + let client = self.session.env().client_pool().get(&workers).await?; + let request = FastInsertRequest { + fast_insert_node: Some(self.fast_insert_node), + }; + let response = client.fast_insert(request).await?; + println!("WKXLOG response: {:?}", response); + Ok(()) + } + + #[inline(always)] + fn get_table_dml_vnode_mapping( + &self, + table_id: &TableId, + ) -> SchedulerResult { + let guard = self.front_env.catalog_reader().read_guard(); + + let table = guard + .get_any_table_by_id(table_id) + .map_err(|e| SchedulerError::Internal(anyhow!(e)))?; + + let fragment_id = match table.dml_fragment_id.as_ref() { + Some(dml_fragment_id) => dml_fragment_id, + // Backward compatibility for those table without `dml_fragment_id`. + None => &table.fragment_id, + }; + + self.worker_node_manager + .manager + .get_streaming_fragment_mapping(fragment_id) + .map_err(|e| e.into()) + } + + fn choose_worker(&self, table_id: &TableId, session_id: u32) -> SchedulerResult { + // dml should use streaming vnode mapping + let vnode_mapping = self.get_table_dml_vnode_mapping(table_id)?; + let worker_node = { + let worker_ids = vnode_mapping.iter_unique().collect_vec(); + let candidates = self + .worker_node_manager + .manager + .get_workers_by_worker_slot_ids(&worker_ids)?; + if candidates.is_empty() { + return Err(BatchError::EmptyWorkerNodes.into()); + } + candidates[session_id as usize % candidates.len()].clone() + }; + Ok(worker_node) + } +} diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 570cff8504a5a..7ac156e5c9587 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -108,6 +108,8 @@ impl LocalQueryExecution { let plan_fragment = self.create_plan_fragment()?; let plan_node = plan_fragment.root.unwrap(); + // println!("WKXLOG plan_node: {:?}", plan_node); + let executor = ExecutorBuilder::new( &plan_node, &task_id, @@ -121,6 +123,8 @@ impl LocalQueryExecution { // Release potential large object in Query and PlanNode early. drop(self); + // println!("WKXLOG executor: {:?}", executor); + #[for_await] for chunk in executor.execute() { yield chunk?; diff --git a/src/frontend/src/scheduler/mod.rs b/src/frontend/src/scheduler/mod.rs index d81510e8d82e9..6b863f6cf481d 100644 --- a/src/frontend/src/scheduler/mod.rs +++ b/src/frontend/src/scheduler/mod.rs @@ -32,8 +32,11 @@ mod snapshot; pub use snapshot::*; mod local; pub use local::*; +mod fast_insert; +pub use fast_insert::*; -use crate::scheduler::task_context::FrontendBatchTaskContext; +// pub use crate::scheduler::fast_insert::{run_inner_call, run_inner_call_2}; +pub use crate::scheduler::task_context::FrontendBatchTaskContext; mod error; pub mod streaming_manager; diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index d409225c96c90..8691a23044126 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -570,6 +570,7 @@ impl Debug for QueryStage { .field("parallelism", &self.parallelism) .field("exchange_info", &self.exchange_info) .field("has_table_scan", &self.has_table_scan()) + .field("stages", &self.root) .finish() } } diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 17bd803cd6bbd..b063ef8903f1b 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -23,7 +23,11 @@ use axum::routing::post; use axum::Router; use pgwire::net::Address; use pgwire::pg_server::SessionManager; +use risingwave_common::array::{Array, ArrayBuilder, DataChunk}; use risingwave_common::secret::LocalSecretManager; +use risingwave_common::types::{DataType, JsonbVal, Scalar}; +use risingwave_pb::batch_plan::FastInsertNode; +use risingwave_pb::catalog::WebhookSourceInfo; use risingwave_sqlparser::ast::{Expr, ObjectName}; use tokio::net::TcpListener; use tower::ServiceBuilder; @@ -31,7 +35,9 @@ use tower_http::add_extension::AddExtensionLayer; use tower_http::compression::CompressionLayer; use tower_http::cors::{self, CorsLayer}; -use crate::handler::handle; +// use crate::handler::fast_insert::handle_fast_insert; +use crate::handler::{handle, HandlerArgs}; +use crate::scheduler::{FastInsertExecution, FrontendBatchTaskContext}; use crate::webhook::utils::{err, Result}; mod utils; @@ -40,6 +46,12 @@ pub type Service = Arc; // We always use the `root` user to connect to the database to allow the webhook service to access all tables. const USER: &str = "root"; +#[derive(Clone)] +pub struct TableWebhookContext { + pub webhook_source_info: WebhookSourceInfo, + pub fast_insert_node: FastInsertNode, +} + #[derive(Clone)] pub struct WebhookService { webhook_addr: SocketAddr, @@ -48,13 +60,20 @@ pub struct WebhookService { pub(super) mod handlers { use std::net::Ipv4Addr; + use iceberg::table::Table; + use jsonbb::Value; + use pgwire::pg_server::Session; + use risingwave_common::array::JsonbArrayBuilder; + use risingwave_common::catalog::TableId; + use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; - use risingwave_sqlparser::ast::{Query, SetExpr, Statement, Value, Values}; + use risingwave_pb::plan_common::DefaultColumns; + // use risingwave_sqlparser::ast::{Query, SetExpr, Statement, Value, Values}; use utils::{header_map_to_json, verify_signature}; use super::*; use crate::catalog::root_catalog::SchemaPath; - use crate::session::SESSION_MANAGER; + use crate::session::{SessionImpl, SESSION_MANAGER}; pub async fn handle_post_request( Extension(_srv): Extension, @@ -69,6 +88,10 @@ pub(super) mod handlers { // Can be any address, we use the port of meta to indicate that it's a internal request. let dummy_addr = Address::Tcp(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691)); + // println!( + // "WKXLOG Received webhook request for {}/{}/{}", + // database, schema, table + // ); // TODO(kexiang): optimize this // get a session object for the corresponding database let session = session_mgr @@ -83,34 +106,26 @@ pub(super) mod handlers { ) })?; + // println!("WKXLOG session created"); + let TableWebhookContext { + webhook_source_info, + mut fast_insert_node, + } = acquire_table_info(&session, &database, &schema, &table).await?; + let WebhookSourceInfo { - secret_ref, signature_expr, - } = { - let search_path = session.config().search_path(); - let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); - - let reader = session.env().catalog_reader().read_guard(); - let (table_catalog, _schema) = reader - .get_any_table_by_name(database.as_str(), schema_path, &table) - .map_err(|e| err(e, StatusCode::NOT_FOUND))?; - - table_catalog - .webhook_info - .as_ref() - .ok_or_else(|| { - err( - anyhow!("Table `{}` is not with webhook source", table), - StatusCode::FORBIDDEN, - ) - })? - .clone() - }; + secret_ref, + .. + } = webhook_source_info; + // println!("WKXLOG webhook source info: {:?}", secret_ref); - let secret_string = LocalSecretManager::global() - .fill_secret(secret_ref.unwrap()) - .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + // let secret_string = LocalSecretManager::global() + // .fill_secret(secret_ref.unwrap()) + // .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + + let secret_string = String::from("TEST_WEBHOOK"); + // println!("WKXLOG secret string: {:?}", secret_string); // Once limitation here is that the key is no longer case-insensitive, users must user the lowercase key when defining the webhook source table. let headers_jsonb = header_map_to_json(&headers); @@ -122,6 +137,8 @@ pub(super) mod handlers { ) .await?; + // println!("WKXLOG is_valid: {:?}", is_valid); + if !is_valid { return Err(err( anyhow!("Signature verification failed"), @@ -129,39 +146,75 @@ pub(super) mod handlers { )); } - let payload = String::from_utf8(body.to_vec()).map_err(|e| { - err( - anyhow!(e).context("Failed to parse body"), - StatusCode::UNPROCESSABLE_ENTITY, - ) - })?; - - let insert_stmt = Statement::Insert { - table_name: ObjectName::from(vec![table.as_str().into()]), - columns: vec![], - source: Box::new(Query { - with: None, - body: SetExpr::Values(Values(vec![vec![Expr::Value(Value::SingleQuotedString( - payload, - ))]])), - order_by: vec![], - limit: None, - offset: None, - fetch: None, - }), - returning: vec![], - }; + // let payload = String::from_utf8(body.to_vec()).map_err(|e| { + // err( + // anyhow!(e).context("Failed to parse body"), + // StatusCode::UNPROCESSABLE_ENTITY, + // ) + // })?; + + let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); + // TODO(kexiang): handle errors + let json_value = Value::from_text(&body.to_vec()).unwrap(); + let jsonb_val = JsonbVal::from(json_value); + // Add 4 ListValues to ArrayBuilder + builder.append(Some(jsonb_val.as_scalar_ref())); + + // Use builder to obtain a single (List) column DataChunk + let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); + fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); + // let context = FrontendBatchTaskContext::new(session.clone()); + let execution = + FastInsertExecution::new(fast_insert_node, session.env().clone(), session.clone()); + let res = execution.my_execute().await.unwrap(); - let _rsp = handle(session, insert_stmt, Arc::from(""), vec![]) - .await - .map_err(|e| { + Ok(()) + } + + async fn acquire_table_info( + session: &Arc, + database: &String, + schema: &String, + table: &String, + ) -> Result { + // println!("WKXLOG session created"); + + let search_path = session.config().search_path(); + let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); + + let reader = session.env().catalog_reader().read_guard(); + let (table_catalog, _schema) = reader + .get_any_table_by_name(database.as_str(), schema_path, &table) + .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + + let webhook_source_info = table_catalog + .webhook_info + .as_ref() + .ok_or_else(|| { err( - anyhow!(e).context("Failed to insert into target table"), - StatusCode::INTERNAL_SERVER_ERROR, + anyhow!("Table `{}` is not with webhook source", table), + StatusCode::FORBIDDEN, ) - })?; + })? + .clone(); + + let fast_insert_node = FastInsertNode { + table_id: table_catalog.id().table_id, + table_version_id: table_catalog.version_id().expect("table must be versioned"), + column_indices: vec![0], + data_chunk: None, + default_columns: Some(DefaultColumns { + default_columns: vec![], + }), + row_id_index: Some(1), + session_id: session.id().0 as u32, + }; - Ok(()) + let table_webhook_context = TableWebhookContext { + webhook_source_info, + fast_insert_node, + }; + Ok(table_webhook_context) } } diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index 38f9ae83270b3..e8eaf1dac2ec1 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -40,8 +40,8 @@ use risingwave_pb::task_service::exchange_service_client::ExchangeServiceClient; use risingwave_pb::task_service::task_service_client::TaskServiceClient; use risingwave_pb::task_service::{ permits, CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, - GetDataRequest, GetDataResponse, GetStreamRequest, GetStreamResponse, PbPermits, - TaskInfoResponse, + FastInsertRequest, FastInsertResponse, GetDataRequest, GetDataResponse, GetStreamRequest, + GetStreamResponse, PbPermits, TaskInfoResponse, }; use tokio::sync::mpsc; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -210,6 +210,16 @@ impl ComputeClient { .into_inner()) } + pub async fn fast_insert(&self, req: FastInsertRequest) -> Result { + Ok(self + .task_client + .to_owned() + .fast_insert(req) + .await + .map_err(RpcError::from_compute_status)? + .into_inner()) + } + pub async fn stack_trace(&self) -> Result { Ok(self .monitor_client diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 8dddf41aa59f5..fbc18c40c7800 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -20,6 +20,7 @@ use futures::TryStreamExt; use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; +use risingwave_common::util::epoch::Epoch; use risingwave_common_rate_limit::{MonitoredRateLimiter, RateLimit, RateLimiter}; use risingwave_dml::dml_manager::DmlManagerRef; use risingwave_expr::codegen::BoxStream; @@ -123,13 +124,18 @@ impl DmlExecutor { // Merge the two streams using `StreamReaderWithPause` because when we receive a pause // barrier, we should stop receiving the data from DML. We poll data from the two streams in // a round robin way. - let mut stream = StreamReaderWithPause::::new(upstream, reader); + let mut stream = + StreamReaderWithPause::>)>::new( + upstream, reader, + ); // If the first barrier requires us to pause on startup, pause the stream. if barrier.is_pause_on_startup() { stream.pause_stream(); } + let mut epoch = barrier.get_curr_epoch(); + yield Message::Barrier(barrier); // Active transactions: txn_id -> TxnBuffer with transaction chunks. @@ -150,6 +156,7 @@ impl DmlExecutor { Either::Left(msg) => { // Stream messages. if let Message::Barrier(barrier) = &msg { + epoch = barrier.get_curr_epoch(); // We should handle barrier messages here to pause or resume the data from // DML. if let Some(mutation) = barrier.mutation.as_deref() { @@ -195,7 +202,7 @@ impl DmlExecutor { } yield msg; } - Either::Right(txn_msg) => { + Either::Right((txn_msg, epoch_notifier)) => { // Batch data. match txn_msg { TxnMsg::Begin(txn_id) => { @@ -206,6 +213,9 @@ impl DmlExecutor { }); } TxnMsg::End(txn_id) => { + if let Some(sender) = epoch_notifier { + let _ = sender.send(epoch); + } let mut txn_buffer = active_txn_map.remove(&txn_id) .unwrap_or_else(|| panic!("Receive an unexpected transaction end message. Active transaction map doesn't contain this transaction txn_id = {}.", txn_id)); @@ -304,30 +314,31 @@ impl Execute for DmlExecutor { } } -type BoxTxnMessageStream = BoxStream<'static, risingwave_dml::error::Result>; -#[try_stream(ok = TxnMsg, error = risingwave_dml::error::DmlError)] +type BoxTxnMessageStream = + BoxStream<'static, risingwave_dml::error::Result<(TxnMsg, Option>)>>; +#[try_stream(ok = (TxnMsg, Option>), error = risingwave_dml::error::DmlError)] async fn apply_dml_rate_limit( stream: BoxTxnMessageStream, rate_limiter: Arc, ) { #[for_await] for txn_msg in stream { - let txn_msg = txn_msg?; + let (txn_msg, epoch_notifier) = txn_msg?; match txn_msg { TxnMsg::Begin(txn_id) => { - yield TxnMsg::Begin(txn_id); + yield (TxnMsg::Begin(txn_id), epoch_notifier); } TxnMsg::End(txn_id) => { - yield TxnMsg::End(txn_id); + yield (TxnMsg::End(txn_id), epoch_notifier); } TxnMsg::Rollback(txn_id) => { - yield TxnMsg::Rollback(txn_id); + yield (TxnMsg::Rollback(txn_id), epoch_notifier); } TxnMsg::Data(txn_id, chunk) => { let chunk_size = chunk.capacity(); if chunk_size == 0 { // empty chunk - yield TxnMsg::Data(txn_id, chunk); + yield (TxnMsg::Data(txn_id, chunk), None); continue; } From 16d0f088a394c1c8926ee9acb569f351c4df3c3a Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Wed, 15 Jan 2025 18:32:54 -0500 Subject: [PATCH 02/27] fix secret --- proto/catalog.proto | 1 + proto/task_service.proto | 2 +- .../executors/src/executor/fast_insert.rs | 30 +++--- src/batch/executors/src/executor/insert.rs | 2 +- src/batch/src/rpc/service/task_service.rs | 49 +++++----- .../src/transaction/transaction_message.rs | 8 +- src/dml/src/table.rs | 49 +++++----- src/dml/src/txn_channel.rs | 20 +--- src/frontend/src/handler/create_table.rs | 12 ++- src/frontend/src/handler/fast_insert.rs | 95 ------------------- src/frontend/src/handler/mod.rs | 1 - src/frontend/src/handler/query.rs | 2 - src/frontend/src/scheduler/fast_insert.rs | 35 ++----- src/frontend/src/webhook/mod.rs | 52 ++++------ src/sqlparser/src/ast/ddl.rs | 1 + src/sqlparser/src/parser.rs | 9 ++ src/stream/src/executor/dml.rs | 28 +++--- 17 files changed, 127 insertions(+), 269 deletions(-) delete mode 100644 src/frontend/src/handler/fast_insert.rs diff --git a/proto/catalog.proto b/proto/catalog.proto index 341b1362b3d6c..7fea9c56adbd6 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -100,6 +100,7 @@ message StreamSourceInfo { message WebhookSourceInfo { secret.SecretRef secret_ref = 1; expr.ExprNode signature_expr = 2; + bool wait_for_persistence = 3; } message Source { diff --git a/proto/task_service.proto b/proto/task_service.proto index 1d98865e2269a..2abf728a78c0c 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -45,10 +45,10 @@ message CreateTaskRequest { } message FastInsertRequest { + bool wait_epoch = 1; batch_plan.FastInsertNode fast_insert_node = 2; } - message FastInsertResponse { // Optional error message for failed task. string error_message = 1; diff --git a/src/batch/executors/src/executor/fast_insert.rs b/src/batch/executors/src/executor/fast_insert.rs index b48c3df6b7140..069579ba38c74 100644 --- a/src/batch/executors/src/executor/fast_insert.rs +++ b/src/batch/executors/src/executor/fast_insert.rs @@ -34,8 +34,6 @@ pub struct FastInsertExecutor { table_id: TableId, table_version_id: TableVersionId, dml_manager: DmlManagerRef, - // TODO(Kexiang): get rid of it? - input_schema: Schema, column_indices: Vec, // TODO(Kexiang): get rid of it? @@ -86,7 +84,6 @@ impl FastInsertExecutor { table_id, insert_node.table_version_id, dml_manager, - schema, column_indices, sorted_default_columns, insert_node.row_id_index.as_ref().map(|index| *index as _), @@ -101,7 +98,6 @@ impl FastInsertExecutor { table_id: TableId, table_version_id: TableVersionId, dml_manager: DmlManagerRef, - input_schema: Schema, column_indices: Vec, sorted_default_columns: Vec<(usize, BoxedExpression)>, row_id_index: Option, @@ -112,7 +108,6 @@ impl FastInsertExecutor { table_id, table_version_id, dml_manager, - input_schema, column_indices, sorted_default_columns, row_id_index, @@ -192,11 +187,13 @@ mod tests { use serde_json::json; use super::*; + use crate::risingwave_common::array::ArrayBuilder; use crate::risingwave_common::types::Scalar; use crate::*; #[tokio::test] - async fn test_insert_executor() -> Result<()> { + async fn test_fast_insert() -> Result<()> { + let epoch = Epoch::now(); let dml_manager = Arc::new(DmlManager::for_test()); let store = MemoryStateStore::new(); // Schema of the table @@ -240,33 +237,28 @@ mod tests { table_id, INITIAL_TABLE_VERSION_ID, dml_manager, - schema, vec![0], // Ignoring insertion order vec![], row_id_index, 0, )); let handle = tokio::spawn(async move { - insert_executor.do_execute(data_chunk).await.unwrap(); + let epoch_recieved = insert_executor.do_execute(data_chunk).await.unwrap(); + assert_eq!(epoch, epoch_recieved); }); // Read - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), _)); - - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Data(_, chunk),_) => { - // assert_eq!( - // chunk.columns()[0].as_int32().iter().collect::>(), - // vec![Some(1), Some(3), Some(5), Some(7), Some(9)] - // ); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::Begin(_)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::Data(_, chunk) => { assert_eq!(chunk.columns().len(),2); let array = chunk.columns()[0].as_jsonb().iter().collect::>(); - println!("WKXLOG arrar: {:?}",array); - // assert_eq!(*array,2); - // assert_eq!(*chunk.columns()[2], array); + assert_eq!(JsonbVal::from(array[0].unwrap()), jsonb_val); }); - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::End(_), _)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(_, Some(epoch_notifier)) => { + epoch_notifier.send(epoch.clone()).unwrap(); + }); let epoch = u64::MAX; let full_range = (Bound::Unbounded, Bound::Unbounded); let store_content = store diff --git a/src/batch/executors/src/executor/insert.rs b/src/batch/executors/src/executor/insert.rs index 3ff8cede9dc0f..9b0b0839e9964 100644 --- a/src/batch/executors/src/executor/insert.rs +++ b/src/batch/executors/src/executor/insert.rs @@ -387,7 +387,7 @@ mod tests { assert_eq!(*chunk.columns()[2], array); }); - assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(_)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(..)); let epoch = u64::MAX; let full_range = (Bound::Unbounded, Bound::Unbounded); let store_content = store diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index bb28f57377d4e..d1dc499c22e0e 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -127,11 +127,9 @@ impl TaskService for BatchServiceImpl { &self, request: Request, ) -> Result, Status> { - let insert_node = request - .into_inner() - .fast_insert_node - .expect("no fast insert node found"); - let res = self.do_fast_insert(insert_node).await; + let req = request.into_inner(); + let insert_node = req.fast_insert_node.expect("no fast insert node found"); + let res = self.do_fast_insert(insert_node, req.wait_epoch).await; match res { Ok(_) => Ok(Response::new(FastInsertResponse { error_message: String::from("success"), @@ -207,28 +205,33 @@ impl BatchServiceImpl { Ok(Response::new(ReceiverStream::new(rx))) } - async fn do_fast_insert(&self, insert_node: FastInsertNode) -> Result<(), BatchError> { + async fn do_fast_insert( + &self, + insert_node: FastInsertNode, + wait_epoch: bool, + ) -> Result<(), BatchError> { let table_id = insert_node.table_id; let (executor, data_chunk) = FastInsertExecutor::build(self.env.dml_manager_ref(), insert_node)?; let epoch = executor.do_execute(data_chunk).await?; - - dispatch_state_store!(self.env.state_store(), store, { - use risingwave_common::catalog::TableId; - use risingwave_hummock_sdk::HummockReadEpoch; - use risingwave_storage::store::TryWaitEpochOptions; - use risingwave_storage::StateStore; - - store - .try_wait_epoch( - HummockReadEpoch::Committed(epoch.0), - TryWaitEpochOptions { - table_id: TableId::new(table_id), - }, - ) - .await - .map_err(BatchError::from)?; - }); + if wait_epoch { + dispatch_state_store!(self.env.state_store(), store, { + use risingwave_common::catalog::TableId; + use risingwave_hummock_sdk::HummockReadEpoch; + use risingwave_storage::store::TryWaitEpochOptions; + use risingwave_storage::StateStore; + + store + .try_wait_epoch( + HummockReadEpoch::Committed(epoch.0), + TryWaitEpochOptions { + table_id: TableId::new(table_id), + }, + ) + .await + .map_err(BatchError::from)?; + }); + } Ok(()) } } diff --git a/src/common/src/transaction/transaction_message.rs b/src/common/src/transaction/transaction_message.rs index 540a25f56ffc1..d4f92fddfb52a 100644 --- a/src/common/src/transaction/transaction_message.rs +++ b/src/common/src/transaction/transaction_message.rs @@ -13,16 +13,18 @@ // limitations under the License. use enum_as_inner::EnumAsInner; +use tokio::sync::oneshot; use crate::array::StreamChunk; use crate::transaction::transaction_id::TxnId; use crate::transaction::transaction_message::TxnMsg::{Begin, Data, End, Rollback}; +use crate::util::epoch::Epoch; #[derive(Debug, EnumAsInner)] pub enum TxnMsg { Begin(TxnId), Data(TxnId, StreamChunk), - End(TxnId), + End(TxnId, Option>), Rollback(TxnId), } @@ -31,14 +33,14 @@ impl TxnMsg { match self { Begin(txn_id) => *txn_id, Data(txn_id, _) => *txn_id, - End(txn_id) => *txn_id, + End(txn_id, _) => *txn_id, Rollback(txn_id) => *txn_id, } } pub fn as_stream_chunk(&self) -> Option<&StreamChunk> { match self { - Begin(_) | End(_) | Rollback(_) => None, + Begin(_) | End(..) | Rollback(_) => None, Data(_, chunk) => Some(chunk), } } diff --git a/src/dml/src/table.rs b/src/dml/src/table.rs index 20d6754d4d80b..d8b8548b95e79 100644 --- a/src/dml/src/table.rs +++ b/src/dml/src/table.rs @@ -190,7 +190,7 @@ impl WriteHandle { assert_eq!(self.txn_state, TxnState::Begin); self.txn_state = TxnState::Committed; // Await the notifier. - let notifier = self.write_txn_control_msg(TxnMsg::End(self.txn_id))?; + let notifier = self.write_txn_control_msg(TxnMsg::End(self.txn_id, None))?; notifier.await.map_err(|_| DmlError::ReaderClosed)?; Ok(()) } @@ -199,10 +199,15 @@ impl WriteHandle { assert_eq!(self.txn_state, TxnState::Begin); self.txn_state = TxnState::Committed; // Await the notifier. - let (notifier, epoch_notifier) = - self.write_txn_control_msg_returning_epoch(TxnMsg::End(self.txn_id))?; + let (epoch_notifier_tx, epoch_notifier_rx) = oneshot::channel(); + let notifier = self.write_txn_control_msg_returning_epoch(TxnMsg::End( + self.txn_id, + Some(epoch_notifier_tx), + ))?; notifier.await.map_err(|_| DmlError::ReaderClosed)?; - let epoch = epoch_notifier.await.map_err(|_| DmlError::ReaderClosed)?; + let epoch = epoch_notifier_rx + .await + .map_err(|_| DmlError::ReaderClosed)?; Ok(epoch) } @@ -224,7 +229,7 @@ impl WriteHandle { async fn write_txn_data_msg(&self, txn_msg: TxnMsg) -> Result> { assert_eq!(self.txn_id, txn_msg.txn_id()); let (notifier_tx, notifier_rx) = oneshot::channel(); - match self.tx.send(txn_msg, notifier_tx, None).await { + match self.tx.send(txn_msg, notifier_tx).await { Ok(_) => Ok(notifier_rx), // It's possible that the source executor is scaled in or migrated, so the channel @@ -238,7 +243,7 @@ impl WriteHandle { fn write_txn_control_msg(&self, txn_msg: TxnMsg) -> Result> { assert_eq!(self.txn_id, txn_msg.txn_id()); let (notifier_tx, notifier_rx) = oneshot::channel(); - match self.tx.send_immediate(txn_msg, notifier_tx, None) { + match self.tx.send_immediate(txn_msg, notifier_tx) { Ok(_) => Ok(notifier_rx), // It's possible that the source executor is scaled in or migrated, so the channel @@ -250,15 +255,11 @@ impl WriteHandle { fn write_txn_control_msg_returning_epoch( &self, txn_msg: TxnMsg, - ) -> Result<(oneshot::Receiver, oneshot::Receiver)> { + ) -> Result> { assert_eq!(self.txn_id, txn_msg.txn_id()); let (notifier_tx, notifier_rx) = oneshot::channel(); - let (epoch_notifier_tx, epoch_notifier_rx) = oneshot::channel(); - match self - .tx - .send_immediate(txn_msg, notifier_tx, Some(epoch_notifier_tx)) - { - Ok(_) => Ok((notifier_rx, epoch_notifier_rx)), + match self.tx.send_immediate(txn_msg, notifier_tx) { + Ok(_) => Ok(notifier_rx), // It's possible that the source executor is scaled in or migrated, so the channel // is closed. To guarantee the transactional atomicity, bail out. @@ -280,10 +281,10 @@ pub struct TableStreamReader { impl TableStreamReader { #[try_stream(boxed, ok = StreamChunk, error = DmlError)] pub async fn into_data_stream_for_test(mut self) { - while let Some((txn_msg, notifier, _epoch_notifier)) = self.rx.recv().await { + while let Some((txn_msg, notifier)) = self.rx.recv().await { // Notify about that we've taken the chunk. match txn_msg { - TxnMsg::Begin(_) | TxnMsg::End(_) | TxnMsg::Rollback(_) => { + TxnMsg::Begin(_) | TxnMsg::End(..) | TxnMsg::Rollback(_) => { _ = notifier.send(0); } TxnMsg::Data(_, chunk) => { @@ -294,18 +295,18 @@ impl TableStreamReader { } } - #[try_stream(boxed, ok = (TxnMsg, Option>), error = DmlError)] + #[try_stream(boxed, ok = TxnMsg, error = DmlError)] pub async fn into_stream(mut self) { - while let Some((txn_msg, notifier, epoch_notifier)) = self.rx.recv().await { + while let Some((txn_msg, notifier)) = self.rx.recv().await { // Notify about that we've taken the chunk. match &txn_msg { - TxnMsg::Begin(_) | TxnMsg::End(_) | TxnMsg::Rollback(_) => { + TxnMsg::Begin(_) | TxnMsg::End(..) | TxnMsg::Rollback(_) => { _ = notifier.send(0); - yield (txn_msg, epoch_notifier); + yield txn_msg; } TxnMsg::Data(_, chunk) => { _ = notifier.send(chunk.cardinality()); - yield (txn_msg, epoch_notifier); + yield txn_msg; } } } @@ -342,7 +343,7 @@ mod tests { .unwrap(); write_handle.begin().unwrap(); - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), None)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::Begin(_)); macro_rules! write_chunk { ($i:expr) => {{ @@ -374,7 +375,7 @@ mod tests { write_handle.end().await.unwrap(); }); - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::End(_), None)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(..)); Ok(()) } @@ -388,7 +389,7 @@ mod tests { .unwrap(); write_handle.begin().unwrap(); - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Begin(_), None)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::Begin(_)); let chunk = StreamChunk::new(vec![Op::Insert], vec![I64Array::from_iter([1]).into_ref()]); write_handle.write_chunk(chunk).await.unwrap(); @@ -400,7 +401,7 @@ mod tests { // Rollback on drop drop(write_handle); - assert_matches!(reader.next().await.unwrap()?, (TxnMsg::Rollback(_), None)); + assert_matches!(reader.next().await.unwrap()?, TxnMsg::Rollback(_)); Ok(()) } diff --git a/src/dml/src/txn_channel.rs b/src/dml/src/txn_channel.rs index fbcb3da067ec5..30eb457c3eeb1 100644 --- a/src/dml/src/txn_channel.rs +++ b/src/dml/src/txn_channel.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use futures::FutureExt; use risingwave_common::transaction::transaction_message::TxnMsg; -use risingwave_common::util::epoch::Epoch; use tokio::sync::{mpsc, oneshot, Semaphore}; pub struct PermitValue(u32); @@ -24,7 +23,6 @@ pub struct PermitValue(u32); pub struct TxnMsgWithPermits { pub txn_msg: TxnMsg, pub notificator: oneshot::Sender, - pub epoch_notificator: Option>, pub permit_value: Option, } @@ -82,7 +80,6 @@ impl Sender { &self, txn_msg: TxnMsg, notificator: oneshot::Sender, - epoch_notificator: Option>, ) -> Result<(), mpsc::error::SendError> { // The semaphores should never be closed. let permits = match &txn_msg { @@ -102,14 +99,13 @@ impl Sender { .forget(); Some(PermitValue(card as _)) } - TxnMsg::Begin(_) | TxnMsg::Rollback(_) | TxnMsg::End(_) => None, + TxnMsg::Begin(_) | TxnMsg::Rollback(_) | TxnMsg::End(..) => None, }; self.tx .send(TxnMsgWithPermits { txn_msg, notificator, - epoch_notificator, permit_value: permits, }) .map_err(|e| mpsc::error::SendError(e.0.txn_msg)) @@ -123,9 +119,8 @@ impl Sender { &self, txn_msg: TxnMsg, notificator: oneshot::Sender, - epoch_notificator: Option>, ) -> Result<(), mpsc::error::SendError> { - self.send(txn_msg, notificator, epoch_notificator) + self.send(txn_msg, notificator) .now_or_never() .expect("cannot send immediately") } @@ -146,17 +141,10 @@ impl Receiver { /// Receive the next message for this receiver, with the permits of this message added back. /// /// Returns `None` if the channel has been closed. - pub async fn recv( - &mut self, - ) -> Option<( - TxnMsg, - oneshot::Sender, - Option>, - )> { + pub async fn recv(&mut self) -> Option<(TxnMsg, oneshot::Sender)> { let TxnMsgWithPermits { txn_msg, notificator, - epoch_notificator, permit_value: permits, } = self.rx.recv().await?; @@ -164,6 +152,6 @@ impl Receiver { self.permits.add_permits(permits); } - Some((txn_msg, notificator, epoch_notificator)) + Some((txn_msg, notificator)) } } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index eefe43ff41a86..e1d7f79b2855e 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -2021,16 +2021,21 @@ fn bind_webhook_info( let WebhookSourceInfo { secret_ref, signature_expr, + wait_for_persistence, } = webhook_info; // validate secret_ref let db_name = &session.database(); let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, secret_ref.secret_name.clone())?; - // let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?; + let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?; let pb_secret_ref = PbSecretRef { - secret_id: 0, - ref_as: PbRefAsType::Text.into(), + secret_id: secret_catalog.id.secret_id(), + ref_as: match secret_ref.ref_as { + SecretRefAsType::Text => PbRefAsType::Text, + SecretRefAsType::File => PbRefAsType::File, + } + .into(), }; let secure_compare_context = SecureCompareContext { @@ -2054,6 +2059,7 @@ fn bind_webhook_info( let pb_webhook_info = PbWebhookSourceInfo { secret_ref: Some(pb_secret_ref), signature_expr: Some(expr.to_expr_proto()), + wait_for_persistence, }; Ok(pb_webhook_info) diff --git a/src/frontend/src/handler/fast_insert.rs b/src/frontend/src/handler/fast_insert.rs deleted file mode 100644 index c8df8c40a2ba3..0000000000000 --- a/src/frontend/src/handler/fast_insert.rs +++ /dev/null @@ -1,95 +0,0 @@ -// Copyright 2024 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 std::collections::HashSet; -use std::sync::Arc; - -use pgwire::pg_response::StatementType; -use risingwave_common::error::BoxedError; -use risingwave_pb::batch_plan::{FastInsertNode, PlanFragment}; -use risingwave_sqlparser::ast::Statement; -use tokio::sync::mpsc; - -use crate::catalog::TableId; -use crate::error::Result; -use crate::handler::query::{ - gen_batch_plan_by_statement, gen_batch_plan_fragmenter, BatchPlanFragmenterResult, -}; -use crate::handler::HandlerArgs; -use crate::optimizer::OptimizerContext; -use crate::scheduler::plan_fragmenter::Query; -use crate::scheduler::FastInsertExecution; -use crate::session::SessionImpl; - -// pub async fn handle_fast_insert( -// handler_args: HandlerArgs, -// node: FastInsertNode, -// ) -> Result { -// let session = handler_args.session.clone(); - -// // Acquire the write guard for DML statements. -// // session.txn_write_guard()?; - -// gen_fast_insert_plan_inner(session, query, &read_storage_tables) -// } - -// fn gen_fast_insert_plan_inner( -// session: Arc, -// query: Query, -// read_storage_tables: &HashSet, -// ) -> Result { -// let front_env = session.env(); -// let snapshot = session.pinned_snapshot(); - -// // TODO: Passing sql here -// let execution = FastInsertExecution::new( -// query, -// front_env.clone(), -// snapshot.support_barrier_read(), -// snapshot.batch_query_epoch(read_storage_tables)?, -// session, -// ); - -// // Ok(execution.gen_plan()?) -// Ok(execution) -// } - -// pub async fn run_fast_insert(plan: PlanFragment) -> Result<()> { -// let compute_runtime = self.front_env.compute_runtime(); -// let (sender, mut receiver) = mpsc::channel(10); -// // let shutdown_rx = self.shutdown_rx().clone(); -// let sender1 = sender.clone(); -// let exec = async move { -// let mut data_stream = self.run().map(|r| r.map_err(|e| Box::new(e) as BoxedError)); -// while let Some(r) = data_stream.next().await { -// // append a query cancelled error if the query is cancelled. -// // if r.is_err() && shutdown_rx.is_cancelled() { -// // r = Err(Box::new(SchedulerError::QueryCancelled( -// // "Cancelled by user".to_string(), -// // )) as BoxedError); -// // } -// if sender1.send(r).await.is_err() { -// tracing::info!("Receiver closed."); -// return; -// } -// } -// }; - -// compute_runtime.spawn(exec); - -// while let Some(result) = receiver.recv().await { -// result?; -// } -// Ok(()) -// } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 1eb5ec49a1778..37de2e604f864 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -91,7 +91,6 @@ pub mod drop_user; mod drop_view; pub mod explain; pub mod extended_handle; -pub mod fast_insert; pub mod fetch_cursor; mod flush; pub mod handle_privilege; diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 7cde172ad3f0d..2a26306575275 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -286,8 +286,6 @@ fn gen_batch_query_plan( QueryMode::Distributed => logical.gen_batch_distributed_plan()?, }; - // println!("WKXLOG Generated query plan: {:?}", physical); - Ok(BatchQueryPlanResult { plan: physical, query_mode, diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 1389da80dffc6..58afb28e3e8ed 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -13,48 +13,24 @@ // limitations under the License. //! Local execution for batch query. -use std::collections::HashMap; -use std::sync::atomic::{AtomicU32, Ordering}; use std::sync::Arc; use anyhow::anyhow; -use futures::stream::BoxStream; -use futures::{FutureExt, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; -use pgwire::pg_server::SessionId; use risingwave_batch::error::BatchError; -use risingwave_batch::executor::ExecutorBuilder; -use risingwave_batch::task::{ShutdownToken, TaskId}; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; -use risingwave_common::array::DataChunk; -use risingwave_common::bail; -use risingwave_common::error::BoxedError; use risingwave_common::hash::WorkerSlotMapping; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::tracing::{InstrumentStream, TracingContext}; -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, FastInsertNode, LocalExecutePlan, PbTaskId, PlanFragment, - PlanNode as PbPlanNode, TaskOutputId, -}; -use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; +use risingwave_pb::batch_plan::FastInsertNode; +use risingwave_pb::common::WorkerNode; use risingwave_pb::task_service::FastInsertRequest; -use tracing::debug; -use super::plan_fragmenter::{PartitionInfo, QueryStage, QueryStageRef}; -use crate::catalog::{FragmentId, TableId}; -use crate::error::RwError; -use crate::optimizer::plan_node::PlanNodeType; -use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; -use crate::scheduler::task_context::FrontendBatchTaskContext; +use crate::catalog::TableId; use crate::scheduler::{SchedulerError, SchedulerResult}; use crate::session::{FrontendEnv, SessionImpl}; pub struct FastInsertExecution { fast_insert_node: FastInsertNode, + wait_for_persistence: bool, front_env: FrontendEnv, session: Arc, worker_node_manager: WorkerNodeSelector, @@ -63,6 +39,7 @@ pub struct FastInsertExecution { impl FastInsertExecution { pub fn new( fast_insert_node: FastInsertNode, + wait_for_persistence: bool, front_env: FrontendEnv, session: Arc, ) -> Self { @@ -71,6 +48,7 @@ impl FastInsertExecution { Self { fast_insert_node, + wait_for_persistence, front_env, session, worker_node_manager, @@ -86,6 +64,7 @@ impl FastInsertExecution { let client = self.session.env().client_pool().get(&workers).await?; let request = FastInsertRequest { fast_insert_node: Some(self.fast_insert_node), + wait_epoch: self.wait_for_persistence, }; let response = client.fast_insert(request).await?; println!("WKXLOG response: {:?}", response); diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index b063ef8903f1b..6841685aa0cd1 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -28,7 +28,6 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::{DataType, JsonbVal, Scalar}; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; -use risingwave_sqlparser::ast::{Expr, ObjectName}; use tokio::net::TcpListener; use tower::ServiceBuilder; use tower_http::add_extension::AddExtensionLayer; @@ -36,8 +35,7 @@ use tower_http::compression::CompressionLayer; use tower_http::cors::{self, CorsLayer}; // use crate::handler::fast_insert::handle_fast_insert; -use crate::handler::{handle, HandlerArgs}; -use crate::scheduler::{FastInsertExecution, FrontendBatchTaskContext}; +use crate::scheduler::FastInsertExecution; use crate::webhook::utils::{err, Result}; mod utils; @@ -47,7 +45,7 @@ pub type Service = Arc; const USER: &str = "root"; #[derive(Clone)] -pub struct TableWebhookContext { +pub struct FastInsertContext { pub webhook_source_info: WebhookSourceInfo, pub fast_insert_node: FastInsertNode, } @@ -60,11 +58,9 @@ pub struct WebhookService { pub(super) mod handlers { use std::net::Ipv4Addr; - use iceberg::table::Table; use jsonbb::Value; use pgwire::pg_server::Session; use risingwave_common::array::JsonbArrayBuilder; - use risingwave_common::catalog::TableId; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; use risingwave_pb::plan_common::DefaultColumns; @@ -88,10 +84,6 @@ pub(super) mod handlers { // Can be any address, we use the port of meta to indicate that it's a internal request. let dummy_addr = Address::Tcp(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691)); - // println!( - // "WKXLOG Received webhook request for {}/{}/{}", - // database, schema, table - // ); // TODO(kexiang): optimize this // get a session object for the corresponding database let session = session_mgr @@ -106,8 +98,7 @@ pub(super) mod handlers { ) })?; - // println!("WKXLOG session created"); - let TableWebhookContext { + let FastInsertContext { webhook_source_info, mut fast_insert_node, } = acquire_table_info(&session, &database, &schema, &table).await?; @@ -115,17 +106,14 @@ pub(super) mod handlers { let WebhookSourceInfo { signature_expr, secret_ref, + wait_for_persistence, .. } = webhook_source_info; - // println!("WKXLOG webhook source info: {:?}", secret_ref); - // let secret_string = LocalSecretManager::global() - // .fill_secret(secret_ref.unwrap()) - // .map_err(|e| err(e, StatusCode::NOT_FOUND))?; - - let secret_string = String::from("TEST_WEBHOOK"); + let secret_string = LocalSecretManager::global() + .fill_secret(secret_ref.unwrap()) + .map_err(|e| err(e, StatusCode::NOT_FOUND))?; - // println!("WKXLOG secret string: {:?}", secret_string); // Once limitation here is that the key is no longer case-insensitive, users must user the lowercase key when defining the webhook source table. let headers_jsonb = header_map_to_json(&headers); @@ -137,8 +125,6 @@ pub(super) mod handlers { ) .await?; - // println!("WKXLOG is_valid: {:?}", is_valid); - if !is_valid { return Err(err( anyhow!("Signature verification failed"), @@ -146,13 +132,6 @@ pub(super) mod handlers { )); } - // let payload = String::from_utf8(body.to_vec()).map_err(|e| { - // err( - // anyhow!(e).context("Failed to parse body"), - // StatusCode::UNPROCESSABLE_ENTITY, - // ) - // })?; - let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); // TODO(kexiang): handle errors let json_value = Value::from_text(&body.to_vec()).unwrap(); @@ -164,8 +143,12 @@ pub(super) mod handlers { let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); // let context = FrontendBatchTaskContext::new(session.clone()); - let execution = - FastInsertExecution::new(fast_insert_node, session.env().clone(), session.clone()); + let execution = FastInsertExecution::new( + fast_insert_node, + wait_for_persistence, + session.env().clone(), + session.clone(), + ); let res = execution.my_execute().await.unwrap(); Ok(()) @@ -176,9 +159,7 @@ pub(super) mod handlers { database: &String, schema: &String, table: &String, - ) -> Result { - // println!("WKXLOG session created"); - + ) -> Result { let search_path = session.config().search_path(); let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); @@ -210,11 +191,10 @@ pub(super) mod handlers { session_id: session.id().0 as u32, }; - let table_webhook_context = TableWebhookContext { + Ok(FastInsertContext { webhook_source_info, fast_insert_node, - }; - Ok(table_webhook_context) + }) } } diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index 5efbc2003cfb1..eaecb276f1218 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -874,4 +874,5 @@ impl fmt::Display for ReferentialAction { pub struct WebhookSourceInfo { pub secret_ref: SecretRefValue, pub signature_expr: Expr, + pub wait_for_persistence: bool, } diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 843dd566212bf..e3d52c1c73521 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -40,6 +40,8 @@ use crate::{impl_parse_to, parser_v2}; pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector"; pub(crate) const WEBHOOK_CONNECTOR: &str = "webhook"; +const WEBHOOK_WAIT_FOR_PERSISTENCE: &str = "webhook.wait_for_persistence"; + #[derive(Debug, Clone, PartialEq)] pub enum ParserError { TokenizerError(String), @@ -2603,6 +2605,12 @@ impl Parser<'_> { parser_err!("VALIDATE is only supported for tables created with webhook source"); } + let wait_for_persistence = with_options + .iter() + .find(|&opt| opt.name.real_value() == WEBHOOK_WAIT_FOR_PERSISTENCE) + .map(|opt| opt.value.to_string().eq_ignore_ascii_case("true")) + .unwrap_or(true); + self.expect_keyword(Keyword::SECRET)?; let secret_ref = self.parse_secret_ref()?; if secret_ref.ref_as == SecretRefAsType::File { @@ -2615,6 +2623,7 @@ impl Parser<'_> { Some(WebhookSourceInfo { secret_ref, signature_expr, + wait_for_persistence, }) } else { None diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index fbc18c40c7800..115145c032a92 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -124,10 +124,7 @@ impl DmlExecutor { // Merge the two streams using `StreamReaderWithPause` because when we receive a pause // barrier, we should stop receiving the data from DML. We poll data from the two streams in // a round robin way. - let mut stream = - StreamReaderWithPause::>)>::new( - upstream, reader, - ); + let mut stream = StreamReaderWithPause::::new(upstream, reader); // If the first barrier requires us to pause on startup, pause the stream. if barrier.is_pause_on_startup() { @@ -202,7 +199,7 @@ impl DmlExecutor { } yield msg; } - Either::Right((txn_msg, epoch_notifier)) => { + Either::Right(txn_msg) => { // Batch data. match txn_msg { TxnMsg::Begin(txn_id) => { @@ -212,7 +209,7 @@ impl DmlExecutor { panic!("Transaction id collision txn_id = {}.", txn_id) }); } - TxnMsg::End(txn_id) => { + TxnMsg::End(txn_id, epoch_notifier) => { if let Some(sender) = epoch_notifier { let _ = sender.send(epoch); } @@ -314,34 +311,31 @@ impl Execute for DmlExecutor { } } -type BoxTxnMessageStream = - BoxStream<'static, risingwave_dml::error::Result<(TxnMsg, Option>)>>; -#[try_stream(ok = (TxnMsg, Option>), error = risingwave_dml::error::DmlError)] +type BoxTxnMessageStream = BoxStream<'static, risingwave_dml::error::Result>; +#[try_stream(ok = TxnMsg, error = risingwave_dml::error::DmlError)] async fn apply_dml_rate_limit( stream: BoxTxnMessageStream, rate_limiter: Arc, ) { #[for_await] for txn_msg in stream { - let (txn_msg, epoch_notifier) = txn_msg?; - match txn_msg { + match txn_msg? { TxnMsg::Begin(txn_id) => { - yield (TxnMsg::Begin(txn_id), epoch_notifier); + yield TxnMsg::Begin(txn_id); } - TxnMsg::End(txn_id) => { - yield (TxnMsg::End(txn_id), epoch_notifier); + TxnMsg::End(txn_id, epoch_notifier) => { + yield TxnMsg::End(txn_id, epoch_notifier); } TxnMsg::Rollback(txn_id) => { - yield (TxnMsg::Rollback(txn_id), epoch_notifier); + yield TxnMsg::Rollback(txn_id); } TxnMsg::Data(txn_id, chunk) => { let chunk_size = chunk.capacity(); if chunk_size == 0 { // empty chunk - yield (TxnMsg::Data(txn_id, chunk), None); + yield TxnMsg::Data(txn_id, chunk); continue; } - let rate_limit = loop { match rate_limiter.rate_limit() { RateLimit::Pause => rate_limiter.wait(0).await, From 5c35519375cfbc321019c66d2248b485729beccf Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sat, 18 Jan 2025 22:11:45 -0500 Subject: [PATCH 03/27] check response --- proto/task_service.proto | 9 +++++++-- .../executors/src/executor/fast_insert.rs | 1 - src/batch/src/rpc/service/task_service.rs | 18 +++++++++++++---- .../src/scheduler/distributed/query.rs | 5 ----- src/frontend/src/scheduler/fast_insert.rs | 7 +++---- src/frontend/src/scheduler/local.rs | 4 ---- src/frontend/src/webhook/mod.rs | 20 ++++++++++++++----- 7 files changed, 39 insertions(+), 25 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index 2abf728a78c0c..9529eaca3ffaf 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -50,8 +50,13 @@ message FastInsertRequest { } message FastInsertResponse { - // Optional error message for failed task. - string error_message = 1; + enum Status { + UNSPECIFIED = 0; + SUCCEEDED = 1; + DML_FAILED = 2; + } + Status status = 1; + string error_message = 2; } message CancelTaskRequest { diff --git a/src/batch/executors/src/executor/fast_insert.rs b/src/batch/executors/src/executor/fast_insert.rs index 069579ba38c74..67e5c60bbcfde 100644 --- a/src/batch/executors/src/executor/fast_insert.rs +++ b/src/batch/executors/src/executor/fast_insert.rs @@ -209,7 +209,6 @@ mod tests { let json_value = json!(header_map); let jsonb_val = JsonbVal::from(json_value); - // Add 4 ListValues to ArrayBuilder builder.append(Some(jsonb_val.as_scalar_ref())); // Use builder to obtain a single (List) column DataChunk diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index d1dc499c22e0e..faacabb31e455 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -18,8 +18,8 @@ use risingwave_common::util::tracing::TracingContext; use risingwave_pb::batch_plan::{FastInsertNode, TaskOutputId}; use risingwave_pb::task_service::task_service_server::TaskService; use risingwave_pb::task_service::{ - CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, FastInsertRequest, - FastInsertResponse, GetDataResponse, TaskInfoResponse, + fast_insert_response, CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, + FastInsertRequest, FastInsertResponse, GetDataResponse, TaskInfoResponse, }; use risingwave_storage::dispatch_state_store; use thiserror_ext::AsReport; @@ -132,9 +132,19 @@ impl TaskService for BatchServiceImpl { let res = self.do_fast_insert(insert_node, req.wait_epoch).await; match res { Ok(_) => Ok(Response::new(FastInsertResponse { - error_message: String::from("success"), + status: fast_insert_response::Status::Succeeded.into(), + error_message: "".to_string(), })), - Err(e) => Err(e.into()), + Err(e) => match e { + BatchError::Dml(e) => Ok(Response::new(FastInsertResponse { + status: fast_insert_response::Status::DmlFailed.into(), + error_message: format!("{}", e.as_report()), + })), + _ => { + error!(error = %e.as_report(), "failed to fast insert"); + Err(e.into()) + } + }, } } } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 5184ddeed333c..62cf4cca08905 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -232,11 +232,6 @@ impl QueryExecution { compute_client_pool: ComputeClientPoolRef, catalog_reader: CatalogReader, ) -> HashMap> { - // println!( - // "WKXLOG self.query.stage_graph.stages.len(): {}, self.query.stage_graph.stages: {:?}", - // self.query.stage_graph.stages.len(), - // self.query.stage_graph.stages - // ); let mut stage_executions: HashMap> = HashMap::with_capacity(self.query.stage_graph.stages.len()); diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 58afb28e3e8ed..8df60b431a505 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -22,7 +22,7 @@ use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::hash::WorkerSlotMapping; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::common::WorkerNode; -use risingwave_pb::task_service::FastInsertRequest; +use risingwave_pb::task_service::{FastInsertRequest, FastInsertResponse}; use crate::catalog::TableId; use crate::scheduler::{SchedulerError, SchedulerResult}; @@ -55,7 +55,7 @@ impl FastInsertExecution { } } - pub async fn my_execute(self) -> SchedulerResult<()> { + pub async fn my_execute(self) -> SchedulerResult { let workers = self.choose_worker( &TableId::new(self.fast_insert_node.table_id), self.fast_insert_node.session_id, @@ -67,8 +67,7 @@ impl FastInsertExecution { wait_epoch: self.wait_for_persistence, }; let response = client.fast_insert(request).await?; - println!("WKXLOG response: {:?}", response); - Ok(()) + Ok(response) } #[inline(always)] diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 7ac156e5c9587..570cff8504a5a 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -108,8 +108,6 @@ impl LocalQueryExecution { let plan_fragment = self.create_plan_fragment()?; let plan_node = plan_fragment.root.unwrap(); - // println!("WKXLOG plan_node: {:?}", plan_node); - let executor = ExecutorBuilder::new( &plan_node, &task_id, @@ -123,8 +121,6 @@ impl LocalQueryExecution { // Release potential large object in Query and PlanNode early. drop(self); - // println!("WKXLOG executor: {:?}", executor); - #[for_await] for chunk in executor.execute() { yield chunk?; diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 6841685aa0cd1..b25b6c9fd889a 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -64,6 +64,7 @@ pub(super) mod handlers { use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; use risingwave_pb::plan_common::DefaultColumns; + use risingwave_pb::task_service::fast_insert_response; // use risingwave_sqlparser::ast::{Query, SetExpr, Statement, Value, Values}; use utils::{header_map_to_json, verify_signature}; @@ -133,10 +134,13 @@ pub(super) mod handlers { } let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); - // TODO(kexiang): handle errors - let json_value = Value::from_text(&body.to_vec()).unwrap(); + let json_value = Value::from_text(&body.to_vec()).map_err(|e| { + err( + anyhow!(e).context("Failed to parse body"), + StatusCode::UNPROCESSABLE_ENTITY, + ) + })?; let jsonb_val = JsonbVal::from(json_value); - // Add 4 ListValues to ArrayBuilder builder.append(Some(jsonb_val.as_scalar_ref())); // Use builder to obtain a single (List) column DataChunk @@ -150,8 +154,14 @@ pub(super) mod handlers { session.clone(), ); let res = execution.my_execute().await.unwrap(); - - Ok(()) + if res.status == fast_insert_response::Status::Succeeded as i32 { + Ok(()) + } else { + Err(err( + anyhow!("Failed to fast insert: {}", res.error_message), + StatusCode::INTERNAL_SERVER_ERROR, + )) + } } async fn acquire_table_info( From 3e861cda74213101ba2f5b5efd077e8c01e3b9fa Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 17:39:00 -0500 Subject: [PATCH 04/27] fix --- src/batch/executors/src/executor.rs | 2 -- src/batch/{executors => }/src/executor/fast_insert.rs | 0 src/batch/src/executor/mod.rs | 2 ++ src/stream/src/executor/dml.rs | 1 - 4 files changed, 2 insertions(+), 3 deletions(-) rename src/batch/{executors => }/src/executor/fast_insert.rs (100%) diff --git a/src/batch/executors/src/executor.rs b/src/batch/executors/src/executor.rs index 80f29ecfd35b5..a2b67f06924ae 100644 --- a/src/batch/executors/src/executor.rs +++ b/src/batch/executors/src/executor.rs @@ -18,7 +18,6 @@ pub mod aggregation; mod azblob_file_scan; mod delete; mod expand; -mod fast_insert; mod filter; mod gcs_file_scan; mod generic_exchange; @@ -59,7 +58,6 @@ pub use filter::*; use gcs_file_scan::GcsFileScanExecutorBuilder; pub use generic_exchange::*; pub use group_top_n::*; -pub use fast_insert::*; pub use hash_agg::*; pub use hop_window::*; pub use iceberg_scan::*; diff --git a/src/batch/executors/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs similarity index 100% rename from src/batch/executors/src/executor/fast_insert.rs rename to src/batch/src/executor/fast_insert.rs diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index e66c6c6c08bd3..2c08961185319 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod fast_insert; mod managed; pub mod test_utils; @@ -20,6 +21,7 @@ use std::sync::Arc; use anyhow::Context; use async_recursion::async_recursion; +pub use fast_insert::*; use futures::future::BoxFuture; use futures::stream::BoxStream; pub use managed::*; diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 115145c032a92..ab31c7dd3a2eb 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -20,7 +20,6 @@ use futures::TryStreamExt; use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; -use risingwave_common::util::epoch::Epoch; use risingwave_common_rate_limit::{MonitoredRateLimiter, RateLimit, RateLimiter}; use risingwave_dml::dml_manager::DmlManagerRef; use risingwave_expr::codegen::BoxStream; From 7c0b5c35cf97819de9f59d0bc7de034cb669d0b3 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 18:08:05 -0500 Subject: [PATCH 05/27] fix --- Cargo.lock | 1 + proto/batch_plan.proto | 5 ++-- proto/task_service.proto | 38 +++++++++++------------- src/batch/Cargo.toml | 1 + src/batch/src/executor/fast_insert.rs | 42 ++------------------------- src/frontend/src/webhook/mod.rs | 5 ---- 6 files changed, 23 insertions(+), 69 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8595ee0ba61f6..cafb9036b9f24 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10358,6 +10358,7 @@ name = "risingwave_batch" version = "2.3.0-alpha" dependencies = [ "anyhow", + "assert_matches", "async-recursion", "async-trait", "either", diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index bfde254f4c770..1012adaf62ba9 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -203,15 +203,14 @@ message FastInsertNode { repeated uint32 column_indices = 3; data.DataChunk data_chunk = 4; - plan_common.DefaultColumns default_columns = 5; // An optional field and will be `None` for tables without user-defined pk. // The `BatchInsertExecutor` should add a column with NULL value which will // be filled in streaming. - optional uint32 row_id_index = 6; + optional uint32 row_id_index = 5; // Session id is used to ensure that dml data from the same session should be // sent to a fixed worker node and channel. - uint32 session_id = 7; + uint32 session_id = 6; } message DeleteNode { diff --git a/proto/task_service.proto b/proto/task_service.proto index 9529eaca3ffaf..6fc6d22a28d57 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -47,6 +47,7 @@ message CreateTaskRequest { message FastInsertRequest { bool wait_epoch = 1; batch_plan.FastInsertNode fast_insert_node = 2; + // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. } message FastInsertResponse { @@ -59,21 +60,13 @@ message FastInsertResponse { string error_message = 2; } -message CancelTaskRequest { - batch_plan.TaskId task_id = 1; -} +message CancelTaskRequest { batch_plan.TaskId task_id = 1; } -message CancelTaskResponse { - common.Status status = 1; -} +message CancelTaskResponse { common.Status status = 1; } -message GetTaskInfoRequest { - batch_plan.TaskId task_id = 1; -} +message GetTaskInfoRequest { batch_plan.TaskId task_id = 1; } -message GetDataResponse { - data.DataChunk record_batch = 2; -} +message GetDataResponse { data.DataChunk record_batch = 2; } message ExecuteRequest { batch_plan.TaskId task_id = 1; @@ -85,15 +78,14 @@ message ExecuteRequest { service TaskService { rpc CreateTask(CreateTaskRequest) returns (stream TaskInfoResponse); - // Cancel an already-died (self execution-failure, previous aborted, completed) task will still succeed. + // Cancel an already-died (self execution-failure, previous aborted, + // completed) task will still succeed. rpc CancelTask(CancelTaskRequest) returns (CancelTaskResponse); rpc Execute(ExecuteRequest) returns (stream GetDataResponse); rpc FastInsert(FastInsertRequest) returns (FastInsertResponse); } -message GetDataRequest { - batch_plan.TaskOutputId task_output_id = 1; -} +message GetDataRequest { batch_plan.TaskOutputId task_output_id = 1; } // The structure for permit-based back-pressure. message Permits { @@ -115,18 +107,22 @@ message GetStreamRequest { } oneof value { - // The first message, which tells the upstream which channel this exchange stream is for. + // The first message, which tells the upstream which channel this exchange + // stream is for. Get get = 1; - // The following messages, which adds the permits back to the upstream to achieve back-pressure. + // The following messages, which adds the permits back to the upstream to + // achieve back-pressure. Permits add_permits = 2; } } message GetStreamResponse { stream_plan.StreamMessage message = 1; - // The number of permits acquired for this message, which should be sent back to the upstream with `add_permits`. - // In theory, this can also be derived from the message itself by the receiver. Here we send it explicitly to - // avoid any sense of inconsistency for the derivation logic, so the receiver can simply send it back verbatim. + // The number of permits acquired for this message, which should be sent back + // to the upstream with `add_permits`. In theory, this can also be derived + // from the message itself by the receiver. Here we send it explicitly to + // avoid any sense of inconsistency for the derivation logic, so the receiver + // can simply send it back verbatim. Permits permits = 2; } diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 56491055cfacd..b83adaa2104fe 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -9,6 +9,7 @@ repository = { workspace = true } [dependencies] anyhow = "1" +assert_matches = "1" async-recursion = "1" async-trait = "0.1" either = "1" diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index 67e5c60bbcfde..69b9943029ef3 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -15,7 +15,6 @@ use std::iter::repeat; use std::sync::Arc; -use anyhow::Context; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op, SerialArray, StreamChunk}; use risingwave_common::catalog::{Field, Schema, TableId, TableVersionId}; @@ -23,12 +22,11 @@ use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::types::DataType; use risingwave_common::util::epoch::Epoch; use risingwave_dml::dml_manager::DmlManagerRef; -use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::FastInsertNode; -use risingwave_pb::plan_common::IndexAndExpr; use crate::error::Result; +/// A fast insert executor spacially designed for non-pgwire inserts like websockets and webhooks. pub struct FastInsertExecutor { /// Target table id. table_id: TableId, @@ -36,9 +34,6 @@ pub struct FastInsertExecutor { dml_manager: DmlManagerRef, column_indices: Vec, - // TODO(Kexiang): get rid of it? - sorted_default_columns: Vec<(usize, BoxedExpression)>, - row_id_index: Option, txn_id: TxnId, session_id: u32, @@ -55,24 +50,6 @@ impl FastInsertExecutor { .iter() .map(|&i| i as usize) .collect(); - let sorted_default_columns = if let Some(default_columns) = &insert_node.default_columns { - let mut default_columns = default_columns - .get_default_columns() - .iter() - .cloned() - .map(|IndexAndExpr { index: i, expr: e }| { - Ok(( - i as usize, - build_from_prost(&e.context("expression is None")?) - .context("failed to build expression")?, - )) - }) - .collect::>>()?; - default_columns.sort_unstable_by_key(|(i, _)| *i); - default_columns - } else { - vec![] - }; let mut schema = Schema::new(vec![Field::unnamed(DataType::Jsonb)]); schema.fields.push(Field::unnamed(DataType::Serial)); // row_id column let data_chunk_pb = insert_node @@ -85,7 +62,6 @@ impl FastInsertExecutor { insert_node.table_version_id, dml_manager, column_indices, - sorted_default_columns, insert_node.row_id_index.as_ref().map(|index| *index as _), insert_node.session_id, ), @@ -99,7 +75,6 @@ impl FastInsertExecutor { table_version_id: TableVersionId, dml_manager: DmlManagerRef, column_indices: Vec, - sorted_default_columns: Vec<(usize, BoxedExpression)>, row_id_index: Option, session_id: u32, ) -> Self { @@ -109,7 +84,6 @@ impl FastInsertExecutor { table_version_id, dml_manager, column_indices, - sorted_default_columns, row_id_index, txn_id, session_id, @@ -138,7 +112,6 @@ impl FastInsertExecutor { .enumerate() .map(|(i, idx)| (*idx, columns[i].clone())) .collect_vec(); - ordered_columns.reserve(ordered_columns.len() + self.sorted_default_columns.len()); ordered_columns.sort_unstable_by_key(|(idx, _)| *idx); columns = ordered_columns @@ -174,14 +147,12 @@ mod tests { use std::ops::Bound; use assert_matches::assert_matches; - use foyer::CacheHint; use futures::StreamExt; use risingwave_common::array::{Array, JsonbArrayBuilder}; use risingwave_common::catalog::{ColumnDesc, ColumnId, INITIAL_TABLE_VERSION_ID}; use risingwave_common::transaction::transaction_message::TxnMsg; use risingwave_common::types::JsonbVal; use risingwave_dml::dml_manager::DmlManager; - use risingwave_storage::hummock::CachePolicy; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::store::{ReadOptions, StateStoreReadExt}; use serde_json::json; @@ -237,7 +208,6 @@ mod tests { INITIAL_TABLE_VERSION_ID, dml_manager, vec![0], // Ignoring insertion order - vec![], row_id_index, 0, )); @@ -261,15 +231,7 @@ mod tests { let epoch = u64::MAX; let full_range = (Bound::Unbounded, Bound::Unbounded); let store_content = store - .scan( - full_range, - epoch, - None, - ReadOptions { - cache_policy: CachePolicy::Fill(CacheHint::Normal), - ..Default::default() - }, - ) + .scan(full_range, epoch, None, ReadOptions::default()) .await?; assert!(store_content.is_empty()); diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index b25b6c9fd889a..1b46b0610d95b 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -63,9 +63,7 @@ pub(super) mod handlers { use risingwave_common::array::JsonbArrayBuilder; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; - use risingwave_pb::plan_common::DefaultColumns; use risingwave_pb::task_service::fast_insert_response; - // use risingwave_sqlparser::ast::{Query, SetExpr, Statement, Value, Values}; use utils::{header_map_to_json, verify_signature}; use super::*; @@ -194,9 +192,6 @@ pub(super) mod handlers { table_version_id: table_catalog.version_id().expect("table must be versioned"), column_indices: vec![0], data_chunk: None, - default_columns: Some(DefaultColumns { - default_columns: vec![], - }), row_id_index: Some(1), session_id: session.id().0 as u32, }; From b808d69a2bbebeed2ffc8218e37dc94721c42e44 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 18:15:38 -0500 Subject: [PATCH 06/27] fix --- proto/catalog.proto | 1 + proto/task_service.proto | 46 ++-- .../src/scheduler/distributed/fast_insert.rs | 247 ------------------ src/frontend/src/scheduler/mod.rs | 3 +- src/frontend/src/scheduler/plan_fragmenter.rs | 1 - src/frontend/src/webhook/mod.rs | 3 +- 6 files changed, 29 insertions(+), 272 deletions(-) delete mode 100644 src/frontend/src/scheduler/distributed/fast_insert.rs diff --git a/proto/catalog.proto b/proto/catalog.proto index 7fea9c56adbd6..f5d47d3fc9da2 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -100,6 +100,7 @@ message StreamSourceInfo { message WebhookSourceInfo { secret.SecretRef secret_ref = 1; expr.ExprNode signature_expr = 2; + // Return until the data is persisted in the storage layer or not. Default is true. bool wait_for_persistence = 3; } diff --git a/proto/task_service.proto b/proto/task_service.proto index 6fc6d22a28d57..c921e9c8a4e3a 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -44,6 +44,23 @@ message CreateTaskRequest { plan_common.ExprContext expr_context = 5; } + +message CancelTaskRequest { + batch_plan.TaskId task_id = 1; +} + +message CancelTaskResponse { + common.Status status = 1; +} + +message GetTaskInfoRequest { + batch_plan.TaskId task_id = 1; +} + +message GetDataResponse { + data.DataChunk record_batch = 2; +} + message FastInsertRequest { bool wait_epoch = 1; batch_plan.FastInsertNode fast_insert_node = 2; @@ -60,14 +77,6 @@ message FastInsertResponse { string error_message = 2; } -message CancelTaskRequest { batch_plan.TaskId task_id = 1; } - -message CancelTaskResponse { common.Status status = 1; } - -message GetTaskInfoRequest { batch_plan.TaskId task_id = 1; } - -message GetDataResponse { data.DataChunk record_batch = 2; } - message ExecuteRequest { batch_plan.TaskId task_id = 1; batch_plan.PlanFragment plan = 2; @@ -78,14 +87,15 @@ message ExecuteRequest { service TaskService { rpc CreateTask(CreateTaskRequest) returns (stream TaskInfoResponse); - // Cancel an already-died (self execution-failure, previous aborted, - // completed) task will still succeed. + // Cancel an already-died (self execution-failure, previous aborted, completed) task will still succeed. rpc CancelTask(CancelTaskRequest) returns (CancelTaskResponse); rpc Execute(ExecuteRequest) returns (stream GetDataResponse); rpc FastInsert(FastInsertRequest) returns (FastInsertResponse); } -message GetDataRequest { batch_plan.TaskOutputId task_output_id = 1; } +message GetDataRequest { + batch_plan.TaskOutputId task_output_id = 1; +} // The structure for permit-based back-pressure. message Permits { @@ -107,22 +117,18 @@ message GetStreamRequest { } oneof value { - // The first message, which tells the upstream which channel this exchange - // stream is for. + // The first message, which tells the upstream which channel this exchange stream is for. Get get = 1; - // The following messages, which adds the permits back to the upstream to - // achieve back-pressure. + // The following messages, which adds the permits back to the upstream to achieve back-pressure. Permits add_permits = 2; } } message GetStreamResponse { stream_plan.StreamMessage message = 1; - // The number of permits acquired for this message, which should be sent back - // to the upstream with `add_permits`. In theory, this can also be derived - // from the message itself by the receiver. Here we send it explicitly to - // avoid any sense of inconsistency for the derivation logic, so the receiver - // can simply send it back verbatim. + // The number of permits acquired for this message, which should be sent back to the upstream with `add_permits`. + // In theory, this can also be derived from the message itself by the receiver. Here we send it explicitly to + // avoid any sense of inconsistency for the derivation logic, so the receiver can simply send it back verbatim. Permits permits = 2; } diff --git a/src/frontend/src/scheduler/distributed/fast_insert.rs b/src/frontend/src/scheduler/distributed/fast_insert.rs deleted file mode 100644 index 0a3fcfd62ee8f..0000000000000 --- a/src/frontend/src/scheduler/distributed/fast_insert.rs +++ /dev/null @@ -1,247 +0,0 @@ -// Copyright 2024 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 std::collections::HashMap; -use std::fmt::{Debug, Formatter}; -use std::mem; -use std::sync::Arc; - -use anyhow::Context; -use futures::executor::block_on; -use petgraph::dot::{Config, Dot}; -use petgraph::Graph; -use pgwire::pg_server::SessionId; -use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; -use risingwave_common::array::DataChunk; -use risingwave_pb::batch_plan::{TaskId as PbTaskId, TaskOutputId as PbTaskOutputId}; -use risingwave_pb::common::{BatchQueryEpoch, HostAddress}; -use risingwave_rpc_client::ComputeClientPoolRef; -use thiserror_ext::AsReport; -use tokio::sync::mpsc::{channel, Receiver, Sender}; -use tokio::sync::{oneshot, RwLock}; -use tokio::task::JoinHandle; -use tracing::{debug, error, info, warn, Instrument}; - -use super::{DistributedQueryMetrics, QueryExecutionInfoRef, QueryResultFetcher, StageEvent}; -use crate::catalog::catalog_service::CatalogReader; -use crate::scheduler::distributed::query::QueryMessage; -use crate::scheduler::distributed::stage::StageEvent::ScheduledRoot; -use crate::scheduler::distributed::StageEvent::Scheduled; -use crate::scheduler::distributed::StageExecution; -use crate::scheduler::plan_fragmenter::{Query, StageId, ROOT_TASK_ID, ROOT_TASK_OUTPUT_ID}; -use crate::scheduler::{ExecutionContextRef, SchedulerError, SchedulerResult}; - -pub struct FastInsertExecution { - query: Arc, - shutdown_tx: Sender, - /// Identified by `process_id`, `secret_key`. Query in the same session should have same key. - pub session_id: SessionId, - /// Permit to execute the query. Once query finishes execution, this is dropped. - pub permit: Option, -} - -impl FastInsertExecution { - #[allow(clippy::too_many_arguments)] - pub fn new( - query: Query, - session_id: SessionId, - permit: Option, - ) -> Self { - let query = Arc::new(query); - let (sender, receiver) = channel(100); - - Self { - query, - shutdown_tx: sender, - session_id, - permit, - } - } - - /// Start execution of this query. - /// Note the two shutdown channel sender and receivers are not dual. - /// One is used for propagate error to `QueryResultFetcher`, one is used for listening on - /// cancel request (from ctrl-c, cli, ui etc). - #[allow(clippy::too_many_arguments)] - pub async fn execute( - self: Arc, - context: ExecutionContextRef, - worker_node_manager: WorkerNodeSelector, - batch_query_epoch: BatchQueryEpoch, - compute_client_pool: ComputeClientPoolRef, - catalog_reader: CatalogReader, - query_execution_info: QueryExecutionInfoRef, - query_metrics: Arc, - ) -> SchedulerResult { - // Start a timer to cancel the query - // let mut timeout_abort_task_handle: Option> = None; - // if let Some(timeout) = context.timeout() { - // let this = self.clone(); - // timeout_abort_task_handle = Some(tokio::spawn(async move { - // tokio::time::sleep(timeout).await; - // warn!( - // "Query {:?} timeout after {} seconds, sending cancel message.", - // this.query.query_id, - // timeout.as_secs(), - // ); - // this.abort(format!("timeout after {} seconds", timeout.as_secs())) - // .await; - // })); - // } - - // Create a oneshot channel for QueryResultFetcher to get failed event. - // let (root_stage_sender, root_stage_receiver) = - // oneshot::channel::>(); - - let runner = QueryRunner { - query: self.query.clone(), - stage_executions, - msg_receiver, - root_stage_sender: Some(root_stage_sender), - scheduled_stages_count: 0, - query_execution_info, - query_metrics, - timeout_abort_task_handle, - }; - - let span = tracing::info_span!( - "distributed_execute", - query_id = self.query.query_id.id, - epoch = ?batch_query_epoch, - ); - - tracing::trace!("Starting query: {:?}", self.query.query_id); - - // Not trace the error here, it will be processed in scheduler. - // tokio::spawn(async move { runner.run().instrument(span).await }); - run_inner(); - // let root_stage = root_stage_receiver - // .await - // .context("Starting query execution failed")??; - - tracing::trace!( - "Received root stage query result fetcher: {:?}, query id: {:?}", - root_stage, - self.query.query_id - ); - - tracing::trace!("Query {:?} started.", self.query.query_id); - Ok(root_stage) - } - - fn run_inner() { - let runner = StageRunner { - epoch: self.epoch, - stage: self.stage.clone(), - worker_node_manager: self.worker_node_manager.clone(), - tasks: self.tasks.clone(), - msg_sender, - children: self.children.clone(), - state: self.state.clone(), - compute_client_pool: self.compute_client_pool.clone(), - catalog_reader: self.catalog_reader.clone(), - ctx: self.ctx.clone(), - }; - - // The channel used for shutdown signal messaging. - let (sender, receiver) = ShutdownToken::new(); - // Fill the shutdown sender. - let mut holder = self.shutdown_tx.write().await; - *holder = Some(sender); - - // Change state before spawn runner. - *s = StageState::Started; - - let span = tracing::info_span!( - "stage", - "otel.name" = format!("Stage {}-{}", self.stage.query_id.id, self.stage.id), - query_id = self.stage.query_id.id, - stage_id = self.stage.id, - ); - self.ctx - .session() - .env() - .compute_runtime() - .spawn(async move { runner.run(receiver).instrument(span).await }); - - tracing::trace!( - "Stage {:?}-{:?} started.", - self.stage.query_id.id, - self.stage.id - ) - } - - fn schedule() { - for id in 0..self.stage.parallelism.unwrap() { - let task_id = PbTaskId { - query_id: self.stage.query_id.id.clone(), - stage_id: self.stage.id, - task_id: id as u64, - }; - let plan_fragment = self.create_plan_fragment(id as u64, None); - let worker = self.choose_worker(&plan_fragment, id, self.stage.dml_table_id)?; - self.call_rpc( - epoch, - worker_node_manager, - task_id, - plan_fragment, - compute_client_pool, - worker, - ); - // futures.push(self.schedule_task(task_id, plan_fragment, worker, expr_context.clone())); - } - } - - fn call_rpc( - epoch: BatchQueryEpoch, - worker_node_manager: WorkerNodeSelector, - task_id: PbTaskId, - plan_fragment: PlanFragment, - compute_client_pool: ComputeClientPoolRef, - worker: Option, - ) { - let mut worker = worker.unwrap_or(worker_node_manager.next_random_worker()?); - let worker_node_addr = worker.host.take().unwrap(); - let compute_client = compute_client_pool - .get_by_addr((&worker_node_addr).into()) - // .inspect_err(|_| self.mask_failed_serving_worker(&worker)) - .map_err(|e| anyhow!(e))?; - - let t_id = task_id.task_id; - - let stream_status: TaskInfoResponse = compute_client - .fast_insert(task_id, plan_fragment, epoch, expr_context) - // .inspect_err(|_| self.mask_failed_serving_worker(&worker)) - .map_err(|e| anyhow!(e))?; - - Ok(stream_status) - } - - // fn mask_failed_serving_worker(&self, worker: &WorkerNode) { - // if !worker.property.as_ref().map_or(false, |p| p.is_serving) { - // return; - // } - // let duration = Duration::from_secs(std::cmp::max( - // self.ctx - // .session - // .env() - // .batch_config() - // .mask_worker_temporary_secs as u64, - // 1, - // )); - // self.worker_node_manager - // .manager - // .mask_worker_node(worker.id, duration); - // } -} diff --git a/src/frontend/src/scheduler/mod.rs b/src/frontend/src/scheduler/mod.rs index 6b863f6cf481d..f3364e78183f9 100644 --- a/src/frontend/src/scheduler/mod.rs +++ b/src/frontend/src/scheduler/mod.rs @@ -35,8 +35,7 @@ pub use local::*; mod fast_insert; pub use fast_insert::*; -// pub use crate::scheduler::fast_insert::{run_inner_call, run_inner_call_2}; -pub use crate::scheduler::task_context::FrontendBatchTaskContext; +use crate::scheduler::task_context::FrontendBatchTaskContext; mod error; pub mod streaming_manager; diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 8691a23044126..d409225c96c90 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -570,7 +570,6 @@ impl Debug for QueryStage { .field("parallelism", &self.parallelism) .field("exchange_info", &self.exchange_info) .field("has_table_scan", &self.has_table_scan()) - .field("stages", &self.root) .finish() } } diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 1b46b0610d95b..fb960a785107b 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -34,7 +34,6 @@ use tower_http::add_extension::AddExtensionLayer; use tower_http::compression::CompressionLayer; use tower_http::cors::{self, CorsLayer}; -// use crate::handler::fast_insert::handle_fast_insert; use crate::scheduler::FastInsertExecution; use crate::webhook::utils::{err, Result}; mod utils; @@ -144,7 +143,7 @@ pub(super) mod handlers { // Use builder to obtain a single (List) column DataChunk let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); - // let context = FrontendBatchTaskContext::new(session.clone()); + let execution = FastInsertExecution::new( fast_insert_node, wait_for_persistence, From a31d9d3f469bf1bef085c8a73bacfc0afec1532a Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 18:16:40 -0500 Subject: [PATCH 07/27] fix --- src/batch/src/executor/fast_insert.rs | 6 +++--- src/frontend/src/scheduler/fast_insert.rs | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index 69b9943029ef3..d392540edb7e1 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -1,4 +1,4 @@ -// Copyright 2024 RisingWave Labs +// Copyright 2025 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -212,8 +212,8 @@ mod tests { 0, )); let handle = tokio::spawn(async move { - let epoch_recieved = insert_executor.do_execute(data_chunk).await.unwrap(); - assert_eq!(epoch, epoch_recieved); + let epoch_received = insert_executor.do_execute(data_chunk).await.unwrap(); + assert_eq!(epoch, epoch_received); }); // Read diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 8df60b431a505..0dae9b01c66ce 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -1,4 +1,4 @@ -// Copyright 2024 RisingWave Labs +// Copyright 2025 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 8a27a53197352f2454a9b822aa86473c16e1de27 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 18:31:38 -0500 Subject: [PATCH 08/27] fmt --- proto/task_service.proto | 2 +- src/batch/src/executor/fast_insert.rs | 21 +++++++++++++++------ src/batch/src/rpc/service/task_service.rs | 12 ++++++++---- src/frontend/src/scheduler/fast_insert.rs | 2 +- 4 files changed, 25 insertions(+), 12 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index c921e9c8a4e3a..3c59d13b4eba7 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -62,7 +62,7 @@ message GetDataResponse { } message FastInsertRequest { - bool wait_epoch = 1; + bool wait_for_persistence = 1; batch_plan.FastInsertNode fast_insert_node = 2; // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. } diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index d392540edb7e1..f85711095f079 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -20,13 +20,13 @@ use risingwave_common::array::{DataChunk, Op, SerialArray, StreamChunk}; use risingwave_common::catalog::{Field, Schema, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::types::DataType; -use risingwave_common::util::epoch::Epoch; +use risingwave_common::util::epoch::{Epoch, INVALID_EPOCH}; use risingwave_dml::dml_manager::DmlManagerRef; use risingwave_pb::batch_plan::FastInsertNode; use crate::error::Result; -/// A fast insert executor spacially designed for non-pgwire inserts like websockets and webhooks. +/// A fast insert executor spacially designed for non-pgwire inserts such as websockets and webhooks. pub struct FastInsertExecutor { /// Target table id. table_id: TableId, @@ -92,7 +92,11 @@ impl FastInsertExecutor { } impl FastInsertExecutor { - pub async fn do_execute(self, data_chunk_to_insert: DataChunk) -> Result { + pub async fn do_execute( + self, + data_chunk_to_insert: DataChunk, + returning_epoch: bool, + ) -> Result { let table_dml_handle = self .dml_manager .table_dml_handle(self.table_id, self.table_version_id)?; @@ -136,8 +140,13 @@ impl FastInsertExecutor { Result::Ok(()) }; write_txn_data(data_chunk_to_insert).await?; - let epoch = write_handle.end_returning_epoch().await?; - return Ok(epoch); + if returning_epoch { + write_handle.end_returning_epoch().await.map_err(Into::into) + } else { + write_handle.end().await?; + // the returned epoch is invalid and should not be used. + Ok(Epoch(INVALID_EPOCH)) + } } } @@ -212,7 +221,7 @@ mod tests { 0, )); let handle = tokio::spawn(async move { - let epoch_received = insert_executor.do_execute(data_chunk).await.unwrap(); + let epoch_received = insert_executor.do_execute(data_chunk, true).await.unwrap(); assert_eq!(epoch, epoch_received); }); diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index faacabb31e455..17e121be740ff 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -129,7 +129,9 @@ impl TaskService for BatchServiceImpl { ) -> Result, Status> { let req = request.into_inner(); let insert_node = req.fast_insert_node.expect("no fast insert node found"); - let res = self.do_fast_insert(insert_node, req.wait_epoch).await; + let res = self + .do_fast_insert(insert_node, req.wait_for_persistence) + .await; match res { Ok(_) => Ok(Response::new(FastInsertResponse { status: fast_insert_response::Status::Succeeded.into(), @@ -218,13 +220,15 @@ impl BatchServiceImpl { async fn do_fast_insert( &self, insert_node: FastInsertNode, - wait_epoch: bool, + wait_for_persistence: bool, ) -> Result<(), BatchError> { let table_id = insert_node.table_id; let (executor, data_chunk) = FastInsertExecutor::build(self.env.dml_manager_ref(), insert_node)?; - let epoch = executor.do_execute(data_chunk).await?; - if wait_epoch { + let epoch = executor + .do_execute(data_chunk, wait_for_persistence) + .await?; + if wait_for_persistence { dispatch_state_store!(self.env.state_store(), store, { use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::HummockReadEpoch; diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 0dae9b01c66ce..7088049f972de 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -64,7 +64,7 @@ impl FastInsertExecution { let client = self.session.env().client_pool().get(&workers).await?; let request = FastInsertRequest { fast_insert_node: Some(self.fast_insert_node), - wait_epoch: self.wait_for_persistence, + wait_for_persistence: self.wait_for_persistence, }; let response = client.fast_insert(request).await?; Ok(response) From e3162ff32ec5f3ecc8e83df5dc7ff5a615e4571b Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 18:48:04 -0500 Subject: [PATCH 09/27] fix --- src/frontend/src/scheduler/fast_insert.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 7088049f972de..1d034a9eaeabe 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -56,12 +56,12 @@ impl FastInsertExecution { } pub async fn my_execute(self) -> SchedulerResult { - let workers = self.choose_worker( + let worker = self.choose_worker( &TableId::new(self.fast_insert_node.table_id), self.fast_insert_node.session_id, )?; - let client = self.session.env().client_pool().get(&workers).await?; + let client = self.session.env().client_pool().get(&worker).await?; let request = FastInsertRequest { fast_insert_node: Some(self.fast_insert_node), wait_for_persistence: self.wait_for_persistence, From 8f29ae58fc280be6a5bf970a97336d68a2a53458 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:12:13 -0500 Subject: [PATCH 10/27] fix --- proto/task_service.proto | 1 - 1 file changed, 1 deletion(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index 3c59d13b4eba7..ebe328602554e 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -44,7 +44,6 @@ message CreateTaskRequest { plan_common.ExprContext expr_context = 5; } - message CancelTaskRequest { batch_plan.TaskId task_id = 1; } From e60a1148e48f7200b289c23956ba8125aedbf968 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:34:22 -0500 Subject: [PATCH 11/27] fix --- src/batch/src/rpc/service/task_service.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index 17e121be740ff..2f578055e7fa0 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -135,7 +135,7 @@ impl TaskService for BatchServiceImpl { match res { Ok(_) => Ok(Response::new(FastInsertResponse { status: fast_insert_response::Status::Succeeded.into(), - error_message: "".to_string(), + error_message: "".to_owned(), })), Err(e) => match e { BatchError::Dml(e) => Ok(Response::new(FastInsertResponse { From 5cb859da13e9b66d39780a30bc499563c3231d29 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:42:54 -0500 Subject: [PATCH 12/27] fix --- src/frontend/src/webhook/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index fb960a785107b..b9a786ef47cfd 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -99,7 +99,7 @@ pub(super) mod handlers { let FastInsertContext { webhook_source_info, mut fast_insert_node, - } = acquire_table_info(&session, &database, &schema, &table).await?; + } = acquire_table_info(&session, &database, &schema, &table)?; let WebhookSourceInfo { signature_expr, @@ -161,7 +161,7 @@ pub(super) mod handlers { } } - async fn acquire_table_info( + fn acquire_table_info( session: &Arc, database: &String, schema: &String, From c961399dc4557f636e62d61f0c5cf24400297e30 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:51:26 -0500 Subject: [PATCH 13/27] fix --- src/batch/src/executor/fast_insert.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index f85711095f079..07716c7805315 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -185,7 +185,7 @@ mod tests { let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); let mut header_map = HashMap::new(); - header_map.insert("data".to_string(), "value1".to_string()); + header_map.insert("data".to_owned(), "value1".to_owned()); let json_value = json!(header_map); let jsonb_val = JsonbVal::from(json_value); @@ -235,7 +235,7 @@ mod tests { }); assert_matches!(reader.next().await.unwrap()?, TxnMsg::End(_, Some(epoch_notifier)) => { - epoch_notifier.send(epoch.clone()).unwrap(); + epoch_notifier.send(epoch).unwrap(); }); let epoch = u64::MAX; let full_range = (Bound::Unbounded, Bound::Unbounded); From b9baa746e722d489aec2d94481841e317feb2c16 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:56:28 -0500 Subject: [PATCH 14/27] tmp --- src/frontend/src/webhook/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index b9a786ef47cfd..6126808c5c857 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -131,7 +131,7 @@ pub(super) mod handlers { } let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); - let json_value = Value::from_text(&body.to_vec()).map_err(|e| { + let json_value = Value::from_text(&body).map_err(|e| { err( anyhow!(e).context("Failed to parse body"), StatusCode::UNPROCESSABLE_ENTITY, @@ -172,7 +172,7 @@ pub(super) mod handlers { let reader = session.env().catalog_reader().read_guard(); let (table_catalog, _schema) = reader - .get_any_table_by_name(database.as_str(), schema_path, &table) + .get_any_table_by_name(database.as_str(), schema_path, table) .map_err(|e| err(e, StatusCode::NOT_FOUND))?; let webhook_source_info = table_catalog From 601fdbd56459b1df7890dc0741e57602b338dbd0 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 20:02:37 -0500 Subject: [PATCH 15/27] tmp --- src/frontend/src/scheduler/fast_insert.rs | 2 +- src/frontend/src/webhook/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 1d034a9eaeabe..76e8818f213cc 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -55,7 +55,7 @@ impl FastInsertExecution { } } - pub async fn my_execute(self) -> SchedulerResult { + pub async fn execute(self) -> SchedulerResult { let worker = self.choose_worker( &TableId::new(self.fast_insert_node.table_id), self.fast_insert_node.session_id, diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 6126808c5c857..ad5d1181b91a8 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -150,7 +150,7 @@ pub(super) mod handlers { session.env().clone(), session.clone(), ); - let res = execution.my_execute().await.unwrap(); + let res = execution.execute().await.unwrap(); if res.status == fast_insert_response::Status::Succeeded as i32 { Ok(()) } else { From 528549469ab4bf71d55abeca52a5cc5323ae513d Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 19:11:24 -0500 Subject: [PATCH 16/27] tmp --- src/frontend/src/webhook/mod.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index ad5d1181b91a8..32cf26c1ca776 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -23,11 +23,13 @@ use axum::routing::post; use axum::Router; use pgwire::net::Address; use pgwire::pg_server::SessionManager; +use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::array::{Array, ArrayBuilder, DataChunk}; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::{DataType, JsonbVal, Scalar}; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; +use risingwave_pb::common::WorkerNode; use tokio::net::TcpListener; use tower::ServiceBuilder; use tower_http::add_extension::AddExtensionLayer; @@ -47,6 +49,7 @@ const USER: &str = "root"; pub struct FastInsertContext { pub webhook_source_info: WebhookSourceInfo, pub fast_insert_node: FastInsertNode, + pub worker_node: WorkerNode, } #[derive(Clone)] @@ -195,6 +198,9 @@ pub(super) mod handlers { session_id: session.id().0 as u32, }; + let worker_node_manager = + WorkerNodeSelector::new(session.env().worker_node_manager_ref(), false); + Ok(FastInsertContext { webhook_source_info, fast_insert_node, From 8b396ded844311075f5500eaf843dd20c430a75b Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 20:52:54 -0500 Subject: [PATCH 17/27] tmp --- src/frontend/src/scheduler/fast_insert.rs | 127 +++++++++------------- src/frontend/src/webhook/mod.rs | 83 ++++++++------ 2 files changed, 102 insertions(+), 108 deletions(-) diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 76e8818f213cc..34e8063e2e35e 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -17,96 +17,73 @@ use std::sync::Arc; use anyhow::anyhow; use itertools::Itertools; +use pgwire::pg_server::Session; use risingwave_batch::error::BatchError; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::hash::WorkerSlotMapping; -use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::common::WorkerNode; -use risingwave_pb::task_service::{FastInsertRequest, FastInsertResponse}; +use risingwave_rpc_client::ComputeClient; use crate::catalog::TableId; use crate::scheduler::{SchedulerError, SchedulerResult}; use crate::session::{FrontendEnv, SessionImpl}; -pub struct FastInsertExecution { - fast_insert_node: FastInsertNode, - wait_for_persistence: bool, - front_env: FrontendEnv, - session: Arc, - worker_node_manager: WorkerNodeSelector, -} - -impl FastInsertExecution { - pub fn new( - fast_insert_node: FastInsertNode, - wait_for_persistence: bool, - front_env: FrontendEnv, - session: Arc, - ) -> Self { - let worker_node_manager = - WorkerNodeSelector::new(front_env.worker_node_manager_ref(), false); +pub async fn choose_fast_insert_client( + table_id: &TableId, + // wait_for_persistence: bool, + session: &Arc, +) -> SchedulerResult { + let worker = choose_worker(table_id, session)?; - Self { - fast_insert_node, - wait_for_persistence, - front_env, - session, - worker_node_manager, - } - } + let client = session.env().client_pool().get(&worker).await?; + return Ok(client); + // let request = FastInsertRequest { + // fast_insert_node: Some(fast_insert_node), + // wait_for_persistence: wait_for_persistence, + // }; + // let response = client.fast_insert(request).await?; + // Ok(response) +} - pub async fn execute(self) -> SchedulerResult { - let worker = self.choose_worker( - &TableId::new(self.fast_insert_node.table_id), - self.fast_insert_node.session_id, - )?; +fn get_table_dml_vnode_mapping( + table_id: &TableId, + frontend_env: &FrontendEnv, + worker_node_manager: &WorkerNodeSelector, +) -> SchedulerResult { + let guard = frontend_env.catalog_reader().read_guard(); - let client = self.session.env().client_pool().get(&worker).await?; - let request = FastInsertRequest { - fast_insert_node: Some(self.fast_insert_node), - wait_for_persistence: self.wait_for_persistence, - }; - let response = client.fast_insert(request).await?; - Ok(response) - } + let table = guard + .get_any_table_by_id(table_id) + .map_err(|e| SchedulerError::Internal(anyhow!(e)))?; - #[inline(always)] - fn get_table_dml_vnode_mapping( - &self, - table_id: &TableId, - ) -> SchedulerResult { - let guard = self.front_env.catalog_reader().read_guard(); + let fragment_id = match table.dml_fragment_id.as_ref() { + Some(dml_fragment_id) => dml_fragment_id, + // Backward compatibility for those table without `dml_fragment_id`. + None => &table.fragment_id, + }; - let table = guard - .get_any_table_by_id(table_id) - .map_err(|e| SchedulerError::Internal(anyhow!(e)))?; + worker_node_manager + .manager + .get_streaming_fragment_mapping(fragment_id) + .map_err(|e| e.into()) +} - let fragment_id = match table.dml_fragment_id.as_ref() { - Some(dml_fragment_id) => dml_fragment_id, - // Backward compatibility for those table without `dml_fragment_id`. - None => &table.fragment_id, - }; +fn choose_worker(table_id: &TableId, session: &Arc) -> SchedulerResult { + let worker_node_manager = + WorkerNodeSelector::new(session.env().worker_node_manager_ref(), false); + let session_id: u32 = session.id().0 as u32; - self.worker_node_manager + // dml should use streaming vnode mapping + let vnode_mapping = get_table_dml_vnode_mapping(table_id, session.env(), &worker_node_manager)?; + let worker_node = { + let worker_ids = vnode_mapping.iter_unique().collect_vec(); + let candidates = worker_node_manager .manager - .get_streaming_fragment_mapping(fragment_id) - .map_err(|e| e.into()) - } - - fn choose_worker(&self, table_id: &TableId, session_id: u32) -> SchedulerResult { - // dml should use streaming vnode mapping - let vnode_mapping = self.get_table_dml_vnode_mapping(table_id)?; - let worker_node = { - let worker_ids = vnode_mapping.iter_unique().collect_vec(); - let candidates = self - .worker_node_manager - .manager - .get_workers_by_worker_slot_ids(&worker_ids)?; - if candidates.is_empty() { - return Err(BatchError::EmptyWorkerNodes.into()); - } - candidates[session_id as usize % candidates.len()].clone() - }; - Ok(worker_node) - } + .get_workers_by_worker_slot_ids(&worker_ids)?; + if candidates.is_empty() { + return Err(BatchError::EmptyWorkerNodes.into()); + } + candidates[session_id as usize % candidates.len()].clone() + }; + Ok(worker_node) } diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 32cf26c1ca776..be21d2625b995 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -23,22 +23,21 @@ use axum::routing::post; use axum::Router; use pgwire::net::Address; use pgwire::pg_server::SessionManager; -use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::array::{Array, ArrayBuilder, DataChunk}; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::{DataType, JsonbVal, Scalar}; use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; -use risingwave_pb::common::WorkerNode; +use risingwave_pb::task_service::{FastInsertRequest, FastInsertResponse}; use tokio::net::TcpListener; use tower::ServiceBuilder; use tower_http::add_extension::AddExtensionLayer; use tower_http::compression::CompressionLayer; use tower_http::cors::{self, CorsLayer}; -use crate::scheduler::FastInsertExecution; use crate::webhook::utils::{err, Result}; mod utils; +use risingwave_rpc_client::ComputeClient; pub type Service = Arc; @@ -49,7 +48,7 @@ const USER: &str = "root"; pub struct FastInsertContext { pub webhook_source_info: WebhookSourceInfo, pub fast_insert_node: FastInsertNode, - pub worker_node: WorkerNode, + pub compute_client: ComputeClient, } #[derive(Clone)] @@ -70,6 +69,7 @@ pub(super) mod handlers { use super::*; use crate::catalog::root_catalog::SchemaPath; + use crate::scheduler::choose_fast_insert_client; use crate::session::{SessionImpl, SESSION_MANAGER}; pub async fn handle_post_request( @@ -102,7 +102,8 @@ pub(super) mod handlers { let FastInsertContext { webhook_source_info, mut fast_insert_node, - } = acquire_table_info(&session, &database, &schema, &table)?; + compute_client, + } = acquire_table_info(&session, &database, &schema, &table).await?; let WebhookSourceInfo { signature_expr, @@ -146,14 +147,7 @@ pub(super) mod handlers { // Use builder to obtain a single (List) column DataChunk let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); - - let execution = FastInsertExecution::new( - fast_insert_node, - wait_for_persistence, - session.env().clone(), - session.clone(), - ); - let res = execution.execute().await.unwrap(); + let res = execute(fast_insert_node, wait_for_persistence, compute_client).await?; if res.status == fast_insert_response::Status::Succeeded as i32 { Ok(()) } else { @@ -164,7 +158,7 @@ pub(super) mod handlers { } } - fn acquire_table_info( + async fn acquire_table_info( session: &Arc, database: &String, schema: &String, @@ -173,39 +167,62 @@ pub(super) mod handlers { let search_path = session.config().search_path(); let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); - let reader = session.env().catalog_reader().read_guard(); - let (table_catalog, _schema) = reader - .get_any_table_by_name(database.as_str(), schema_path, table) - .map_err(|e| err(e, StatusCode::NOT_FOUND))?; - - let webhook_source_info = table_catalog - .webhook_info - .as_ref() - .ok_or_else(|| { - err( - anyhow!("Table `{}` is not with webhook source", table), - StatusCode::FORBIDDEN, - ) - })? - .clone(); + let (webhook_source_info, table_id, version_id) = { + let reader = session.env().catalog_reader().read_guard(); + let (table_catalog, _schema) = reader + .get_any_table_by_name(database.as_str(), schema_path, table) + .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + + let webhook_source_info = table_catalog + .webhook_info + .as_ref() + .ok_or_else(|| { + err( + anyhow!("Table `{}` is not with webhook source", table), + StatusCode::FORBIDDEN, + ) + })? + .clone(); + ( + webhook_source_info, + table_catalog.id(), + table_catalog.version_id().expect("table must be versioned"), + ) + }; let fast_insert_node = FastInsertNode { - table_id: table_catalog.id().table_id, - table_version_id: table_catalog.version_id().expect("table must be versioned"), + table_id: table_id.table_id, + table_version_id: version_id, column_indices: vec![0], data_chunk: None, row_id_index: Some(1), session_id: session.id().0 as u32, }; - let worker_node_manager = - WorkerNodeSelector::new(session.env().worker_node_manager_ref(), false); + let compute_client = choose_fast_insert_client(&table_id, &session) + .await + .unwrap(); Ok(FastInsertContext { webhook_source_info, fast_insert_node, + compute_client, }) } + + async fn execute( + fast_insert_node: FastInsertNode, + wait_for_persistence: bool, + client: ComputeClient, + ) -> Result { + let request = FastInsertRequest { + fast_insert_node: Some(fast_insert_node), + wait_for_persistence, + }; + // WKXTODO: handle error + let response = client.fast_insert(request).await.unwrap(); + Ok(response) + } } impl WebhookService { From f76bad341e191e7667f52cc7449395ff35893fa1 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 22:16:06 -0500 Subject: [PATCH 18/27] refactor --- proto/batch_plan.proto | 1 + proto/task_service.proto | 1 + src/frontend/src/scheduler/fast_insert.rs | 7 ------ src/frontend/src/webhook/mod.rs | 28 +++++++++++++++-------- 4 files changed, 21 insertions(+), 16 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 1012adaf62ba9..433c3564ed112 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -195,6 +195,7 @@ message InsertNode { uint32 session_id = 7; } +// A special insert node for non-pgwire insert, not really a batch node. message FastInsertNode { // Id of the table to perform inserting. uint32 table_id = 1; diff --git a/proto/task_service.proto b/proto/task_service.proto index ebe328602554e..129c04309da37 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -89,6 +89,7 @@ service TaskService { // Cancel an already-died (self execution-failure, previous aborted, completed) task will still succeed. rpc CancelTask(CancelTaskRequest) returns (CancelTaskResponse); rpc Execute(ExecuteRequest) returns (stream GetDataResponse); + // A lightweight version insert, only for non-pgwire insert, such as inserts from webhooks and websockets. rpc FastInsert(FastInsertRequest) returns (FastInsertResponse); } diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 34e8063e2e35e..fe3bfaf26c6cf 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -34,15 +34,8 @@ pub async fn choose_fast_insert_client( session: &Arc, ) -> SchedulerResult { let worker = choose_worker(table_id, session)?; - let client = session.env().client_pool().get(&worker).await?; return Ok(client); - // let request = FastInsertRequest { - // fast_insert_node: Some(fast_insert_node), - // wait_for_persistence: wait_for_persistence, - // }; - // let response = client.fast_insert(request).await?; - // Ok(response) } fn get_table_dml_vnode_mapping( diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index be21d2625b995..5b84c28f05f37 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -78,15 +78,15 @@ pub(super) mod handlers { Path((database, schema, table)): Path<(String, String, String)>, body: Bytes, ) -> Result<()> { - let session_mgr = SESSION_MANAGER - .get() - .expect("session manager has been initialized"); - // Can be any address, we use the port of meta to indicate that it's a internal request. let dummy_addr = Address::Tcp(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691)); + // FIXME(kexiang): the dummy_session can lead to memory leakage // TODO(kexiang): optimize this // get a session object for the corresponding database + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); let session = session_mgr .connect(database.as_str(), USER, Arc::new(dummy_addr)) .map_err(|e| { @@ -109,7 +109,6 @@ pub(super) mod handlers { signature_expr, secret_ref, wait_for_persistence, - .. } = webhook_source_info; let secret_string = LocalSecretManager::global() @@ -119,6 +118,7 @@ pub(super) mod handlers { // Once limitation here is that the key is no longer case-insensitive, users must user the lowercase key when defining the webhook source table. let headers_jsonb = header_map_to_json(&headers); + // verify the signature let is_valid = verify_signature( headers_jsonb, secret_string.as_str(), @@ -134,6 +134,7 @@ pub(super) mod handlers { )); } + // Use builder to obtain a single column & single row DataChunk let mut builder = JsonbArrayBuilder::with_type(1, DataType::Jsonb); let json_value = Value::from_text(&body).map_err(|e| { err( @@ -143,11 +144,14 @@ pub(super) mod handlers { })?; let jsonb_val = JsonbVal::from(json_value); builder.append(Some(jsonb_val.as_scalar_ref())); - - // Use builder to obtain a single (List) column DataChunk let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); + + // fill the data_chunk fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); + + // execute on the compute node let res = execute(fast_insert_node, wait_for_persistence, compute_client).await?; + if res.status == fast_insert_response::Status::Succeeded as i32 { Ok(()) } else { @@ -194,6 +198,7 @@ pub(super) mod handlers { table_id: table_id.table_id, table_version_id: version_id, column_indices: vec![0], + // leave the data_chunk empty for now data_chunk: None, row_id_index: Some(1), session_id: session.id().0 as u32, @@ -219,8 +224,13 @@ pub(super) mod handlers { fast_insert_node: Some(fast_insert_node), wait_for_persistence, }; - // WKXTODO: handle error - let response = client.fast_insert(request).await.unwrap(); + + let response = client.fast_insert(request).await.map_err(|e| { + err( + anyhow!(e).context("Failed to execute on compute node"), + StatusCode::INTERNAL_SERVER_ERROR, + ) + })?; Ok(response) } } From c35c44b2772558b2754bd90de4178a21b8194817 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 22:41:53 -0500 Subject: [PATCH 19/27] fix --- src/frontend/src/scheduler/fast_insert.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index fe3bfaf26c6cf..e463b6d6f63e2 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -35,7 +35,7 @@ pub async fn choose_fast_insert_client( ) -> SchedulerResult { let worker = choose_worker(table_id, session)?; let client = session.env().client_pool().get(&worker).await?; - return Ok(client); + Ok(client) } fn get_table_dml_vnode_mapping( From e3eb444428c63eccbc90f38e1712e051a412610d Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 23:07:08 -0500 Subject: [PATCH 20/27] tmp --- src/frontend/src/webhook/mod.rs | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 5b84c28f05f37..4a4dec8e9cec7 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -204,9 +204,7 @@ pub(super) mod handlers { session_id: session.id().0 as u32, }; - let compute_client = choose_fast_insert_client(&table_id, &session) - .await - .unwrap(); + let compute_client = choose_fast_insert_client(&table_id, session).await.unwrap(); Ok(FastInsertContext { webhook_source_info, From 4fa62ff9bac833f52f734b56e41cd27b627b22c8 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Tue, 21 Jan 2025 23:40:25 -0500 Subject: [PATCH 21/27] empty From 93446bbfe5a63ecdf7455e2939f7072565651783 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sat, 25 Jan 2025 01:03:39 -0500 Subject: [PATCH 22/27] fix --- proto/batch_plan.proto | 19 ------ proto/task_service.proto | 17 +++++- src/batch/src/executor/fast_insert.rs | 16 ++--- src/batch/src/rpc/service/task_service.rs | 18 ++---- src/frontend/src/scheduler/fast_insert.rs | 26 ++++----- src/frontend/src/session.rs | 6 +- src/frontend/src/webhook/mod.rs | 71 ++++++++--------------- 7 files changed, 72 insertions(+), 101 deletions(-) diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 433c3564ed112..6e07ceae4d5d4 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -195,25 +195,6 @@ message InsertNode { uint32 session_id = 7; } -// A special insert node for non-pgwire insert, not really a batch node. -message FastInsertNode { - // Id of the table to perform inserting. - uint32 table_id = 1; - // Version of the table. - uint64 table_version_id = 2; - repeated uint32 column_indices = 3; - data.DataChunk data_chunk = 4; - - // An optional field and will be `None` for tables without user-defined pk. - // The `BatchInsertExecutor` should add a column with NULL value which will - // be filled in streaming. - optional uint32 row_id_index = 5; - - // Session id is used to ensure that dml data from the same session should be - // sent to a fixed worker node and channel. - uint32 session_id = 6; -} - message DeleteNode { // Id of the table to perform deleting. uint32 table_id = 1; diff --git a/proto/task_service.proto b/proto/task_service.proto index 129c04309da37..77fa71587486c 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -61,8 +61,21 @@ message GetDataResponse { } message FastInsertRequest { - bool wait_for_persistence = 1; - batch_plan.FastInsertNode fast_insert_node = 2; + // Id of the table to perform inserting. + uint32 table_id = 1; + // Version of the table. + uint64 table_version_id = 2; + repeated uint32 column_indices = 3; + data.DataChunk data_chunk = 4; + + // An optional field and will be `None` for tables without user-defined pk. + // The `BatchInsertExecutor` should add a column with NULL value which will + // be filled in streaming. + optional uint32 row_id_index = 5; + + // Use session id to assign the insert to different fixed worker node and channel. + uint32 session_id = 6; + bool wait_for_persistence = 7; // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. } diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index 07716c7805315..1a9bbc353da99 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -22,7 +22,7 @@ use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::types::DataType; use risingwave_common::util::epoch::{Epoch, INVALID_EPOCH}; use risingwave_dml::dml_manager::DmlManagerRef; -use risingwave_pb::batch_plan::FastInsertNode; +use risingwave_pb::task_service::FastInsertRequest; use crate::error::Result; @@ -42,28 +42,28 @@ pub struct FastInsertExecutor { impl FastInsertExecutor { pub fn build( dml_manager: DmlManagerRef, - insert_node: FastInsertNode, + insert_req: FastInsertRequest, ) -> Result<(FastInsertExecutor, DataChunk)> { - let table_id = TableId::new(insert_node.table_id); - let column_indices = insert_node + let table_id = TableId::new(insert_req.table_id); + let column_indices = insert_req .column_indices .iter() .map(|&i| i as usize) .collect(); let mut schema = Schema::new(vec![Field::unnamed(DataType::Jsonb)]); schema.fields.push(Field::unnamed(DataType::Serial)); // row_id column - let data_chunk_pb = insert_node + let data_chunk_pb = insert_req .data_chunk .expect("no data_chunk found in fast insert node"); Ok(( FastInsertExecutor::new( table_id, - insert_node.table_version_id, + insert_req.table_version_id, dml_manager, column_indices, - insert_node.row_id_index.as_ref().map(|index| *index as _), - insert_node.session_id, + insert_req.row_id_index.as_ref().map(|index| *index as _), + insert_req.session_id, ), DataChunk::from_protobuf(&data_chunk_pb)?, )) diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index 2f578055e7fa0..4a08e56ea4fe1 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -15,7 +15,7 @@ use std::sync::Arc; use risingwave_common::util::tracing::TracingContext; -use risingwave_pb::batch_plan::{FastInsertNode, TaskOutputId}; +use risingwave_pb::batch_plan::TaskOutputId; use risingwave_pb::task_service::task_service_server::TaskService; use risingwave_pb::task_service::{ fast_insert_response, CancelTaskRequest, CancelTaskResponse, CreateTaskRequest, ExecuteRequest, @@ -128,10 +128,7 @@ impl TaskService for BatchServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let insert_node = req.fast_insert_node.expect("no fast insert node found"); - let res = self - .do_fast_insert(insert_node, req.wait_for_persistence) - .await; + let res = self.do_fast_insert(req).await; match res { Ok(_) => Ok(Response::new(FastInsertResponse { status: fast_insert_response::Status::Succeeded.into(), @@ -217,14 +214,11 @@ impl BatchServiceImpl { Ok(Response::new(ReceiverStream::new(rx))) } - async fn do_fast_insert( - &self, - insert_node: FastInsertNode, - wait_for_persistence: bool, - ) -> Result<(), BatchError> { - let table_id = insert_node.table_id; + async fn do_fast_insert(&self, insert_req: FastInsertRequest) -> Result<(), BatchError> { + let table_id = insert_req.table_id; + let wait_for_persistence = insert_req.wait_for_persistence; let (executor, data_chunk) = - FastInsertExecutor::build(self.env.dml_manager_ref(), insert_node)?; + FastInsertExecutor::build(self.env.dml_manager_ref(), insert_req)?; let epoch = executor .do_execute(data_chunk, wait_for_persistence) .await?; diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index e463b6d6f63e2..67f47d4f03576 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -12,12 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Local execution for batch query. -use std::sync::Arc; - use anyhow::anyhow; use itertools::Itertools; -use pgwire::pg_server::Session; use risingwave_batch::error::BatchError; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::hash::WorkerSlotMapping; @@ -26,15 +22,15 @@ use risingwave_rpc_client::ComputeClient; use crate::catalog::TableId; use crate::scheduler::{SchedulerError, SchedulerResult}; -use crate::session::{FrontendEnv, SessionImpl}; +use crate::session::FrontendEnv; pub async fn choose_fast_insert_client( table_id: &TableId, - // wait_for_persistence: bool, - session: &Arc, + frontend_env: &FrontendEnv, + session_id: i32, ) -> SchedulerResult { - let worker = choose_worker(table_id, session)?; - let client = session.env().client_pool().get(&worker).await?; + let worker = choose_worker(table_id, frontend_env, session_id)?; + let client = frontend_env.client_pool().get(&worker).await?; Ok(client) } @@ -61,13 +57,17 @@ fn get_table_dml_vnode_mapping( .map_err(|e| e.into()) } -fn choose_worker(table_id: &TableId, session: &Arc) -> SchedulerResult { +fn choose_worker( + table_id: &TableId, + frontend_env: &FrontendEnv, + session_id: i32, +) -> SchedulerResult { let worker_node_manager = - WorkerNodeSelector::new(session.env().worker_node_manager_ref(), false); - let session_id: u32 = session.id().0 as u32; + WorkerNodeSelector::new(frontend_env.worker_node_manager_ref(), false); + let session_id: u32 = session_id as u32; // dml should use streaming vnode mapping - let vnode_mapping = get_table_dml_vnode_mapping(table_id, session.env(), &worker_node_manager)?; + let vnode_mapping = get_table_dml_vnode_mapping(table_id, frontend_env, &worker_node_manager)?; let worker_node = { let worker_ids = vnode_mapping.iter_unique().collect_vec(); let candidates = worker_node_manager diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 4060a779657e1..9db4eb6c7e06a 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1455,7 +1455,7 @@ impl SessionManagerImpl { }; // Assign a session id and insert into sessions map (for cancel request). - let secret_key = self.number.fetch_add(1, Ordering::Relaxed); + let secret_key = self.generate_secret_key(); // Use a trivial strategy: process_id and secret_key are equal. let id = (secret_key, secret_key); // Read session params snapshot from frontend env. @@ -1480,6 +1480,10 @@ impl SessionManagerImpl { ))) } } + + pub fn generate_secret_key(&self) -> i32 { + self.number.fetch_add(1, Ordering::Relaxed) + } } impl Session for SessionImpl { diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 4a4dec8e9cec7..8fbaa8acbfc61 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::net::{IpAddr, SocketAddr}; +use std::net::SocketAddr; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -21,12 +21,9 @@ use axum::extract::{Extension, Path}; use axum::http::{HeaderMap, Method, StatusCode}; use axum::routing::post; use axum::Router; -use pgwire::net::Address; -use pgwire::pg_server::SessionManager; use risingwave_common::array::{Array, ArrayBuilder, DataChunk}; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::{DataType, JsonbVal, Scalar}; -use risingwave_pb::batch_plan::FastInsertNode; use risingwave_pb::catalog::WebhookSourceInfo; use risingwave_pb::task_service::{FastInsertRequest, FastInsertResponse}; use tokio::net::TcpListener; @@ -47,7 +44,7 @@ const USER: &str = "root"; #[derive(Clone)] pub struct FastInsertContext { pub webhook_source_info: WebhookSourceInfo, - pub fast_insert_node: FastInsertNode, + pub fast_insert_request: FastInsertRequest, pub compute_client: ComputeClient, } @@ -57,12 +54,9 @@ pub struct WebhookService { } pub(super) mod handlers { - use std::net::Ipv4Addr; - use jsonbb::Value; - use pgwire::pg_server::Session; use risingwave_common::array::JsonbArrayBuilder; - use risingwave_pb::batch_plan::FastInsertNode; + use risingwave_common::session_config::SearchPath; use risingwave_pb::catalog::WebhookSourceInfo; use risingwave_pb::task_service::fast_insert_response; use utils::{header_map_to_json, verify_signature}; @@ -70,7 +64,7 @@ pub(super) mod handlers { use super::*; use crate::catalog::root_catalog::SchemaPath; use crate::scheduler::choose_fast_insert_client; - use crate::session::{SessionImpl, SESSION_MANAGER}; + use crate::session::{FrontendEnv, SESSION_MANAGER}; pub async fn handle_post_request( Extension(_srv): Extension, @@ -78,37 +72,25 @@ pub(super) mod handlers { Path((database, schema, table)): Path<(String, String, String)>, body: Bytes, ) -> Result<()> { - // Can be any address, we use the port of meta to indicate that it's a internal request. - let dummy_addr = Address::Tcp(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691)); - // FIXME(kexiang): the dummy_session can lead to memory leakage - // TODO(kexiang): optimize this - // get a session object for the corresponding database let session_mgr = SESSION_MANAGER .get() .expect("session manager has been initialized"); - let session = session_mgr - .connect(database.as_str(), USER, Arc::new(dummy_addr)) - .map_err(|e| { - err( - anyhow!(e).context(format!( - "Failed to create session for database `{}` with user `{}`", - database, USER - )), - StatusCode::UNAUTHORIZED, - ) - })?; + + let frontend_env = session_mgr.env().clone(); + // FIXME(kexiang): the session_id is i32, overflow is possible + let session_id = session_mgr.generate_secret_key(); let FastInsertContext { webhook_source_info, - mut fast_insert_node, + mut fast_insert_request, compute_client, - } = acquire_table_info(&session, &database, &schema, &table).await?; + } = acquire_table_info(&frontend_env, session_id, &database, &schema, &table).await?; let WebhookSourceInfo { signature_expr, secret_ref, - wait_for_persistence, + wait_for_persistence: _, } = webhook_source_info; let secret_string = LocalSecretManager::global() @@ -147,10 +129,9 @@ pub(super) mod handlers { let data_chunk = DataChunk::new(vec![builder.finish().into_ref()], 1); // fill the data_chunk - fast_insert_node.data_chunk = Some(data_chunk.to_protobuf()); - + fast_insert_request.data_chunk = Some(data_chunk.to_protobuf()); // execute on the compute node - let res = execute(fast_insert_node, wait_for_persistence, compute_client).await?; + let res = execute(fast_insert_request, compute_client).await?; if res.status == fast_insert_response::Status::Succeeded as i32 { Ok(()) @@ -163,16 +144,17 @@ pub(super) mod handlers { } async fn acquire_table_info( - session: &Arc, + frontend_env: &FrontendEnv, + session_id: i32, database: &String, schema: &String, table: &String, ) -> Result { - let search_path = session.config().search_path(); + let search_path = SearchPath::default(); let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); let (webhook_source_info, table_id, version_id) = { - let reader = session.env().catalog_reader().read_guard(); + let reader = frontend_env.catalog_reader().read_guard(); let (table_catalog, _schema) = reader .get_any_table_by_name(database.as_str(), schema_path, table) .map_err(|e| err(e, StatusCode::NOT_FOUND))?; @@ -194,35 +176,32 @@ pub(super) mod handlers { ) }; - let fast_insert_node = FastInsertNode { + let fast_insert_request = FastInsertRequest { table_id: table_id.table_id, table_version_id: version_id, column_indices: vec![0], // leave the data_chunk empty for now data_chunk: None, row_id_index: Some(1), - session_id: session.id().0 as u32, + session_id: session_id as u32, + wait_for_persistence: webhook_source_info.wait_for_persistence, }; - let compute_client = choose_fast_insert_client(&table_id, session).await.unwrap(); + let compute_client = choose_fast_insert_client(&table_id, &frontend_env, session_id) + .await + .unwrap(); Ok(FastInsertContext { webhook_source_info, - fast_insert_node, + fast_insert_request, compute_client, }) } async fn execute( - fast_insert_node: FastInsertNode, - wait_for_persistence: bool, + request: FastInsertRequest, client: ComputeClient, ) -> Result { - let request = FastInsertRequest { - fast_insert_node: Some(fast_insert_node), - wait_for_persistence, - }; - let response = client.fast_insert(request).await.map_err(|e| { err( anyhow!(e).context("Failed to execute on compute node"), From e6a7847fa6f324ed38a765cb30f025d9c9942019 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sat, 25 Jan 2025 01:34:13 -0500 Subject: [PATCH 23/27] fix --- proto/task_service.proto | 4 +-- src/batch/src/executor/fast_insert.rs | 10 +++--- src/frontend/src/scheduler/fast_insert.rs | 9 +++--- src/frontend/src/session.rs | 6 +--- src/frontend/src/webhook/mod.rs | 39 ++++++++++++----------- 5 files changed, 33 insertions(+), 35 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index 77fa71587486c..b873cddac0356 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -73,8 +73,8 @@ message FastInsertRequest { // be filled in streaming. optional uint32 row_id_index = 5; - // Use session id to assign the insert to different fixed worker node and channel. - uint32 session_id = 6; + // Use this number to assign the insert to different fixed worker node and channel. + uint32 counter = 6; bool wait_for_persistence = 7; // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. } diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index 1a9bbc353da99..52300e7c60f79 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -36,7 +36,7 @@ pub struct FastInsertExecutor { row_id_index: Option, txn_id: TxnId, - session_id: u32, + counter: u32, } impl FastInsertExecutor { @@ -63,7 +63,7 @@ impl FastInsertExecutor { dml_manager, column_indices, insert_req.row_id_index.as_ref().map(|index| *index as _), - insert_req.session_id, + insert_req.counter, ), DataChunk::from_protobuf(&data_chunk_pb)?, )) @@ -76,7 +76,7 @@ impl FastInsertExecutor { dml_manager: DmlManagerRef, column_indices: Vec, row_id_index: Option, - session_id: u32, + counter: u32, ) -> Self { let txn_id = dml_manager.gen_txn_id(); Self { @@ -86,7 +86,7 @@ impl FastInsertExecutor { column_indices, row_id_index, txn_id, - session_id, + counter, } } } @@ -100,7 +100,7 @@ impl FastInsertExecutor { let table_dml_handle = self .dml_manager .table_dml_handle(self.table_id, self.table_version_id)?; - let mut write_handle = table_dml_handle.write_handle(self.session_id, self.txn_id)?; + let mut write_handle = table_dml_handle.write_handle(self.counter, self.txn_id)?; write_handle.begin()?; diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 67f47d4f03576..7fa69477892f3 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -27,9 +27,9 @@ use crate::session::FrontendEnv; pub async fn choose_fast_insert_client( table_id: &TableId, frontend_env: &FrontendEnv, - session_id: i32, + counter: u32, ) -> SchedulerResult { - let worker = choose_worker(table_id, frontend_env, session_id)?; + let worker = choose_worker(table_id, frontend_env, counter)?; let client = frontend_env.client_pool().get(&worker).await?; Ok(client) } @@ -60,11 +60,10 @@ fn get_table_dml_vnode_mapping( fn choose_worker( table_id: &TableId, frontend_env: &FrontendEnv, - session_id: i32, + counter: u32, ) -> SchedulerResult { let worker_node_manager = WorkerNodeSelector::new(frontend_env.worker_node_manager_ref(), false); - let session_id: u32 = session_id as u32; // dml should use streaming vnode mapping let vnode_mapping = get_table_dml_vnode_mapping(table_id, frontend_env, &worker_node_manager)?; @@ -76,7 +75,7 @@ fn choose_worker( if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } - candidates[session_id as usize % candidates.len()].clone() + candidates[counter as usize % candidates.len()].clone() }; Ok(worker_node) } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 9db4eb6c7e06a..4060a779657e1 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1455,7 +1455,7 @@ impl SessionManagerImpl { }; // Assign a session id and insert into sessions map (for cancel request). - let secret_key = self.generate_secret_key(); + let secret_key = self.number.fetch_add(1, Ordering::Relaxed); // Use a trivial strategy: process_id and secret_key are equal. let id = (secret_key, secret_key); // Read session params snapshot from frontend env. @@ -1480,10 +1480,6 @@ impl SessionManagerImpl { ))) } } - - pub fn generate_secret_key(&self) -> i32 { - self.number.fetch_add(1, Ordering::Relaxed) - } } impl Session for SessionImpl { diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 8fbaa8acbfc61..c8efbbc16290c 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::net::SocketAddr; +use std::sync::atomic::AtomicU32; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -48,9 +49,9 @@ pub struct FastInsertContext { pub compute_client: ComputeClient, } -#[derive(Clone)] pub struct WebhookService { webhook_addr: SocketAddr, + counter: AtomicU32, } pub(super) mod handlers { @@ -64,28 +65,22 @@ pub(super) mod handlers { use super::*; use crate::catalog::root_catalog::SchemaPath; use crate::scheduler::choose_fast_insert_client; - use crate::session::{FrontendEnv, SESSION_MANAGER}; + use crate::session::SESSION_MANAGER; pub async fn handle_post_request( - Extension(_srv): Extension, + Extension(srv): Extension, headers: HeaderMap, Path((database, schema, table)): Path<(String, String, String)>, body: Bytes, ) -> Result<()> { - // FIXME(kexiang): the dummy_session can lead to memory leakage - let session_mgr = SESSION_MANAGER - .get() - .expect("session manager has been initialized"); - - let frontend_env = session_mgr.env().clone(); - // FIXME(kexiang): the session_id is i32, overflow is possible - let session_id = session_mgr.generate_secret_key(); - + let counter = srv + .counter + .fetch_add(1, std::sync::atomic::Ordering::Relaxed); let FastInsertContext { webhook_source_info, mut fast_insert_request, compute_client, - } = acquire_table_info(&frontend_env, session_id, &database, &schema, &table).await?; + } = acquire_table_info(counter, &database, &schema, &table).await?; let WebhookSourceInfo { signature_expr, @@ -144,12 +139,17 @@ pub(super) mod handlers { } async fn acquire_table_info( - frontend_env: &FrontendEnv, - session_id: i32, + counter: u32, database: &String, schema: &String, table: &String, ) -> Result { + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + + let frontend_env = session_mgr.env(); + let search_path = SearchPath::default(); let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); @@ -183,11 +183,11 @@ pub(super) mod handlers { // leave the data_chunk empty for now data_chunk: None, row_id_index: Some(1), - session_id: session_id as u32, + counter, wait_for_persistence: webhook_source_info.wait_for_persistence, }; - let compute_client = choose_fast_insert_client(&table_id, &frontend_env, session_id) + let compute_client = choose_fast_insert_client(&table_id, frontend_env, counter) .await .unwrap(); @@ -214,7 +214,10 @@ pub(super) mod handlers { impl WebhookService { pub fn new(webhook_addr: SocketAddr) -> Self { - Self { webhook_addr } + Self { + webhook_addr, + counter: AtomicU32::new(0), + } } pub async fn serve(self) -> anyhow::Result<()> { From ac00519729ff9c13a2e7989567e49549989fe8cf Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sat, 25 Jan 2025 16:36:04 -0500 Subject: [PATCH 24/27] rename --- proto/task_service.proto | 4 ++-- src/batch/src/executor/fast_insert.rs | 11 ++++++----- src/frontend/src/scheduler/fast_insert.rs | 8 ++++---- src/frontend/src/webhook/mod.rs | 10 +++++----- 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index b873cddac0356..b8a2c628d03f9 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -73,8 +73,8 @@ message FastInsertRequest { // be filled in streaming. optional uint32 row_id_index = 5; - // Use this number to assign the insert to different fixed worker node and channel. - uint32 counter = 6; + // Use this number to assign the insert req to different worker nodes and dml channels. + uint32 request_id = 6; bool wait_for_persistence = 7; // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. } diff --git a/src/batch/src/executor/fast_insert.rs b/src/batch/src/executor/fast_insert.rs index 52300e7c60f79..898fe370a6eba 100644 --- a/src/batch/src/executor/fast_insert.rs +++ b/src/batch/src/executor/fast_insert.rs @@ -36,7 +36,7 @@ pub struct FastInsertExecutor { row_id_index: Option, txn_id: TxnId, - counter: u32, + request_id: u32, } impl FastInsertExecutor { @@ -63,7 +63,7 @@ impl FastInsertExecutor { dml_manager, column_indices, insert_req.row_id_index.as_ref().map(|index| *index as _), - insert_req.counter, + insert_req.request_id, ), DataChunk::from_protobuf(&data_chunk_pb)?, )) @@ -76,7 +76,7 @@ impl FastInsertExecutor { dml_manager: DmlManagerRef, column_indices: Vec, row_id_index: Option, - counter: u32, + request_id: u32, ) -> Self { let txn_id = dml_manager.gen_txn_id(); Self { @@ -86,7 +86,7 @@ impl FastInsertExecutor { column_indices, row_id_index, txn_id, - counter, + request_id, } } } @@ -100,7 +100,8 @@ impl FastInsertExecutor { let table_dml_handle = self .dml_manager .table_dml_handle(self.table_id, self.table_version_id)?; - let mut write_handle = table_dml_handle.write_handle(self.counter, self.txn_id)?; + // instead of session id, we use request id here to select a write handle. + let mut write_handle = table_dml_handle.write_handle(self.request_id, self.txn_id)?; write_handle.begin()?; diff --git a/src/frontend/src/scheduler/fast_insert.rs b/src/frontend/src/scheduler/fast_insert.rs index 7fa69477892f3..d2e0080b75ab0 100644 --- a/src/frontend/src/scheduler/fast_insert.rs +++ b/src/frontend/src/scheduler/fast_insert.rs @@ -27,9 +27,9 @@ use crate::session::FrontendEnv; pub async fn choose_fast_insert_client( table_id: &TableId, frontend_env: &FrontendEnv, - counter: u32, + request_id: u32, ) -> SchedulerResult { - let worker = choose_worker(table_id, frontend_env, counter)?; + let worker = choose_worker(table_id, frontend_env, request_id)?; let client = frontend_env.client_pool().get(&worker).await?; Ok(client) } @@ -60,7 +60,7 @@ fn get_table_dml_vnode_mapping( fn choose_worker( table_id: &TableId, frontend_env: &FrontendEnv, - counter: u32, + request_id: u32, ) -> SchedulerResult { let worker_node_manager = WorkerNodeSelector::new(frontend_env.worker_node_manager_ref(), false); @@ -75,7 +75,7 @@ fn choose_worker( if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } - candidates[counter as usize % candidates.len()].clone() + candidates[request_id as usize % candidates.len()].clone() }; Ok(worker_node) } diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index c8efbbc16290c..293b492508f3b 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -73,14 +73,14 @@ pub(super) mod handlers { Path((database, schema, table)): Path<(String, String, String)>, body: Bytes, ) -> Result<()> { - let counter = srv + let request_id = srv .counter .fetch_add(1, std::sync::atomic::Ordering::Relaxed); let FastInsertContext { webhook_source_info, mut fast_insert_request, compute_client, - } = acquire_table_info(counter, &database, &schema, &table).await?; + } = acquire_table_info(request_id, &database, &schema, &table).await?; let WebhookSourceInfo { signature_expr, @@ -139,7 +139,7 @@ pub(super) mod handlers { } async fn acquire_table_info( - counter: u32, + request_id: u32, database: &String, schema: &String, table: &String, @@ -183,11 +183,11 @@ pub(super) mod handlers { // leave the data_chunk empty for now data_chunk: None, row_id_index: Some(1), - counter, + request_id, wait_for_persistence: webhook_source_info.wait_for_persistence, }; - let compute_client = choose_fast_insert_client(&table_id, frontend_env, counter) + let compute_client = choose_fast_insert_client(&table_id, frontend_env, request_id) .await .unwrap(); From 7ddf3fae3fce54d8c3652b989cccb72a723527e8 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sat, 25 Jan 2025 16:50:02 -0500 Subject: [PATCH 25/27] fmt --- proto/task_service.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/proto/task_service.proto b/proto/task_service.proto index b8a2c628d03f9..a9176202e966f 100644 --- a/proto/task_service.proto +++ b/proto/task_service.proto @@ -73,7 +73,7 @@ message FastInsertRequest { // be filled in streaming. optional uint32 row_id_index = 5; - // Use this number to assign the insert req to different worker nodes and dml channels. + // Use this number to assign the insert req to different worker nodes and dml channels. uint32 request_id = 6; bool wait_for_persistence = 7; // TODO(kexiang): add support for default columns. plan_common.ExprContext expr_context is needed for it. From 01a065966819c60ffc183b6310a5d78051e95f24 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Thu, 13 Feb 2025 23:58:42 -0500 Subject: [PATCH 26/27] prohibit alter for webhook --- e2e_test/webhook/check_1.slt.part | 4 ++-- e2e_test/webhook/check_2.slt.part | 7 ++++++- e2e_test/webhook/check_3.slt.part | 7 ++++++- e2e_test/webhook/create_table.slt.part | 13 +++++++++++++ e2e_test/webhook/drop_table.slt.part | 3 +++ e2e_test/webhook/sender.py | 19 +++++++++++++++++++ .../src/handler/alter_table_column.rs | 6 ++++++ src/frontend/src/webhook/mod.rs | 5 +++-- 8 files changed, 58 insertions(+), 6 deletions(-) diff --git a/e2e_test/webhook/check_1.slt.part b/e2e_test/webhook/check_1.slt.part index c207ba53afb00..b5b2888ff5ee4 100644 --- a/e2e_test/webhook/check_1.slt.part +++ b/e2e_test/webhook/check_1.slt.part @@ -19,6 +19,6 @@ select data ->> 'source', data->> 'auth_algo' from segment_hmac_sha1; segment hmac_sha1 query TT -select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; +select data ->> 'source', data->> 'auth_algo' from test_primary_key; ---- -hubspot sha256_v2 \ No newline at end of file +github hmac_sha1 diff --git a/e2e_test/webhook/check_2.slt.part b/e2e_test/webhook/check_2.slt.part index 0b6305701d882..7fbd8516ce560 100644 --- a/e2e_test/webhook/check_2.slt.part +++ b/e2e_test/webhook/check_2.slt.part @@ -26,4 +26,9 @@ query TT select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; ---- hubspot sha256_v2 -hubspot sha256_v2 \ No newline at end of file +hubspot sha256_v2 + +query TT +select data ->> 'source', data->> 'auth_algo' from test_primary_key; +---- +github hmac_sha1 diff --git a/e2e_test/webhook/check_3.slt.part b/e2e_test/webhook/check_3.slt.part index 0ad97c19f0880..4c75ef84f49c4 100644 --- a/e2e_test/webhook/check_3.slt.part +++ b/e2e_test/webhook/check_3.slt.part @@ -31,4 +31,9 @@ select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; ---- hubspot sha256_v2 hubspot sha256_v2 -hubspot sha256_v2 \ No newline at end of file +hubspot sha256_v2 + +query TT +select data ->> 'source', data->> 'auth_algo' from test_primary_key; +---- +github hmac_sha1 diff --git a/e2e_test/webhook/create_table.slt.part b/e2e_test/webhook/create_table.slt.part index a7b0eb19a48f5..fe806f5938f08 100644 --- a/e2e_test/webhook/create_table.slt.part +++ b/e2e_test/webhook/create_table.slt.part @@ -53,3 +53,16 @@ create table hubspot_sha256_v2 ( , 'UTF8') ), 'hex') ); + +statement ok +create table test_primary_key ( + data JSONB PRIMARY KEY +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'x-hub-signature', + 'sha1=' || encode(hmac(test_secret, data, 'sha1'), 'hex') +); + +statement error Adding/dropping a column of a table with webhook has not been implemented. +ALTER TABLE github_hmac_sha1 ADD COLUMN new_col int; diff --git a/e2e_test/webhook/drop_table.slt.part b/e2e_test/webhook/drop_table.slt.part index 32a2a40800b87..fd9cced253e09 100644 --- a/e2e_test/webhook/drop_table.slt.part +++ b/e2e_test/webhook/drop_table.slt.part @@ -1,3 +1,6 @@ +statement ok +DROP TABLE test_primary_key; + statement ok DROP TABLE hubspot_sha256_v2; diff --git a/e2e_test/webhook/sender.py b/e2e_test/webhook/sender.py index 446f674348444..43ce35bf9e1e1 100644 --- a/e2e_test/webhook/sender.py +++ b/e2e_test/webhook/sender.py @@ -62,6 +62,22 @@ def send_github_hmac_sha1(secret): send_webhook(url, headers, payload_json) +def send_test_primary_key(secret): + payload = message + payload['source'] = "github" + payload['auth_algo'] = "hmac_sha1" + url = SERVER_URL + "test_primary_key" + + payload_json = json.dumps(payload) + signature = generate_signature_hmac(secret, payload_json, 'sha1', "sha1=") + # Webhook message headers + headers = { + "Content-Type": "application/json", + "X-Hub-Signature": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + def send_github_hmac_sha256(secret): payload = message payload['source'] = "github" @@ -143,3 +159,6 @@ def send_hubspot_sha256_v2(secret): send_segment_hmac_sha1(secret) # hubspot send_hubspot_sha256_v2(secret) + + # ensure the single column can still work as normal + send_test_primary_key(secret) diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index ceaac06e77803..d60d1e58c8003 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -258,6 +258,12 @@ pub async fn handle_alter_table_column( ))); } + if let Some(_) = original_catalog.webhook_info { + return Err(RwError::from(ErrorCode::BindError( + "Adding/dropping a column of a table with webhook has not been implemented.".to_owned(), + ))); + } + // Retrieve the original table definition and parse it to AST. let mut definition = original_catalog.create_sql_ast_purified()?; let Statement::CreateTable { columns, .. } = &mut definition else { diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs index 293b492508f3b..3b12e7e4fa0ec 100644 --- a/src/frontend/src/webhook/mod.rs +++ b/src/frontend/src/webhook/mod.rs @@ -153,7 +153,7 @@ pub(super) mod handlers { let search_path = SearchPath::default(); let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); - let (webhook_source_info, table_id, version_id) = { + let (webhook_source_info, table_id, version_id, row_id_index) = { let reader = frontend_env.catalog_reader().read_guard(); let (table_catalog, _schema) = reader .get_any_table_by_name(database.as_str(), schema_path, table) @@ -173,6 +173,7 @@ pub(super) mod handlers { webhook_source_info, table_catalog.id(), table_catalog.version_id().expect("table must be versioned"), + table_catalog.row_id_index.map(|idx| idx as u32), ) }; @@ -182,7 +183,7 @@ pub(super) mod handlers { column_indices: vec![0], // leave the data_chunk empty for now data_chunk: None, - row_id_index: Some(1), + row_id_index, request_id, wait_for_persistence: webhook_source_info.wait_for_persistence, }; From dd771b28d5922f60a4973e91ad28cfc6896d2b45 Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Fri, 14 Feb 2025 00:10:18 -0500 Subject: [PATCH 27/27] tmp --- src/frontend/src/handler/alter_table_column.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index d60d1e58c8003..1f408f987b395 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -258,7 +258,7 @@ pub async fn handle_alter_table_column( ))); } - if let Some(_) = original_catalog.webhook_info { + if original_catalog.webhook_info.is_some() { return Err(RwError::from(ErrorCode::BindError( "Adding/dropping a column of a table with webhook has not been implemented.".to_owned(), )));