From 606b1d034ffc9559def31c0a4b6c2221e96c6489 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 16 Jun 2024 16:55:16 +0800 Subject: [PATCH 01/75] enable ddl event --- .../connector/source/core/DbzCdcEngine.java | 1 + .../source/core/DbzChangeEventConsumer.java | 66 +++++++++++++++++-- .../src/main/resources/mysql.properties | 2 +- 3 files changed, 64 insertions(+), 5 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java index ed22fe36416ec..9227d8225a65c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java @@ -47,6 +47,7 @@ public DbzCdcEngine( sourceId, heartbeatTopicPrefix, transactionTopic, + topicPrefix, new ArrayBlockingQueue<>(DEFAULT_QUEUE_CAPACITY)); // Builds a debezium engine but not start it diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index cabf9e29a68aa..fb74a192917c8 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -17,6 +17,7 @@ import com.risingwave.connector.api.source.SourceTypeE; import com.risingwave.connector.cdc.debezium.internal.DebeziumOffset; import com.risingwave.connector.cdc.debezium.internal.DebeziumOffsetSerializer; +import com.risingwave.connector.source.common.CdcConnectorException; import com.risingwave.proto.ConnectorServiceProto.CdcMessage; import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; import io.debezium.connector.postgresql.PostgresOffsetContext; @@ -43,6 +44,7 @@ enum EventType { HEARTBEAT, TRANSACTION, DATA, + SCHEMA_CHANGE, } public class DbzChangeEventConsumer @@ -57,6 +59,7 @@ public class DbzChangeEventConsumer private final JsonConverter keyConverter; private final String heartbeatTopicPrefix; private final String transactionTopic; + private final String schemaChangeTopic; private volatile DebeziumEngine.RecordCommitter> currentRecordCommitter; @@ -66,12 +69,14 @@ public class DbzChangeEventConsumer long sourceId, String heartbeatTopicPrefix, String transactionTopic, + String schemaChangeTopic, BlockingQueue queue) { this.connector = connector; this.sourceId = sourceId; this.outputChannel = queue; this.heartbeatTopicPrefix = heartbeatTopicPrefix; this.transactionTopic = transactionTopic; + this.schemaChangeTopic = schemaChangeTopic; LOG.info("heartbeat topic: {}, trnx topic: {}", heartbeatTopicPrefix, transactionTopic); // The default JSON converter will output the schema field in the JSON which is unnecessary @@ -105,6 +110,8 @@ private EventType getEventType(SourceRecord record) { return EventType.HEARTBEAT; } else if (isTransactionMetaEvent(record)) { return EventType.TRANSACTION; + } else if (isSchemaChangeEvent(record)) { + return EventType.SCHEMA_CHANGE; } else { return EventType.DATA; } @@ -122,6 +129,11 @@ private boolean isTransactionMetaEvent(SourceRecord record) { return topic != null && topic.equals(transactionTopic); } + private boolean isSchemaChangeEvent(SourceRecord record) { + String topic = record.topic(); + return topic != null && topic.equals(schemaChangeTopic); + } + @Override public void handleBatch( List> events, @@ -131,6 +143,7 @@ public void handleBatch( currentRecordCommitter = committer; for (ChangeEvent event : events) { var record = event.value(); + LOG.info("connect record: {}", record); EventType eventType = getEventType(record); DebeziumOffset offset = new DebeziumOffset( @@ -176,6 +189,49 @@ var record = event.value(); respBuilder.addEvents(message); break; } + + case SCHEMA_CHANGE: + { + var ddl = ((Struct) record.value()).getString("ddl"); + if (ddl.contains("CREATE") || ddl.contains("DROP")) { + LOG.info("skip create/drop table event"); + continue; + } + + var sourceStruct = ((Struct) record.value()).getStruct("source"); + if (sourceStruct == null) { + throw new CdcConnectorException( + "source field is missing in schema change event"); + } + + // upstream event time + long sourceTsMs = sourceStruct.getInt64("ts_ms"); + // concat full table name, right now we only support MySQL schema change + // event + var fullTableName = + String.format( + "%s.%s", + sourceStruct.getString("db"), + sourceStruct.getString("table")); + byte[] payload = + payloadConverter.fromConnectData( + record.topic(), record.valueSchema(), record.value()); + + var message = + msgBuilder + .setFullTableName(fullTableName) + .setPayload(new String(payload, StandardCharsets.UTF_8)) + .setSourceTsMs(sourceTsMs) + .build(); + LOG.info( + "offset => {}, key => {}, payload => {}", + message.getOffset(), + message.getKey(), + message.getPayload()); + respBuilder.addEvents(message); + break; + } + case DATA: { // Topic naming conventions @@ -192,16 +248,18 @@ var record = event.value(); } // get upstream event time from the "source" field var sourceStruct = ((Struct) record.value()).getStruct("source"); - long sourceTsMs = - sourceStruct == null - ? System.currentTimeMillis() - : sourceStruct.getInt64("ts_ms"); + if (sourceStruct == null) { + throw new CdcConnectorException( + "source field is missing in data change event"); + } + long sourceTsMs = sourceStruct.getInt64("ts_ms"); byte[] payload = payloadConverter.fromConnectData( record.topic(), record.valueSchema(), record.value()); byte[] key = keyConverter.fromConnectData( record.topic(), record.keySchema(), record.key()); + var message = msgBuilder .setFullTableName(fullTableName) diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties index 0c62a51986b1c..a1dc27879fc6f 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties @@ -13,7 +13,7 @@ table.include.list=${database.name}.${table.name:-*} schema.history.internal.store.only.captured.tables.ddl=true schema.history.internal.store.only.captured.databases.ddl=true # default to disable schema change events -include.schema.changes=${debezium.include.schema.changes:-false} +include.schema.changes=${debezium.include.schema.changes:-true} database.server.id=${server.id} # default to use unencrypted connection database.ssl.mode=${ssl.mode:-disabled} From 99c078581c73867f2b4a7fdce32a36376d8475ef Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 8 Jul 2024 16:10:37 +0800 Subject: [PATCH 02/75] split schema event to a chunk --- .../connector/source/core/DbzChangeEventConsumer.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index fb74a192917c8..e0f8591212467 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -143,7 +143,6 @@ public void handleBatch( currentRecordCommitter = committer; for (ChangeEvent event : events) { var record = event.value(); - LOG.info("connect record: {}", record); EventType eventType = getEventType(record); DebeziumOffset offset = new DebeziumOffset( @@ -229,6 +228,14 @@ var record = event.value(); message.getKey(), message.getPayload()); respBuilder.addEvents(message); + + // emit the schema change event as a single response + respBuilder.setSourceId(sourceId); + var response = respBuilder.build(); + outputChannel.put(response); + + // reset the response builder + respBuilder = GetEventStreamResponse.newBuilder(); break; } From dbb18a5f80fa37a02ce961f3c27e498bb5b67154 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 11 Jul 2024 17:10:29 +0800 Subject: [PATCH 03/75] skeleton to parse cdc message --- proto/connector_service.proto | 14 +++- src/connector/Cargo.toml | 1 + src/connector/src/parser/mod.rs | 14 +++- src/connector/src/parser/plain_parser.rs | 72 +++++++++++++++---- src/connector/src/parser/unified/debezium.rs | 21 +++++- .../src/source/cdc/source/message.rs | 36 ++++++++-- 6 files changed, 135 insertions(+), 23 deletions(-) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index da2c2b88087ea..13cf2b2f68cbc 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -147,15 +147,27 @@ message SinkCoordinatorStreamResponse { /* Source Service */ message CdcMessage { + enum CdcMessageType { + UNSPECIFIED = 0; + DATA = 1; + TRANSACTION_META = 2; + SCHEMA_CHANGE = 3; + } + // The value of the Debezium message string payload = 1; string partition = 2; string offset = 3; string full_table_name = 4; int64 source_ts_ms = 5; - bool is_transaction_meta = 6; + + // Deprecated: use `type` instead + reserved "is_transaction_meta"; + reserved 6; + // The key of the Debezium message, which only used by `mongodb-cdc` connector. string key = 7; + CdcMessageType msg_type = 8; } enum SourceType { diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index bccd77faf8905..bf8c61716d1a9 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -134,6 +134,7 @@ sea-schema = { version = "0.14", default-features = false, features = [ "discovery", "sqlx-postgres", "sqlx-mysql", + "sqlx-sqlite", ] } serde = { version = "1", features = ["derive", "rc"] } serde_derive = "1" diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index fee737fce1106..913121fb64545 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -27,7 +27,9 @@ pub use json_parser::*; pub use protobuf::*; use risingwave_common::array::{ArrayBuilderImpl, Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::{KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME}; +use risingwave_common::catalog::{ + ColumnCatalog, KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME, +}; use risingwave_common::log::LogSuppresser; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::types::{Datum, DatumCow, DatumRef, ScalarRefImpl}; @@ -568,6 +570,11 @@ pub enum TransactionControl { Commit { id: Box }, } +#[derive(Debug)] +pub struct TableSchemaChange { + columns: Vec, +} + /// The result of parsing a message. #[derive(Debug)] pub enum ParseResult { @@ -575,6 +582,9 @@ pub enum ParseResult { Rows, /// A transaction control message is parsed. TransactionControl(TransactionControl), + + /// A schema change message is parsed. + SchemaChange(TableSchemaChange), } #[derive(Clone, Copy, Debug, PartialEq)] @@ -825,6 +835,8 @@ async fn into_chunk_stream_inner( } } }, + + Ok(ParseResult::SchemaChange(_)) => todo!(), } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index f34fd29837174..21f3ea4745e01 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -23,10 +23,11 @@ use super::{ use crate::error::ConnectorResult; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; -use crate::parser::unified::debezium::parse_transaction_meta; +use crate::parser::unified::debezium::{parse_schema_change, parse_transaction_meta}; use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; use crate::parser::{BytesProperties, ParseResult, ParserFormat}; +use crate::source::cdc::CdcMessageType; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMeta}; /// Parser for `FORMAT PLAIN`, i.e., append-only source. @@ -38,6 +39,7 @@ pub struct PlainParser { pub source_ctx: SourceContextRef, // parsing transaction metadata for shared cdc source pub transaction_meta_builder: Option, + pub schema_change_builder: Option, } impl PlainParser { @@ -69,12 +71,18 @@ impl PlainParser { let transaction_meta_builder = Some(AccessBuilderImpl::DebeziumJson( DebeziumJsonAccessBuilder::new(TimestamptzHandling::GuessNumberUnit)?, )); + + let schema_change_builder = Some(AccessBuilderImpl::DebeziumJson( + DebeziumJsonAccessBuilder::new(TimestamptzHandling::GuessNumberUnit)?, + )); + Ok(Self { key_builder, payload_builder, rw_columns, source_ctx, transaction_meta_builder, + schema_change_builder, }) } @@ -82,26 +90,62 @@ impl PlainParser { &mut self, key: Option>, payload: Option>, - mut writer: SourceStreamChunkRowWriter<'_>, + writer: SourceStreamChunkRowWriter<'_>, ) -> ConnectorResult { - // if the message is transaction metadata, parse it and return + // plain parser also used in the shared cdc source, + // we need to handle transaction metadata and schema change messages here if let Some(msg_meta) = writer.row_meta && let SourceMeta::DebeziumCdc(cdc_meta) = msg_meta.meta - && cdc_meta.is_transaction_meta && let Some(data) = payload { - let accessor = self - .transaction_meta_builder - .as_mut() - .expect("expect transaction metadata access builder") - .generate_accessor(data) - .await?; - return match parse_transaction_meta(&accessor, &self.source_ctx.connector_props) { - Ok(transaction_control) => Ok(ParseResult::TransactionControl(transaction_control)), - Err(err) => Err(err)?, - }; + match cdc_meta.msg_type { + CdcMessageType::Data => { + return self.parse_rows(key, Some(data), writer).await; + } + CdcMessageType::TransactionMeta => { + let accessor = self + .transaction_meta_builder + .as_mut() + .expect("expect transaction metadata access builder") + .generate_accessor(data) + .await?; + return match parse_transaction_meta(&accessor, &self.source_ctx.connector_props) + { + Ok(transaction_control) => { + Ok(ParseResult::TransactionControl(transaction_control)) + } + Err(err) => Err(err)?, + }; + } + CdcMessageType::SchemaChange => { + let accessor = self + .schema_change_builder + .as_mut() + .expect("expect schema change access builder") + .generate_accessor(data) + .await?; + + return match parse_schema_change(&accessor, &self.source_ctx.connector_props) { + Ok(schema_change) => Ok(ParseResult::SchemaChange(schema_change)), + Err(err) => Err(err)?, + }; + } + CdcMessageType::Unknown => { + unreachable!() + } + } } + // for non-cdc source messages + self.parse_rows(key, payload, writer).await + } + + async fn parse_rows( + &mut self, + key: Option>, + payload: Option>, + mut writer: SourceStreamChunkRowWriter<'_>, + ) -> ConnectorResult { let mut row_op: KvEvent, AccessImpl<'_>> = KvEvent::default(); if let Some(data) = key diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index d90463698577d..7760eaf7f4469 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -19,7 +19,7 @@ use risingwave_connector_codec::decoder::AccessExt; use risingwave_pb::plan_common::additional_column::ColumnType; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; -use crate::parser::TransactionControl; +use crate::parser::{TableSchemaChange, TransactionControl}; use crate::source::{ConnectorProperties, SourceColumnDesc}; // Example of Debezium JSON value: @@ -75,6 +75,8 @@ const OP: &str = "op"; pub const TRANSACTION_STATUS: &str = "status"; pub const TRANSACTION_ID: &str = "id"; +pub const TABLE_CHANGES: &str = "tableChanges"; + pub const DEBEZIUM_READ_OP: &str = "r"; pub const DEBEZIUM_CREATE_OP: &str = "c"; pub const DEBEZIUM_UPDATE_OP: &str = "u"; @@ -129,6 +131,23 @@ pub fn parse_transaction_meta( }) } +pub fn parse_schema_change( + accessor: &impl Access, + _connector_props: &ConnectorProperties, +) -> AccessResult { + if let Some(ScalarRefImpl::Jsonb(table_changes)) = accessor + .access(&[TABLE_CHANGES], &DataType::Jsonb)? + .to_datum_ref() + { + tracing::info!("table_changes: {:?}", table_changes); + } + + Err(AccessError::Undefined { + name: "table schema change".into(), + path: TRANSACTION_STATUS.into(), + }) +} + impl DebeziumChangeEvent where A: Access, diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index f12d18339b527..4c79257adb71d 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -13,11 +13,30 @@ // limitations under the License. use risingwave_common::types::{DatumRef, ScalarRefImpl, Timestamptz}; -use risingwave_pb::connector_service::CdcMessage; +use risingwave_pb::connector_service::{cdc_message, CdcMessage}; use crate::source::base::SourceMessage; use crate::source::SourceMeta; +#[derive(Clone, Debug)] +pub enum CdcMessageType { + Unknown, + Data, + TransactionMeta, + SchemaChange, +} + +impl From for CdcMessageType { + fn from(msg_type: cdc_message::CdcMessageType) -> Self { + match msg_type { + cdc_message::CdcMessageType::Data => CdcMessageType::Data, + cdc_message::CdcMessageType::TransactionMeta => CdcMessageType::TransactionMeta, + cdc_message::CdcMessageType::SchemaChange => CdcMessageType::SchemaChange, + cdc_message::CdcMessageType::Unspecified => CdcMessageType::Unknown, + } + } +} + #[derive(Debug, Clone)] pub struct DebeziumCdcMeta { db_name_prefix_len: usize, @@ -25,11 +44,11 @@ pub struct DebeziumCdcMeta { pub full_table_name: String, // extracted from `payload.source.ts_ms`, the time that the change event was made in the database pub source_ts_ms: i64, - // Whether the message is a transaction metadata - pub is_transaction_meta: bool, + pub msg_type: CdcMessageType, } impl DebeziumCdcMeta { + // These `extract_xxx` methods are used to support the `INCLUDE TIMESTAMP/DATABASE_NAME/TABLE_NAME` feature pub fn extract_timestamp(&self) -> DatumRef<'_> { Some(ScalarRefImpl::Timestamptz( Timestamptz::from_millis(self.source_ts_ms).unwrap(), @@ -48,20 +67,25 @@ impl DebeziumCdcMeta { )) } - pub fn new(full_table_name: String, source_ts_ms: i64, is_transaction_meta: bool) -> Self { + pub fn new( + full_table_name: String, + source_ts_ms: i64, + msg_type: cdc_message::CdcMessageType, + ) -> Self { // full_table_name is in the format of `database_name.table_name` let db_name_prefix_len = full_table_name.as_str().find('.').unwrap_or(0); Self { db_name_prefix_len, full_table_name, source_ts_ms, - is_transaction_meta, + msg_type: msg_type.into(), } } } impl From for SourceMessage { fn from(message: CdcMessage) -> Self { + let msg_type = message.get_msg_type().expect("invalid message type"); SourceMessage { key: if message.key.is_empty() { None // only data message has key @@ -78,7 +102,7 @@ impl From for SourceMessage { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( message.full_table_name, message.source_ts_ms, - message.is_transaction_meta, + msg_type, )), } } From 5971ea25a73aef9be667ed236d91c000fa24b10a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 11 Jul 2024 17:54:40 +0800 Subject: [PATCH 04/75] minor --- .../connector/source/core/DbzChangeEventConsumer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index bc328c46c7cfc..d3343b8ae2a8b 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -180,7 +180,7 @@ var record = event.value(); record.topic(), record.valueSchema(), record.value()); var message = msgBuilder - .setIsTransactionMeta(true) + .setMsgType(CdcMessage.CdcMessageType.TRANSACTION_META) .setPayload(new String(payload, StandardCharsets.UTF_8)) .setSourceTsMs(trxTs) .build(); @@ -218,6 +218,7 @@ var record = event.value(); var message = msgBuilder + .setMsgType(CdcMessage.CdcMessageType.SCHEMA_CHANGE) .setFullTableName(fullTableName) .setPayload(new String(payload, StandardCharsets.UTF_8)) .setSourceTsMs(sourceTsMs) @@ -272,6 +273,7 @@ var record = event.value(); String msgKey = key == null ? "" : new String(key, StandardCharsets.UTF_8); var message = msgBuilder + .setMsgType(CdcMessage.CdcMessageType.DATA) .setFullTableName(fullTableName) .setPayload(msgPayload) .setKey(msgKey) From 1d60a6a01aeb3457b87a62ccec3b9f16bfea1a26 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 16 Jul 2024 16:36:54 +0800 Subject: [PATCH 05/75] impl mysql schema event parser --- src/common/src/types/jsonb.rs | 7 ++ src/connector/codec/src/decoder/mod.rs | 3 + src/connector/src/parser/debezium/mod.rs | 2 + .../src/parser/debezium/schema_change.rs | 12 +++ .../src/parser/debezium/simd_json_parser.rs | 7 ++ src/connector/src/parser/mod.rs | 8 +- src/connector/src/parser/mysql.rs | 19 ++++ src/connector/src/parser/plain_parser.rs | 72 ++++++++++++-- src/connector/src/parser/unified/debezium.rs | 96 ++++++++++++++++--- 9 files changed, 202 insertions(+), 24 deletions(-) create mode 100644 src/connector/src/parser/debezium/schema_change.rs diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 642b363a8c67e..a16d3ca53a255 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -301,6 +301,13 @@ impl<'a> JsonbRef<'a> { .ok_or_else(|| format!("cannot cast jsonb {} to type boolean", self.type_name())) } + pub fn as_string(&self) -> Result { + self.0 + .as_str() + .map(|s| s.to_owned()) + .ok_or_else(|| format!("cannot cast jsonb {} to type string", self.type_name())) + } + /// Attempt to read jsonb as a JSON number. /// /// According to RFC 8259, only number within IEEE 754 binary64 (double precision) has good diff --git a/src/connector/codec/src/decoder/mod.rs b/src/connector/codec/src/decoder/mod.rs index cd7fe14ab74ea..2e62aabb22b85 100644 --- a/src/connector/codec/src/decoder/mod.rs +++ b/src/connector/codec/src/decoder/mod.rs @@ -44,6 +44,9 @@ pub enum AccessError { #[error(transparent)] NotImplemented(#[from] NotImplemented), + + #[error(transparent)] + Other(#[from] anyhow::Error), } pub type AccessResult = std::result::Result; diff --git a/src/connector/src/parser/debezium/mod.rs b/src/connector/src/parser/debezium/mod.rs index 5b5416e647268..8852bfc25eb41 100644 --- a/src/connector/src/parser/debezium/mod.rs +++ b/src/connector/src/parser/debezium/mod.rs @@ -17,7 +17,9 @@ mod avro_parser; mod debezium_parser; mod mongo_json_parser; +pub mod schema_change; pub mod simd_json_parser; + pub use avro_parser::*; pub use debezium_parser::*; pub use mongo_json_parser::DebeziumMongoJsonParser; diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs new file mode 100644 index 0000000000000..0881cd26cc503 --- /dev/null +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -0,0 +1,12 @@ +use risingwave_common::catalog::ColumnCatalog; + +#[derive(Debug)] +pub struct SchemaChangeEnvelope { + pub table_changes: Vec, +} + +#[derive(Debug)] +pub struct TableSchemaChange { + pub(crate) up_table_full_name: String, + pub(crate) columns: Vec, +} diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index f9738eb9e357e..08dd4ef7c2bdc 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -37,6 +37,13 @@ impl DebeziumJsonAccessBuilder { json_parse_options: JsonParseOptions::new_for_debezium(timestamptz_handling), }) } + + pub fn new_for_schema_event() -> ConnectorResult { + Ok(Self { + value: None, + json_parse_options: JsonParseOptions::default(), + }) + } } impl AccessBuilder for DebeziumJsonAccessBuilder { diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 913121fb64545..964221108e5d9 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -87,6 +87,7 @@ mod unified; mod upsert_parser; mod util; +use debezium::schema_change::SchemaChangeEnvelope; pub use debezium::DEBEZIUM_IGNORE_KEY; use risingwave_common::bitmap::BitmapBuilder; pub use unified::{AccessError, AccessResult}; @@ -570,11 +571,6 @@ pub enum TransactionControl { Commit { id: Box }, } -#[derive(Debug)] -pub struct TableSchemaChange { - columns: Vec, -} - /// The result of parsing a message. #[derive(Debug)] pub enum ParseResult { @@ -584,7 +580,7 @@ pub enum ParseResult { TransactionControl(TransactionControl), /// A schema change message is parsed. - SchemaChange(TableSchemaChange), + SchemaChange(SchemaChangeEnvelope), } #[derive(Clone, Copy, Debug, PartialEq)] diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index d1df27263e808..1b7cef53e3073 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -141,6 +141,25 @@ pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> Owne OwnedRow::new(datums) } +pub fn mysql_typename_to_rw_type(type_name: &str) -> anyhow::Result { + match type_name.to_lowercase().as_str() { + "tinyint" | "smallint" => Ok(DataType::Int16), + "int" => Ok(DataType::Int32), + "bigint" => Ok(DataType::Int64), + "float" => Ok(DataType::Float32), + "double" => Ok(DataType::Float64), + "decimal" => Ok(DataType::Decimal), + "char" | "varchar" | "text" | "mediumtext" | "longtext" => Ok(DataType::Varchar), + "date" => Ok(DataType::Date), + "time" => Ok(DataType::Time), + "timestamp" => Ok(DataType::Timestamptz), + "datetime" => Ok(DataType::Timestamptz), + "json" => Ok(DataType::Jsonb), + "binary" | "varbinary" | "blob" | "mediumblob" | "longblob" => Ok(DataType::Bytea), + _ => Err(anyhow::anyhow!("unsupported type: {}", type_name)), + } +} + #[cfg(test)] mod tests { diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 21f3ea4745e01..ec4d631a1011c 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -26,7 +26,9 @@ use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::{parse_schema_change, parse_transaction_meta}; use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; -use crate::parser::{BytesProperties, ParseResult, ParserFormat}; +use crate::parser::{ + BytesProperties, JsonAccessBuilder, JsonProperties, ParseResult, ParserFormat, +}; use crate::source::cdc::CdcMessageType; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMeta}; @@ -73,7 +75,7 @@ impl PlainParser { )); let schema_change_builder = Some(AccessBuilderImpl::DebeziumJson( - DebeziumJsonAccessBuilder::new(TimestamptzHandling::GuessNumberUnit)?, + DebeziumJsonAccessBuilder::new_for_schema_event()?, )); Ok(Self { @@ -207,6 +209,7 @@ mod tests { use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; + use risingwave_pb::connector_service::cdc_message; use super::*; use crate::parser::{MessageMeta, SourceStreamChunkBuilder, TransactionControl}; @@ -325,7 +328,7 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional, + cdc_message::CdcMessageType::TransactionMeta, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -339,7 +342,7 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - false, + cdc_message::CdcMessageType::Data, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -353,7 +356,7 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional, + cdc_message::CdcMessageType::TransactionMeta, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -399,7 +402,11 @@ mod tests { let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"3E11FA47-71CA-11E1-9E33-C80AA9429562:23","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#; let commit_msg = r#"{"schema":null,"payload":{"status":"END","id":"3E11FA47-71CA-11E1-9E33-C80AA9429562:23","event_count":11,"data_collections":[{"data_collection":"public.orders_tx","event_count":5},{"data_collection":"public.person","event_count":6}],"ts_ms":1704269323180}}"#; - let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new("orders".to_string(), 0, true)); + let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( + "orders".to_string(), + 0, + cdc_message::CdcMessageType::TransactionMeta, + )); let msg_meta = MessageMeta { meta: &cdc_meta, split_id: "1001", @@ -437,4 +444,57 @@ mod tests { let output = builder.take(10); assert_eq!(0, output.cardinality()); } + + #[tokio::test] + async fn test_parse_schema_change() { + let schema = vec![ + ColumnCatalog { + column_desc: ColumnDesc::named("payload", ColumnId::placeholder(), DataType::Jsonb), + is_hidden: false, + }, + ColumnCatalog::offset_column(), + ColumnCatalog::cdc_table_name_column(), + ]; + + let columns = schema + .iter() + .map(|c| SourceColumnDesc::from(&c.column_desc)) + .collect::>(); + + // format plain encode json parser + let source_ctx = SourceContext { + connector_props: ConnectorProperties::MysqlCdc(Box::default()), + ..SourceContext::dummy() + }; + let mut parser = PlainParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + columns.clone(), + Arc::new(source_ctx), + ) + .await + .unwrap(); + let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); + + let msg = r#"{"schema":null,"payload": { "databaseName": "mydb", "ddl": "ALTER TABLE test add column v2 varchar(32)", "schemaName": null, "source": { "connector": "mysql", "db": "mydb", "file": "binlog.000065", "gtid": null, "name": "RW_CDC_1", "pos": 234, "query": null, "row": 0, "sequence": null, "server_id": 1, "snapshot": "false", "table": "test", "thread": null, "ts_ms": 1718354727000, "version": "2.4.2.Final" }, "tableChanges": [ { "id": "\"mydb\".\"test\"", "table": { "columns": [ { "autoIncremented": false, "charsetName": null, "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 4, "length": null, "name": "id", "nativeType": null, "optional": false, "position": 1, "scale": null, "typeExpression": "INT", "typeName": "INT" }, { "autoIncremented": false, "charsetName": null, "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 2014, "length": null, "name": "v1", "nativeType": null, "optional": true, "position": 2, "scale": null, "typeExpression": "TIMESTAMP", "typeName": "TIMESTAMP" }, { "autoIncremented": false, "charsetName": "utf8mb4", "comment": null, "defaultValueExpression": null, "enumValues": null, "generated": false, "jdbcType": 12, "length": 32, "name": "v2", "nativeType": null, "optional": true, "position": 3, "scale": null, "typeExpression": "VARCHAR", "typeName": "VARCHAR" } ], "comment": null, "defaultCharsetName": "utf8mb4", "primaryKeyColumnNames": [ "id" ] }, "type": "ALTER" } ], "ts_ms": 1718354727594 }}"#; + let cdc_meta = SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( + "mydb.test".to_string(), + 0, + cdc_message::CdcMessageType::SchemaChange, + )); + let msg_meta = MessageMeta { + meta: &cdc_meta, + split_id: "1001", + offset: "", + }; + + let res = parser + .parse_one_with_txn( + None, + Some(msg.as_bytes().to_vec()), + builder.row_writer().with_meta(msg_meta), + ) + .await; + + res.unwrap(); + } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 7760eaf7f4469..af74a80fc13ab 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -12,14 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; +use jsonbb::ValueRef; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; use risingwave_common::types::{ - DataType, Datum, DatumCow, Scalar, ScalarImpl, ScalarRefImpl, Timestamptz, ToDatumRef, + DataType, Datum, DatumCow, Scalar, ScalarImpl, ScalarRefImpl, StructType, Timestamptz, + ToDatumRef, }; use risingwave_connector_codec::decoder::AccessExt; use risingwave_pb::plan_common::additional_column::ColumnType; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; -use crate::parser::{TableSchemaChange, TransactionControl}; +use crate::parser::debezium::schema_change::{SchemaChangeEnvelope, TableSchemaChange}; +use crate::parser::mysql::mysql_typename_to_rw_type; +use crate::parser::TransactionControl; use crate::source::{ConnectorProperties, SourceColumnDesc}; // Example of Debezium JSON value: @@ -133,19 +139,85 @@ pub fn parse_transaction_meta( pub fn parse_schema_change( accessor: &impl Access, - _connector_props: &ConnectorProperties, -) -> AccessResult { - if let Some(ScalarRefImpl::Jsonb(table_changes)) = accessor - .access(&[TABLE_CHANGES], &DataType::Jsonb)? + connector_props: &ConnectorProperties, +) -> AccessResult { + let mut schema_changes = vec![]; + + if let Some(ScalarRefImpl::List(table_changes)) = accessor + .access(&[TABLE_CHANGES], &DataType::List(Box::new(DataType::Jsonb)))? .to_datum_ref() { - tracing::info!("table_changes: {:?}", table_changes); - } + for datum in table_changes.iter() { + let jsonb = match datum { + Some(ScalarRefImpl::Jsonb(jsonb)) => jsonb, + _ => unreachable!(""), + }; - Err(AccessError::Undefined { - name: "table schema change".into(), - path: TRANSACTION_STATUS.into(), - }) + let id = jsonb + .access_object_field("id") + .unwrap() + .as_string() + .unwrap(); + + println!("id: {}", id); + + let mut column_descs: Vec = vec![]; + if let Some(table) = jsonb.access_object_field("table") + && let Some(columns) = table.access_object_field("columns") + { + for col in columns.array_elements().unwrap() { + let name = col + .access_object_field("name") + .unwrap() + .as_string() + .unwrap(); + let type_name = col + .access_object_field("typeName") + .unwrap() + .as_string() + .unwrap(); + let position = col + .access_object_field("position") + .unwrap() + .as_number() + .unwrap(); + + let data_type = match *connector_props { + ConnectorProperties::PostgresCdc(_) => { + unimplemented!() + } + ConnectorProperties::MysqlCdc(_) => { + mysql_typename_to_rw_type(type_name.as_str())? + } + _ => { + unreachable!() + } + }; + + column_descs.push(ColumnDesc::named(name, ColumnId::placeholder(), data_type)); + } + } + schema_changes.push(TableSchemaChange { + up_table_full_name: id, + columns: column_descs + .into_iter() + .map(|column_desc| ColumnCatalog { + column_desc, + is_hidden: false, + }) + .collect_vec(), + }); + } + + Ok(SchemaChangeEnvelope { + table_changes: schema_changes, + }) + } else { + Err(AccessError::Undefined { + name: "table schema change".into(), + path: TABLE_CHANGES.into(), + }) + } } impl DebeziumChangeEvent From 7d2a408afef7a82b1489ac4c645a697c1facc1ad Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 21 Jul 2024 19:16:03 +0800 Subject: [PATCH 06/75] meta->frontend rpc skeleton --- proto/ddl_service.proto | 14 ++ proto/frontend_service.proto | 39 +++++ .../src/parser/debezium/schema_change.rs | 20 +++ src/connector/src/parser/mod.rs | 7 +- .../src/schema/auto_schema_change/mod.rs | 25 +++ src/connector/src/schema/mod.rs | 1 + src/frontend/src/lib.rs | 10 +- src/frontend/src/rpc/mod.rs | 57 ++++++ src/frontend/src/session.rs | 164 +++++++++++------- src/frontend/src/test_utils.rs | 8 + src/meta/service/src/ddl_service.rs | 15 ++ src/prost/build.rs | 1 + src/prost/src/lib.rs | 3 + src/utils/pgwire/src/pg_server.rs | 18 +- 14 files changed, 314 insertions(+), 68 deletions(-) create mode 100644 proto/frontend_service.proto create mode 100644 src/connector/src/schema/auto_schema_change/mod.rs create mode 100644 src/frontend/src/rpc/mod.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 7c6a078ce9ce4..134c50310aefe 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -6,6 +6,7 @@ import "catalog.proto"; import "common.proto"; import "meta.proto"; import "stream_plan.proto"; +import "plan_common.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; @@ -434,6 +435,18 @@ message CommentOnResponse { uint64 version = 2; } +message SchemaChangeEnvelope { + string up_table_name = 1; + repeated plan_common.ColumnCatalog column_descs = 2; +} + +message AutoSchemaChangeRequest { + SchemaChangeEnvelope schema_change = 1; +} + +message AutoSchemaChangeResponse {} + + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -472,4 +485,5 @@ service DdlService { rpc GetTables(GetTablesRequest) returns (GetTablesResponse); rpc Wait(WaitRequest) returns (WaitResponse); rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); + rpc AutoSchemaChange(AutoSchemaChangeRequest) returns (AutoSchemaChangeResponse); } diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto new file mode 100644 index 0000000000000..81666cc3f6d81 --- /dev/null +++ b/proto/frontend_service.proto @@ -0,0 +1,39 @@ +syntax = "proto3"; + +package frontend_service; + +import "ddl_service.proto"; + +option java_package = "com.risingwave.proto"; +option optimize_for = SPEED; + + +message SchemaChangeRequest { + message GetReplaceTablePlanRequest { + uint32 database_id = 1; + string owner = 2; + ddl_service.SchemaChangeEnvelope schema_change = 3; + } + + oneof request { + GetReplaceTablePlanRequest replace_table_plan = 1; + } + +} + +message SchemaChangeResponse { + message GetReplaceTablePlanResponse { + ddl_service.ReplaceTablePlan table_plan = 1; + } + + oneof response { + GetReplaceTablePlanResponse replace_table_plan = 1; + } + +} + + +// Frontend will start a background worker to +service SchemaChangeService { + rpc GetNewTableStreamingGraph(SchemaChangeRequest) returns (SchemaChangeResponse); +} diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 0881cd26cc503..958748d04bb31 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -1,3 +1,17 @@ +// 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 risingwave_common::catalog::ColumnCatalog; #[derive(Debug)] @@ -10,3 +24,9 @@ pub struct TableSchemaChange { pub(crate) up_table_full_name: String, pub(crate) columns: Vec, } + +impl SchemaChangeEnvelope { + pub fn to_protobuf(&self) { + todo!() + } +} diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 60f02f6474638..1f263ac5adde4 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -836,7 +836,12 @@ async fn into_chunk_stream_inner( } }, - Ok(ParseResult::SchemaChange(_)) => todo!(), + Ok(ParseResult::SchemaChange(schema_change)) => { + // TODO: + // 1. block source executor + // 2. send schema change to Meta + // 3. wait for Meta to finish schema change + } } } diff --git a/src/connector/src/schema/auto_schema_change/mod.rs b/src/connector/src/schema/auto_schema_change/mod.rs new file mode 100644 index 0000000000000..44b9c88f9e347 --- /dev/null +++ b/src/connector/src/schema/auto_schema_change/mod.rs @@ -0,0 +1,25 @@ +// 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 crate::parser::schema_change::SchemaChangeEnvelope; + +/// client for auto schema change +/// Can be a global client +pub struct AutoSchemaChangeClient {} + +impl AutoSchemaChangeClient { + pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { + Ok(()) + } +} diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 9b3757e29c094..b7a50f72a1817 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -14,6 +14,7 @@ use crate::error::ConnectorError; +pub mod auto_schema_change; pub mod avro; mod loader; pub mod protobuf; diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 185d65cb567a0..c43074cbac7d8 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -37,6 +37,7 @@ #![feature(used_with_arg)] #![feature(entry_insert)] #![recursion_limit = "256"] +#![feature(once_cell_try)] #[cfg(test)] risingwave_expr_impl::enable!(); @@ -74,6 +75,7 @@ mod user; pub mod health_service; mod monitor; +pub mod rpc; mod telemetry; use std::ffi::OsString; @@ -175,6 +177,8 @@ use std::pin::Pin; use pgwire::pg_protocol::TlsConfig; +use crate::session::SESSION_MANAGER; + /// Start frontend pub fn start( opts: FrontendOpts, @@ -184,7 +188,9 @@ pub fn start( // slow compile in release mode. Box::pin(async move { let listen_addr = opts.listen_addr.clone(); - let session_mgr = SessionManagerImpl::new(opts).await.unwrap(); + let session_mgr = SESSION_MANAGER + .get_or_init(|| async { Arc::new(SessionManagerImpl::new(opts).await.unwrap()) }) + .await; let redact_sql_option_keywords = Arc::new( session_mgr .env() @@ -197,7 +203,7 @@ pub fn start( pg_serve( &listen_addr, - session_mgr, + session_mgr.clone(), TlsConfig::new_default(), Some(redact_sql_option_keywords), shutdown, diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs new file mode 100644 index 0000000000000..f312f9a31c61d --- /dev/null +++ b/src/frontend/src/rpc/mod.rs @@ -0,0 +1,57 @@ +// 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 pgwire::pg_server::SessionManager; +use risingwave_pb::frontend_service::schema_change_request::Request; +use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; +use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; + +use crate::session::SESSION_MANAGER; + +#[derive(Default)] +pub struct SchemaChangeServiceImpl {} + +impl SchemaChangeServiceImpl { + pub fn new() -> Self { + Self {} + } +} + +#[async_trait::async_trait] +impl SchemaChangeService for SchemaChangeServiceImpl { + async fn get_new_table_streaming_graph( + &self, + request: RpcRequest, + ) -> Result, Status> { + let req = request.into_inner(); + + match req.request.unwrap() { + Request::ReplaceTablePlan(req) => { + let change = req.schema_change.expect("schema_change"); + // get a session object + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + let _session = session_mgr + .get_session(req.database_id, &req.owner) + .map_err(|e| Status::internal(format!("Failed to get session: {}", e)))?; + + // call the handle alter method + } + }; + + Ok(RpcResponse::new(SchemaChangeResponse { response: None })) + } +} diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 7c694ba002662..df3cd6cc6e94e 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -14,7 +14,6 @@ use std::collections::HashMap; use std::io::{Error, ErrorKind}; -#[cfg(test)] use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::sync::atomic::{AtomicI32, Ordering}; use std::sync::{Arc, Weak}; @@ -80,6 +79,7 @@ use tokio::sync::oneshot::Sender; use tokio::sync::watch; use tokio::task::JoinHandle; use tracing::info; +use tracing::log::error; use self::cursor_manager::CursorManager; use crate::binder::{Binder, BoundStatement, ResolveQualifiedNameError}; @@ -377,6 +377,7 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) + // .add_service() .serve(host.parse().unwrap()) .await .unwrap(); @@ -1121,6 +1122,9 @@ impl SessionImpl { } } +pub static SESSION_MANAGER: tokio::sync::OnceCell> = + tokio::sync::OnceCell::const_new(); + pub struct SessionManagerImpl { env: FrontendEnv, _join_handles: Vec>, @@ -1131,6 +1135,18 @@ pub struct SessionManagerImpl { impl SessionManager for SessionManagerImpl { type Session = SessionImpl; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> std::result::Result, BoxedError> { + let dumb_addr = Address::Tcp(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), + 5691, // port of meta + )); + self.connect_inner(database_id, user_name, Arc::new(dumb_addr)) + } + fn connect( &self, database: &str, @@ -1148,6 +1164,89 @@ impl SessionManager for SessionManagerImpl { )) })? .id(); + + self.connect_inner(database_id, user_name, peer_addr) + } + + /// Used when cancel request happened. + fn cancel_queries_in_session(&self, session_id: SessionId) { + self.env.cancel_queries_in_session(session_id); + } + + fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { + self.env.cancel_creating_jobs_in_session(session_id); + } + + fn end_session(&self, session: &Self::Session) { + self.delete_session(&session.session_id()); + } + + async fn shutdown(&self) { + // Clean up the session map. + self.env.sessions_map().write().clear(); + // Unregister from the meta service. + self.env.meta_client().try_unregister().await; + } +} + +impl SessionManagerImpl { + pub async fn new(opts: FrontendOpts) -> Result { + // TODO(shutdown): only save join handles that **need** to be shutdown + let (env, join_handles, shutdown_senders) = FrontendEnv::init(opts).await?; + Ok(Self { + env, + _join_handles: join_handles, + _shutdown_senders: shutdown_senders, + number: AtomicI32::new(0), + }) + } + + pub fn env(&self) -> &FrontendEnv { + &self.env + } + + fn insert_session(&self, session: Arc) { + let active_sessions = { + let mut write_guard = self.env.sessions_map.write(); + write_guard.insert(session.id(), session); + write_guard.len() + }; + self.env + .frontend_metrics + .active_sessions + .set(active_sessions as i64); + } + + fn delete_session(&self, session_id: &SessionId) { + let active_sessions = { + let mut write_guard = self.env.sessions_map.write(); + write_guard.remove(session_id); + write_guard.len() + }; + self.env + .frontend_metrics + .active_sessions + .set(active_sessions as i64); + } + + fn connect_inner( + &self, + database_id: u32, + user_name: &str, + peer_addr: AddressRef, + ) -> std::result::Result, BoxedError> { + let catalog_reader = self.env.catalog_reader(); + let reader = catalog_reader.read_guard(); + let database_name = reader + .get_database_by_id(&database_id) + .map_err(|_| { + Box::new(Error::new( + ErrorKind::InvalidInput, + format!("database \"{}\" does not exist", database_id), + )) + })? + .name(); + let user_reader = self.env.user_info_reader(); let reader = user_reader.read_guard(); if let Some(user) = reader.get_user_by_name(user_name) { @@ -1202,7 +1301,7 @@ impl SessionManager for SessionManagerImpl { let session_impl: Arc = SessionImpl::new( self.env.clone(), Arc::new(AuthContext::new( - database.to_string(), + database_name.to_string(), user_name.to_string(), user.id, )), @@ -1222,67 +1321,6 @@ impl SessionManager for SessionManagerImpl { ))) } } - - /// Used when cancel request happened. - fn cancel_queries_in_session(&self, session_id: SessionId) { - self.env.cancel_queries_in_session(session_id); - } - - fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { - self.env.cancel_creating_jobs_in_session(session_id); - } - - fn end_session(&self, session: &Self::Session) { - self.delete_session(&session.session_id()); - } - - async fn shutdown(&self) { - // Clean up the session map. - self.env.sessions_map().write().clear(); - // Unregister from the meta service. - self.env.meta_client().try_unregister().await; - } -} - -impl SessionManagerImpl { - pub async fn new(opts: FrontendOpts) -> Result { - // TODO(shutdown): only save join handles that **need** to be shutdown - let (env, join_handles, shutdown_senders) = FrontendEnv::init(opts).await?; - Ok(Self { - env, - _join_handles: join_handles, - _shutdown_senders: shutdown_senders, - number: AtomicI32::new(0), - }) - } - - pub fn env(&self) -> &FrontendEnv { - &self.env - } - - fn insert_session(&self, session: Arc) { - let active_sessions = { - let mut write_guard = self.env.sessions_map.write(); - write_guard.insert(session.id(), session); - write_guard.len() - }; - self.env - .frontend_metrics - .active_sessions - .set(active_sessions as i64); - } - - fn delete_session(&self, session_id: &SessionId) { - let active_sessions = { - let mut write_guard = self.env.sessions_map.write(); - write_guard.remove(session_id); - write_guard.len() - }; - self.env - .frontend_metrics - .active_sessions - .set(active_sessions as i64); - } } impl Session for SessionImpl { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 24453f766c72c..6c6d9958210d6 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -85,6 +85,14 @@ pub struct LocalFrontend { impl SessionManager for LocalFrontend { type Session = SessionImpl; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> std::result::Result, BoxedError> { + todo!() + } + fn connect( &self, _database: &str, diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 0eec9add5b185..560c6b369e946 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -916,6 +916,21 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(AlterParallelismResponse {})) } + + async fn auto_schema_change( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + + let schema_change = req.schema_change.unwrap(); + + // TODO: can we build a ReplaceTablePlan using the info in schema_change? + + // send a request to the frontend to get the ReplaceTablePlan + + Ok(Response::new(AutoSchemaChangeResponse {})) + } } impl DdlServiceImpl { diff --git a/src/prost/build.rs b/src/prost/build.rs index 729651b71d69a..b662e6dd93760 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -54,6 +54,7 @@ fn main() -> Result<(), Box> { "telemetry", "user", "secret", + "frontend_service", ]; let protos: Vec = proto_files .iter() diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index 2a20f2e5530d4..4c139cb5b7e20 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -83,6 +83,9 @@ pub mod monitor_service; #[cfg_attr(madsim, path = "sim/backup_service.rs")] pub mod backup_service; #[rustfmt::skip] +#[cfg_attr(madsim, path = "sim/frontend_service.rs")] +pub mod frontend_service; +#[rustfmt::skip] #[cfg_attr(madsim, path = "sim/java_binding.rs")] pub mod java_binding; #[rustfmt::skip] diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 840f21dda1be2..e785f5409f0dd 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -47,6 +47,12 @@ pub type SessionId = (ProcessId, SecretKey); pub trait SessionManager: Send + Sync + 'static { type Session: Session; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> Result, BoxedError>; + fn connect( &self, database: &str, @@ -257,7 +263,7 @@ impl UserAuthenticator { /// Returns when the `shutdown` token is triggered. pub async fn pg_serve( addr: &str, - session_mgr: impl SessionManager, + session_mgr: Arc, tls_config: Option, redact_sql_option_keywords: Option, shutdown: CancellationToken, @@ -280,7 +286,7 @@ pub async fn pg_serve( #[cfg(madsim)] let worker_runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); - let session_mgr = Arc::new(session_mgr); + // let session_mgr = Arc::new(session_mgr); let session_mgr_clone = session_mgr.clone(); let f = async move { loop { @@ -380,6 +386,14 @@ mod tests { impl SessionManager for MockSessionManager { type Session = MockSession; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> Result, BoxedError> { + todo!() + } + fn connect( &self, _database: &str, From e1cc54a36bdc6a816dd03f7abce27a0c380cc106 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 17:12:31 +0800 Subject: [PATCH 07/75] call frontend to generate new table plan --- proto/frontend_service.proto | 11 +- .../src/handler/alter_table_column.rs | 125 ++++++++++++++++-- src/frontend/src/handler/mod.rs | 2 + src/frontend/src/rpc/mod.rs | 71 ++++++++-- src/rpc_client/src/error.rs | 2 +- 5 files changed, 185 insertions(+), 26 deletions(-) diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 81666cc3f6d81..ce5f49a49be27 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -9,25 +9,26 @@ option optimize_for = SPEED; message SchemaChangeRequest { - message GetReplaceTablePlanRequest { + message GetNewTablePlanRequest { uint32 database_id = 1; string owner = 2; - ddl_service.SchemaChangeEnvelope schema_change = 3; + string table_name = 3; + ddl_service.SchemaChangeEnvelope schema_change = 4; } oneof request { - GetReplaceTablePlanRequest replace_table_plan = 1; + GetNewTablePlanRequest get_new_table_plan = 1; } } message SchemaChangeResponse { - message GetReplaceTablePlanResponse { + message GetNewTablePlanResponse { ddl_service.ReplaceTablePlan table_plan = 1; } oneof response { - GetReplaceTablePlanResponse replace_table_plan = 1; + GetNewTablePlanResponse replace_table_plan = 1; } } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 0af3ace68b6cb..704220cc7ccab 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -14,13 +14,19 @@ use std::sync::Arc; -use anyhow::Context; +use anyhow::{anyhow, Context}; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::catalog::ColumnCatalog; +use risingwave_common::types::DataType; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::{bail, bail_not_implemented}; +use risingwave_pb::catalog::{Source, Table}; +use risingwave_pb::ddl_service::TableJobType; +use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - AlterTableOperation, ColumnOption, ConnectorSchema, Encode, ObjectName, Statement, + AlterTableOperation, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Encode, + ObjectName, Statement, StructField, }; use risingwave_sqlparser::parser::Parser; @@ -30,7 +36,7 @@ use super::util::SourceSchemaCompatExt; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; -use crate::error::{ErrorCode, Result}; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::ExprImpl; use crate::session::SessionImpl; use crate::{Binder, TableCatalog, WithOptions}; @@ -42,6 +48,112 @@ pub async fn replace_table_with_definition( original_catalog: &Arc, source_schema: Option, ) -> Result<()> { + let (source, table, graph, col_index_mapping, job_type) = get_replace_table_plan( + session, + table_name, + definition, + original_catalog, + source_schema, + ) + .await?; + + let catalog_writer = session.catalog_writer()?; + + catalog_writer + .replace_table(source, table, graph, col_index_mapping, job_type) + .await?; + Ok(()) +} + +pub async fn get_new_table_definition_for_cdc_table( + session: &Arc, + table_name: ObjectName, + new_columns: Vec, +) -> Result<(Statement, Arc)> { + let original_catalog = fetch_table_catalog_for_alter(session.as_ref(), &table_name)?; + + // Retrieve the original table definition and parse it to AST. + let [mut definition]: [_; 1] = Parser::parse_sql(&original_catalog.definition) + .context("unable to parse original table definition")? + .try_into() + .unwrap(); + let Statement::CreateTable { + columns: original_columns, + source_schema, + .. + } = &mut definition + else { + panic!("unexpected statement: {:?}", definition); + }; + + assert!( + source_schema.is_none(), + "source schema should be None for CDC table" + ); + if original_columns.is_empty() { + Err(ErrorCode::NotSupported( + "alter a table with empty column definitions".to_string(), + "Please recreate the table with column definitions.".to_string(), + ))? + } + + // since the DDL is committed on upstream, so we can safely replace the original columns with new columns + // replace original columns with new columns + let mut new_column_defs = vec![]; + for col in new_columns.into_iter() { + let ty = to_ast_data_type(col.data_type())?; + new_column_defs.push(ColumnDef::new(col.name().into(), ty, None, vec![])); + } + *original_columns = new_column_defs; + + Ok((definition, original_catalog)) +} + +fn to_ast_data_type(ty: &DataType) -> Result { + match ty { + DataType::Boolean => Ok(AstDataType::Boolean), + DataType::Int16 => Ok(AstDataType::SmallInt), + DataType::Int32 => Ok(AstDataType::Int), + DataType::Int64 => Ok(AstDataType::BigInt), + DataType::Float32 => Ok(AstDataType::Real), + DataType::Float64 => Ok(AstDataType::Double), + DataType::Date => Ok(AstDataType::Date), + DataType::Time => Ok(AstDataType::Time(false)), + DataType::Timestamp => Ok(AstDataType::Timestamp(false)), + DataType::Timestamptz => Ok(AstDataType::Timestamp(true)), + DataType::Interval => Ok(AstDataType::Interval), + DataType::Jsonb => Ok(AstDataType::Jsonb), + DataType::Bytea => Ok(AstDataType::Bytea), + DataType::List(item_ty) => Ok(AstDataType::Array(Box::new(to_ast_data_type(item_ty)?))), + DataType::Struct(fields) => { + let fields = fields + .iter() + .map(|(name, ty)| { + Ok::(StructField { + name: name.into(), + data_type: to_ast_data_type(ty)?, + }) + }) + .try_collect()?; + Ok(AstDataType::Struct(fields)) + } + _ => Err(anyhow!("unsupported data type: {:?}", ty).context("to_ast_data_type"))?, + } +} + +pub async fn get_replace_table_plan( + session: &Arc, + table_name: ObjectName, + definition: Statement, + original_catalog: &Arc, + source_schema: Option, +) -> Result<( + Option, + Table, + StreamFragmentGraph, + ColIndexMapping, + TableJobType, +)> { // Create handler args as if we're creating a new table with the altered definition. let handler_args = HandlerArgs::new(session.clone(), &definition, Arc::from(""))?; let col_id_gen = ColumnIdGenerator::new_alter(original_catalog); @@ -92,12 +204,7 @@ pub async fn replace_table_with_definition( table.columns.len(), ); - let catalog_writer = session.catalog_writer()?; - - catalog_writer - .replace_table(source, table, graph, col_index_mapping, job_type) - .await?; - Ok(()) + Ok((source, table, graph, col_index_mapping, job_type)) } /// Handle `ALTER TABLE [ADD|DROP] COLUMN` statements. The `operation` must be either `AddColumn` or diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index f8beeedb19438..73fb4f08ff09a 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -100,6 +100,8 @@ pub mod util; pub mod variable; mod wait; +pub use alter_table_column::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; + /// The [`PgResponseBuilder`] used by RisingWave. pub type RwPgResponseBuilder = PgResponseBuilder; diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index f312f9a31c61d..af5be7b3df118 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -12,14 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. -use pgwire::pg_server::SessionManager; +use futures::TryStreamExt; +use pgwire::pg_server::{BoxedError, SessionManager}; +use risingwave_pb::ddl_service::SchemaChangeEnvelope; use risingwave_pb::frontend_service::schema_change_request::Request; use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use risingwave_rpc_client::error::ToTonicStatus; +use risingwave_sqlparser::ast::ObjectName; use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; +use crate::error::RwError; +use crate::handler::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; use crate::session::SESSION_MANAGER; +#[derive(thiserror::Error, Debug)] +pub enum AutoSchemaChangeError { + #[error("frontend error")] + FrontendError( + #[from] + #[backtrace] + RwError, + ), +} + +impl From for AutoSchemaChangeError { + fn from(err: BoxedError) -> Self { + AutoSchemaChangeError::FrontendError(RwError::from(err)) + } +} + +impl From for tonic::Status { + fn from(err: AutoSchemaChangeError) -> Self { + err.to_status(tonic::Code::Internal, "frontend") + } +} + #[derive(Default)] pub struct SchemaChangeServiceImpl {} @@ -38,20 +66,41 @@ impl SchemaChangeService for SchemaChangeServiceImpl { let req = request.into_inner(); match req.request.unwrap() { - Request::ReplaceTablePlan(req) => { + Request::GetNewTablePlan(req) => { let change = req.schema_change.expect("schema_change"); - // get a session object - let session_mgr = SESSION_MANAGER - .get() - .expect("session manager has been initialized"); - let _session = session_mgr - .get_session(req.database_id, &req.owner) - .map_err(|e| Status::internal(format!("Failed to get session: {}", e)))?; - - // call the handle alter method + get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; } }; Ok(RpcResponse::new(SchemaChangeResponse { response: None })) } } + +async fn get_new_table_plan( + change: SchemaChangeEnvelope, + table_name: String, + database_id: u32, + owner: String, +) -> Result<(), AutoSchemaChangeError> { + // get a session object + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + let session = session_mgr.get_session(database_id, &owner)?; + + // call the handle alter method + let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); + let table_name = ObjectName::from(vec![table_name.as_str().into()]); + let (new_table_definition, original_catalog) = + get_new_table_definition_for_cdc_table(&session, table_name.clone(), new_columns).await?; + let (_, table, graph, col_index_mapping, job_type) = get_replace_table_plan( + &session, + table_name, + new_table_definition, + &original_catalog, + None, + ) + .await?; + + Ok(()) +} diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index 5626912c2f88b..e90e2b9c60873 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -69,4 +69,4 @@ macro_rules! impl_from_status { }; } -impl_from_status!(stream, batch, meta, compute, compactor, connector); +impl_from_status!(stream, batch, meta, compute, compactor, connector, frontend); From 58e181792cf692fe3a474cb0b0de700c8c4866ce Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 17:45:12 +0800 Subject: [PATCH 08/75] minor --- src/frontend/src/rpc/mod.rs | 37 +++++++++++++++++++++++++------------ 1 file changed, 25 insertions(+), 12 deletions(-) diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index af5be7b3df118..c7c4162ac1591 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::TryStreamExt; use pgwire::pg_server::{BoxedError, SessionManager}; -use risingwave_pb::ddl_service::SchemaChangeEnvelope; +use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; use risingwave_pb::frontend_service::schema_change_request::Request; +use risingwave_pb::frontend_service::schema_change_response::{GetNewTablePlanResponse, Response}; use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; use risingwave_rpc_client::error::ToTonicStatus; @@ -65,14 +65,20 @@ impl SchemaChangeService for SchemaChangeServiceImpl { ) -> Result, Status> { let req = request.into_inner(); - match req.request.unwrap() { - Request::GetNewTablePlan(req) => { - let change = req.schema_change.expect("schema_change"); + if let Some(Request::GetNewTablePlan(req)) = req.request { + let change = req + .schema_change + .expect("schema change message is required"); + let replace_plan = get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; - } - }; - - Ok(RpcResponse::new(SchemaChangeResponse { response: None })) + Ok(RpcResponse::new(SchemaChangeResponse { + response: Some(Response::ReplaceTablePlan(GetNewTablePlanResponse { + table_plan: Some(replace_plan), + })), + })) + } else { + Err(Status::invalid_argument("invalid schema change request")) + } } } @@ -81,11 +87,12 @@ async fn get_new_table_plan( table_name: String, database_id: u32, owner: String, -) -> Result<(), AutoSchemaChangeError> { - // get a session object +) -> Result { let session_mgr = SESSION_MANAGER .get() .expect("session manager has been initialized"); + + // get a session object for the corresponding user and database let session = session_mgr.get_session(database_id, &owner)?; // call the handle alter method @@ -102,5 +109,11 @@ async fn get_new_table_plan( ) .await?; - Ok(()) + Ok(ReplaceTablePlan { + table: Some(table), + fragment_graph: Some(graph), + table_col_index_mapping: Some(col_index_mapping.to_protobuf()), + source: None, + job_type: job_type as _, + }) } From 35f9b4f7b60c3fe95f90c5af807cf7e104446008 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 18:11:40 +0800 Subject: [PATCH 09/75] launch rpc service --- src/frontend/src/session.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index df3cd6cc6e94e..5bae42df1c532 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -67,6 +67,7 @@ use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_connector::source::monitor::{SourceMetrics, GLOBAL_SOURCE_METRICS}; use risingwave_pb::common::WorkerType; +use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeServiceServer; use risingwave_pb::health::health_server::HealthServer; use risingwave_pb::user::auth_info::EncryptionType; use risingwave_pb::user::grant_privilege::Object; @@ -105,6 +106,7 @@ use crate::health_service::HealthServiceImpl; use crate::meta_client::{FrontendMetaClient, FrontendMetaClientImpl}; use crate::monitor::{FrontendMetrics, GLOBAL_FRONTEND_METRICS}; use crate::observer::FrontendObserverNode; +use crate::rpc::SchemaChangeServiceImpl; use crate::scheduler::streaming_manager::{StreamingJobTracker, StreamingJobTrackerRef}; use crate::scheduler::{ DistributedQueryMetrics, HummockSnapshotManager, HummockSnapshotManagerRef, QueryManager, @@ -357,6 +359,7 @@ impl FrontendEnv { } let health_srv = HealthServiceImpl::new(); + let schema_change_srv = SchemaChangeServiceImpl::new(); let host = opts.health_check_listener_addr.clone(); let telemetry_manager = TelemetryManager::new( @@ -377,13 +380,13 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) - // .add_service() + .add_service(SchemaChangeServiceServer::new(schema_change_srv)) .serve(host.parse().unwrap()) .await .unwrap(); }); info!( - "Health Check RPC Listener is set up on {}", + "Frontend RPC Listener is set up on {}", opts.health_check_listener_addr.clone() ); From 532c31eb9c13597bd6b1fda309ffe3cabcc476eb Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 11:11:48 +0800 Subject: [PATCH 10/75] refactor frontend service --- proto/frontend_service.proto | 32 +++++++--------------- src/frontend/src/rpc/mod.rs | 42 ++++++++++++----------------- src/frontend/src/session.rs | 8 +++--- src/meta/service/src/ddl_service.rs | 1 + 4 files changed, 31 insertions(+), 52 deletions(-) diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index ce5f49a49be27..d633734bd58b5 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -8,33 +8,19 @@ option java_package = "com.risingwave.proto"; option optimize_for = SPEED; -message SchemaChangeRequest { - message GetNewTablePlanRequest { - uint32 database_id = 1; - string owner = 2; - string table_name = 3; - ddl_service.SchemaChangeEnvelope schema_change = 4; - } - - oneof request { - GetNewTablePlanRequest get_new_table_plan = 1; - } - +message GetTableReplacePlanRequest { + uint32 database_id = 1; + string owner = 2; + string table_name = 3; + ddl_service.SchemaChangeEnvelope schema_change = 4; } -message SchemaChangeResponse { - message GetNewTablePlanResponse { - ddl_service.ReplaceTablePlan table_plan = 1; - } - - oneof response { - GetNewTablePlanResponse replace_table_plan = 1; - } +message GetTableReplacePlanResponse { + ddl_service.ReplaceTablePlan table_plan = 1; } - // Frontend will start a background worker to -service SchemaChangeService { - rpc GetNewTableStreamingGraph(SchemaChangeRequest) returns (SchemaChangeResponse); +service FrontendService { + rpc GetTableReplacePlan(GetTableReplacePlanRequest) returns (GetTableReplacePlanResponse); } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index c7c4162ac1591..acf37d7cab323 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -14,13 +14,11 @@ use pgwire::pg_server::{BoxedError, SessionManager}; use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; -use risingwave_pb::frontend_service::schema_change_request::Request; -use risingwave_pb::frontend_service::schema_change_response::{GetNewTablePlanResponse, Response}; -use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; -use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use risingwave_pb::frontend_service::frontend_service_server::FrontendService; +use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; use risingwave_rpc_client::error::ToTonicStatus; use risingwave_sqlparser::ast::ObjectName; -use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; +use tonic::{Request as RpcRequest, Response as RpcResponse, Response, Status}; use crate::error::RwError; use crate::handler::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; @@ -49,36 +47,30 @@ impl From for tonic::Status { } #[derive(Default)] -pub struct SchemaChangeServiceImpl {} +pub struct FrontendServiceImpl {} -impl SchemaChangeServiceImpl { +impl FrontendServiceImpl { pub fn new() -> Self { Self {} } } #[async_trait::async_trait] -impl SchemaChangeService for SchemaChangeServiceImpl { - async fn get_new_table_streaming_graph( +impl FrontendService for FrontendServiceImpl { + async fn get_table_replace_plan( &self, - request: RpcRequest, - ) -> Result, Status> { + request: RpcRequest, + ) -> Result, Status> { let req = request.into_inner(); - if let Some(Request::GetNewTablePlan(req)) = req.request { - let change = req - .schema_change - .expect("schema change message is required"); - let replace_plan = - get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; - Ok(RpcResponse::new(SchemaChangeResponse { - response: Some(Response::ReplaceTablePlan(GetNewTablePlanResponse { - table_plan: Some(replace_plan), - })), - })) - } else { - Err(Status::invalid_argument("invalid schema change request")) - } + let change = req + .schema_change + .expect("schema change message is required"); + let replace_plan = + get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; + Ok(RpcResponse::new(GetTableReplacePlanResponse { + table_plan: Some(replace_plan), + })) } } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 5bae42df1c532..6b03703653faa 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -67,7 +67,7 @@ use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_connector::source::monitor::{SourceMetrics, GLOBAL_SOURCE_METRICS}; use risingwave_pb::common::WorkerType; -use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeServiceServer; +use risingwave_pb::frontend_service::frontend_service_server::FrontendServiceServer; use risingwave_pb::health::health_server::HealthServer; use risingwave_pb::user::auth_info::EncryptionType; use risingwave_pb::user::grant_privilege::Object; @@ -106,7 +106,7 @@ use crate::health_service::HealthServiceImpl; use crate::meta_client::{FrontendMetaClient, FrontendMetaClientImpl}; use crate::monitor::{FrontendMetrics, GLOBAL_FRONTEND_METRICS}; use crate::observer::FrontendObserverNode; -use crate::rpc::SchemaChangeServiceImpl; +use crate::rpc::FrontendServiceImpl; use crate::scheduler::streaming_manager::{StreamingJobTracker, StreamingJobTrackerRef}; use crate::scheduler::{ DistributedQueryMetrics, HummockSnapshotManager, HummockSnapshotManagerRef, QueryManager, @@ -359,7 +359,7 @@ impl FrontendEnv { } let health_srv = HealthServiceImpl::new(); - let schema_change_srv = SchemaChangeServiceImpl::new(); + let frontend_srv = FrontendServiceImpl::new(); let host = opts.health_check_listener_addr.clone(); let telemetry_manager = TelemetryManager::new( @@ -380,7 +380,7 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) - .add_service(SchemaChangeServiceServer::new(schema_change_srv)) + .add_service(FrontendServiceServer::new(frontend_srv)) .serve(host.parse().unwrap()) .await .unwrap(); diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 560c6b369e946..6fdb622e3e096 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -49,6 +49,7 @@ pub struct DdlServiceImpl { sink_manager: SinkCoordinatorManager, ddl_controller: DdlController, aws_client: Arc>, + // frontend_client: } impl DdlServiceImpl { From f5071960180971fc0bb2fa18aaedbbc03fe3eaf4 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 18:07:51 +0800 Subject: [PATCH 11/75] add cdc table name to PbTable --- proto/catalog.proto | 2 ++ src/frontend/src/catalog/table_catalog.rs | 4 ++++ src/frontend/src/handler/create_table.rs | 4 +++- src/frontend/src/optimizer/mod.rs | 2 ++ src/frontend/src/optimizer/plan_node/stream_materialize.rs | 6 +++++- src/frontend/src/optimizer/plan_node/utils.rs | 1 + src/meta/model_v2/src/table.rs | 2 ++ src/meta/src/controller/mod.rs | 1 + 8 files changed, 20 insertions(+), 2 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index b18275e32d4ce..3c3471d2fffbe 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -406,6 +406,8 @@ message Table { // conflict" operations. optional uint32 version_column_index = 38; + optional string cdc_table_name = 39; + // Per-table catalog version, used by schema change. `None` for internal // tables and tests. Not to be confused with the global catalog version for // notification service. diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 29c74a64e2551..5d5e7e9060872 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -168,6 +168,8 @@ pub struct TableCatalog { pub created_at_cluster_version: Option, pub initialized_at_cluster_version: Option, + + pub cdc_table_name: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -425,6 +427,7 @@ impl TableCatalog { created_at_cluster_version: self.created_at_cluster_version.clone(), initialized_at_cluster_version: self.initialized_at_cluster_version.clone(), retention_seconds: self.retention_seconds, + cdc_table_name: self.cdc_table_name.clone(), } } @@ -580,6 +583,7 @@ impl From for TableCatalog { .into_iter() .map(TableId::from) .collect_vec(), + cdc_table_name: tb.cdc_table_name, } } } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5dce6028d0cc0..6d5fae241a02a 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -727,6 +727,7 @@ fn gen_table_plan_inner( version, is_external_source, retention_seconds, + None, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -810,7 +811,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_table( let options = CdcScanOptions::from_with_options(context.with_options())?; let logical_scan = LogicalCdcScan::create( - external_table_name, + external_table_name.clone(), Rc::new(cdc_table_desc), context.clone(), options, @@ -840,6 +841,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_table( Some(col_id_gen.into_version()), true, None, + Some(external_table_name), )?; let mut table = materialize.table().to_prost(schema_id, database_id); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 491f3b71875b1..2a4139e7c10fa 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -632,6 +632,7 @@ impl PlanRoot { version: Option, with_external_source: bool, retention_seconds: Option, + cdc_table_name: Option, ) -> Result { assert_eq!(self.phase, PlanPhase::Logical); assert_eq!(self.plan.convention(), Convention::Logical); @@ -861,6 +862,7 @@ impl PlanRoot { row_id_index, version, retention_seconds, + cdc_table_name, ) } diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index e5a2496916adc..657b838fae3fa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -132,10 +132,11 @@ impl StreamMaterialize { row_id_index: Option, version: Option, retention_seconds: Option, + cdc_table_name: Option, ) -> Result { let input = Self::rewrite_input(input, user_distributed_by, TableType::Table)?; - let table = Self::derive_table_catalog( + let mut table = Self::derive_table_catalog( input.clone(), name, user_order_by, @@ -152,6 +153,8 @@ impl StreamMaterialize { CreateType::Foreground, )?; + table.cdc_table_name = cdc_table_name; + Ok(Self::new(input, table)) } @@ -278,6 +281,7 @@ impl StreamMaterialize { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: retention_seconds.map(|i| i.into()), + cdc_table_name: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 155381ab4310e..ef2920abd6111 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -178,6 +178,7 @@ impl TableCatalogBuilder { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: None, + cdc_table_name: None, } } diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 75fb66fcb2a5e..5696a9a91caca 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -132,6 +132,7 @@ pub struct Model { pub version: Option, pub retention_seconds: Option, pub incoming_sinks: I32Array, + pub cdc_table_name: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -243,6 +244,7 @@ impl From for ActiveModel { version: Set(pb_table.version.as_ref().map(|v| v.into())), retention_seconds: Set(pb_table.retention_seconds.map(|i| i as _)), incoming_sinks: Set(pb_table.incoming_sinks.into()), + cdc_table_name: Set(pb_table.cdc_table_name), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 5ace222739d52..5dcc936e0778b 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -161,6 +161,7 @@ impl From> for PbTable { initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, retention_seconds: value.0.retention_seconds.map(|id| id as u32), + cdc_table_name: value.0.cdc_table_name, } } } From 9b8588737d74d557a0ac41b50243f587f769df80 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 19:41:38 +0800 Subject: [PATCH 12/75] wip: meta->frontend rpc implementation --- proto/ddl_service.proto | 2 +- proto/frontend_service.proto | 2 +- src/frontend/src/rpc/mod.rs | 4 +- src/frontend/src/session.rs | 13 +++- src/frontend/src/test_utils.rs | 2 +- src/meta/service/src/ddl_service.rs | 44 +++++++++++++- src/meta/src/controller/catalog.rs | 16 +++++ src/meta/src/manager/catalog/database.rs | 9 +++ src/meta/src/manager/catalog/mod.rs | 8 +++ src/meta/src/manager/env.rs | 14 ++++- src/meta/src/manager/metadata.rs | 16 +++++ src/rpc_client/src/frontend_client.rs | 76 ++++++++++++++++++++++++ src/rpc_client/src/lib.rs | 19 ++++++ src/utils/pgwire/src/pg_server.rs | 9 +-- 14 files changed, 217 insertions(+), 17 deletions(-) create mode 100644 src/rpc_client/src/frontend_client.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 134c50310aefe..45e5ccd474754 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -436,7 +436,7 @@ message CommentOnResponse { } message SchemaChangeEnvelope { - string up_table_name = 1; + string cdc_table_name = 1; repeated plan_common.ColumnCatalog column_descs = 2; } diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index d633734bd58b5..247c580cb392e 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -10,7 +10,7 @@ option optimize_for = SPEED; message GetTableReplacePlanRequest { uint32 database_id = 1; - string owner = 2; + uint32 owner = 2; string table_name = 3; ddl_service.SchemaChangeEnvelope schema_change = 4; } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index acf37d7cab323..8651a19165db6 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -78,14 +78,14 @@ async fn get_new_table_plan( change: SchemaChangeEnvelope, table_name: String, database_id: u32, - owner: String, + owner: u32, ) -> Result { let session_mgr = SESSION_MANAGER .get() .expect("session manager has been initialized"); // get a session object for the corresponding user and database - let session = session_mgr.get_session(database_id, &owner)?; + let session = session_mgr.get_session(database_id, owner)?; // call the handle alter method let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 6b03703653faa..9f97ff7a67685 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1141,13 +1141,22 @@ impl SessionManager for SessionManagerImpl { fn get_session( &self, database_id: u32, - user_name: &str, + user_id: u32, ) -> std::result::Result, BoxedError> { let dumb_addr = Address::Tcp(SocketAddr::new( IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691, // port of meta )); - self.connect_inner(database_id, user_name, Arc::new(dumb_addr)) + let user_reader = self.env.user_info_reader(); + let reader = user_reader.read_guard(); + if let Some(user_name) = reader.get_user_name_by_id(user_id) { + self.connect_inner(database_id, user_name.as_str(), Arc::new(dumb_addr)) + } else { + Err(Box::new(Error::new( + ErrorKind::InvalidInput, + format!("Role id {} does not exist", user_id), + ))) + } } fn connect( diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 6c6d9958210d6..0f4a656e7dc9b 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -88,7 +88,7 @@ impl SessionManager for LocalFrontend { fn get_session( &self, database_id: u32, - user_name: &str, + user_name: u32, ) -> std::result::Result, BoxedError> { todo!() } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 6fdb622e3e096..59294e2ffdb56 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -16,6 +16,8 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::anyhow; +use rand::seq::SliceRandom; +use rand::thread_rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_connector::sink::catalog::SinkId; use risingwave_meta::manager::MetadataManager; @@ -25,10 +27,13 @@ use risingwave_pb::catalog::connection::private_link_service::{ }; use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, Secret}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, Secret, Table}; +use risingwave_pb::common::worker_node::State; +use risingwave_pb::common::WorkerType; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; +use risingwave_pb::frontend_service::GetTableReplacePlanRequest; use tonic::{Request, Response, Status}; use crate::barrier::BarrierManagerRef; @@ -49,7 +54,6 @@ pub struct DdlServiceImpl { sink_manager: SinkCoordinatorManager, ddl_controller: DdlController, aws_client: Arc>, - // frontend_client: } impl DdlServiceImpl { @@ -925,10 +929,44 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); let schema_change = req.schema_change.unwrap(); + let cdc_table_name = schema_change.cdc_table_name.clone(); - // TODO: can we build a ReplaceTablePlan using the info in schema_change? + // get the table catalog corresponding to the + let tables: Vec = self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) + .await?; // send a request to the frontend to get the ReplaceTablePlan + let mut workers = self + .metadata_manager + .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) + .await?; + workers.shuffle(&mut thread_rng()); + let worker = workers + .first() + .ok_or_else(|| anyhow!("no frontend worker available"))?; + let client = self.env.frontend_client_pool().get(worker).await?; + + for table in tables { + let resp = client + .get_table_replace_plan(GetTableReplacePlanRequest { + database_id: table.database_id, + owner: table.owner, + table_name: table.name, + schema_change: Some(schema_change), + }) + .await?; + + if let Some(plan) = resp.table_plan { + // start the schema change procedure + self.ddl_controller + .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( + plan, + ))) + .await?; + } + } Ok(Response::new(AutoSchemaChangeResponse {})) } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index cb5dc2bf41b7e..c818156a80f2a 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2838,6 +2838,22 @@ impl CatalogController { .collect()) } + pub async fn get_table_by_cdc_table_name( + &self, + cdc_table_name: String, + ) -> MetaResult> { + let inner = self.inner.read().await; + let table_objs = Table::find() + .find_also_related(Object) + .filter(table::Column::CdcTableName.eq(cdc_table_name)) + .all(&inner.db) + .await?; + Ok(table_objs + .into_iter() + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + pub async fn get_created_table_ids(&self) -> MetaResult> { let inner = self.inner.read().await; diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index e3b2e0d02cae7..2c977981c4676 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -248,6 +248,15 @@ impl DatabaseManager { .collect() } + pub fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ + let cdc_table_name = Some(cdc_table_name); + self.tables + .values() + .filter(|t| t.cdc_table_name == cdc_table_name) + .cloned() + .collect() + } + pub fn check_relation_name_duplicated(&self, relation_key: &RelationKey) -> MetaResult<()> { if let Some(t) = self.tables.values().find(|x| { x.database_id == relation_key.0 diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 02cc9ee8de0bf..c6ce0334dcfa9 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -4063,6 +4063,14 @@ impl CatalogManager { .get_table_name_and_type_mapping() } + pub async fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ + self.core + .lock() + .await + .database + .get_table_by_cdc_table_name(cdc_table_name) + } + /// `list_stream_job_ids` returns all running and creating stream job ids, this is for recovery /// clean up progress. pub async fn list_stream_job_ids(&self) -> MetaResult> { diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 3c6a18adef747..93c0d33936e80 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -23,7 +23,9 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; -use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; +use risingwave_rpc_client::{ + FrontendClientPool, FrontendClientPoolRef, StreamClientPool, StreamClientPoolRef, +}; use sea_orm::EntityTrait; use super::{ @@ -123,6 +125,9 @@ pub struct MetaSrvEnv { /// stream client pool memorization. stream_client_pool: StreamClientPoolRef, + /// rpc client pool for frontend nodes. + frontend_client_pool: FrontendClientPoolRef, + /// idle status manager. idle_manager: IdleManagerRef, @@ -389,6 +394,7 @@ impl MetaSrvEnv { ) -> MetaResult { let idle_manager = Arc::new(IdleManager::new(opts.max_idle_ms)); let stream_client_pool = Arc::new(StreamClientPool::default()); + let frontend_client_pool = Arc::new(FrontendClientPool::default()); let event_log_manager = Arc::new(start_event_log_manager( opts.event_log_enabled, opts.event_log_channel_max_size, @@ -444,6 +450,7 @@ impl MetaSrvEnv { meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, + frontend_client_pool, idle_manager, event_log_manager, cluster_id, @@ -499,6 +506,7 @@ impl MetaSrvEnv { meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, + frontend_client_pool, idle_manager, event_log_manager, cluster_id, @@ -563,6 +571,10 @@ impl MetaSrvEnv { self.stream_client_pool.deref() } + pub fn frontend_client_pool(&self) -> &FrontendClientPool { + self.frontend_client_pool.deref() + } + pub fn cluster_id(&self) -> &ClusterId { &self.cluster_id } diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index ecd9d4971d2b8..71f4174e0d837 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -538,6 +538,22 @@ impl MetadataManager { } } + pub async fn get_table_catalog_by_cdc_table_name( + &self, + name: String, + ) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => { + Ok(mgr.catalog_manager.get_table_by_cdc_table_name(name).await) + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_table_by_cdc_table_name(name) + .await + } + } + } + pub async fn get_downstream_chain_fragments( &self, job_id: u32, diff --git a/src/rpc_client/src/frontend_client.rs b/src/rpc_client/src/frontend_client.rs new file mode 100644 index 0000000000000..088cf74e33e76 --- /dev/null +++ b/src/rpc_client/src/frontend_client.rs @@ -0,0 +1,76 @@ +// 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::sync::Arc; +use std::time::Duration; + +use anyhow::anyhow; +use async_trait::async_trait; +use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; +use risingwave_common::monitor::{EndpointExt, TcpConfig}; +use risingwave_common::util::addr::HostAddr; +use risingwave_pb::frontend_service::frontend_service_client::FrontendServiceClient; +use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; +use tonic::transport::Endpoint; + +use crate::error::{Result, RpcError}; +use crate::tracing::{Channel, TracingInjectedChannelExt}; +use crate::{frontend_rpc_client_method_impl, RpcClient, RpcClientPool}; + +#[derive(Clone)] +pub struct FrontendClient(FrontendServiceClient); + +#[async_trait] +impl RpcClient for FrontendClient { + async fn new_client(host_addr: HostAddr) -> Result { + Self::new(host_addr).await + } +} + +impl FrontendClient { + async fn new(host_addr: HostAddr) -> Result { + let channel = Endpoint::from_shared(format!("http://{}", &host_addr))? + .initial_connection_window_size(MAX_CONNECTION_WINDOW_SIZE) + .connect_timeout(Duration::from_secs(5)) + .monitored_connect( + "grpc-frontend-client", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + ) + .await? + .tracing_injected(); + + Ok(Self( + FrontendServiceClient::new(channel).max_decoding_message_size(usize::MAX), + )) + } +} + +// similar to the stream_client used in the Meta node +pub type FrontendClientPool = RpcClientPool; +pub type FrontendClientPoolRef = Arc; + +macro_rules! for_all_frontend_rpc { + ($macro:ident) => { + $macro! { + { 0, get_table_replace_plan, GetTableReplacePlanRequest, GetTableReplacePlanResponse } + } + }; +} + +impl FrontendClient { + for_all_frontend_rpc! { frontend_rpc_client_method_impl } +} diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index fa276bdd0a5ce..eeaf658ce19ba 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -53,6 +53,7 @@ use error::Result; mod compactor_client; mod compute_client; mod connector_client; +mod frontend_client; mod hummock_meta_client; mod meta_client; mod sink_coordinate_client; @@ -62,6 +63,7 @@ mod tracing; pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; +pub use frontend_client::{FrontendClient, FrontendClientPool, FrontendClientPoolRef}; pub use hummock_meta_client::{CompactionEventItem, HummockMetaClient}; pub use meta_client::{MetaClient, SinkCoordinationRpcClient}; use rw_futures_util::await_future_with_monitor_error_stream; @@ -180,6 +182,23 @@ macro_rules! meta_rpc_client_method_impl { } } +#[macro_export] +macro_rules! frontend_rpc_client_method_impl { + ($( { $client:tt, $fn_name:ident, $req:ty, $resp:ty }),*) => { + $( + pub async fn $fn_name(&self, request: $req) -> $crate::Result<$resp> { + Ok(self + .$client + .to_owned() + .$fn_name(request) + .await + .map_err($crate::error::RpcError::from_frontend_status)? + .into_inner()) + } + )* + } +} + pub const DEFAULT_BUFFER_SIZE: usize = 16; pub struct BidiStreamSender { diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index e785f5409f0dd..b7fea536f9053 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -47,11 +47,8 @@ pub type SessionId = (ProcessId, SecretKey); pub trait SessionManager: Send + Sync + 'static { type Session: Session; - fn get_session( - &self, - database_id: u32, - user_name: &str, - ) -> Result, BoxedError>; + fn get_session(&self, database_id: u32, user_id: u32) + -> Result, BoxedError>; fn connect( &self, @@ -389,7 +386,7 @@ mod tests { fn get_session( &self, database_id: u32, - user_name: &str, + user_name: u32, ) -> Result, BoxedError> { todo!() } From 33857bb3aaff4a152f43467df44589765f7c051a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 14:03:55 +0800 Subject: [PATCH 13/75] wip: rpc source parser->meta --- proto/ddl_service.proto | 6 +++- .../src/parser/debezium/schema_change.rs | 14 +++++++-- src/connector/src/parser/unified/debezium.rs | 2 +- .../src/schema/auto_schema_change/client.rs | 31 +++++++++++++++++++ .../src/schema/auto_schema_change/mod.rs | 12 +------ src/meta/service/src/ddl_service.rs | 16 +++++----- src/rpc_client/src/meta_client.rs | 9 ++++++ 7 files changed, 67 insertions(+), 23 deletions(-) create mode 100644 src/connector/src/schema/auto_schema_change/client.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 45e5ccd474754..51a15ba7916de 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -435,11 +435,15 @@ message CommentOnResponse { uint64 version = 2; } -message SchemaChangeEnvelope { +message TableSchemaChange { string cdc_table_name = 1; repeated plan_common.ColumnCatalog column_descs = 2; } +message SchemaChangeEnvelope { + repeated TableSchemaChange table_changes = 1; +} + message AutoSchemaChangeRequest { SchemaChangeEnvelope schema_change = 1; } diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 958748d04bb31..66258c4a41ef5 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -14,6 +14,8 @@ use risingwave_common::catalog::ColumnCatalog; +use risingwave_pb::ddl_service::SchemaChangeEnvelope as PbSchemaChangeEnvelope; + #[derive(Debug)] pub struct SchemaChangeEnvelope { pub table_changes: Vec, @@ -21,12 +23,18 @@ pub struct SchemaChangeEnvelope { #[derive(Debug)] pub struct TableSchemaChange { - pub(crate) up_table_full_name: String, + pub(crate) cdc_table_name: String, pub(crate) columns: Vec, } impl SchemaChangeEnvelope { - pub fn to_protobuf(&self) { - todo!() + pub fn to_protobuf(&self) -> Vec { + + PbSchemaChangeEnvelope { + cdc_table_name: self.c + column_descs: vec![], + } + + } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index af74a80fc13ab..9ce8ae993fa24 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -198,7 +198,7 @@ pub fn parse_schema_change( } } schema_changes.push(TableSchemaChange { - up_table_full_name: id, + cdc_table_name: id, columns: column_descs .into_iter() .map(|column_desc| ColumnCatalog { diff --git a/src/connector/src/schema/auto_schema_change/client.rs b/src/connector/src/schema/auto_schema_change/client.rs new file mode 100644 index 0000000000000..ac7afe0bd53e5 --- /dev/null +++ b/src/connector/src/schema/auto_schema_change/client.rs @@ -0,0 +1,31 @@ +// 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 risingwave_rpc_client::MetaClient; + +use crate::parser::schema_change::SchemaChangeEnvelope; + +/// client for auto schema change +/// we may collect some metrics here +pub struct AutoSchemaChangeClient { + meta_client: MetaClient, +} + +impl AutoSchemaChangeClient { + pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { + // TODO: + + Ok(()) + } +} diff --git a/src/connector/src/schema/auto_schema_change/mod.rs b/src/connector/src/schema/auto_schema_change/mod.rs index 44b9c88f9e347..f2b2f45ae6a2e 100644 --- a/src/connector/src/schema/auto_schema_change/mod.rs +++ b/src/connector/src/schema/auto_schema_change/mod.rs @@ -12,14 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::parser::schema_change::SchemaChangeEnvelope; - -/// client for auto schema change -/// Can be a global client -pub struct AutoSchemaChangeClient {} - -impl AutoSchemaChangeClient { - pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { - Ok(()) - } -} +mod client; diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 59294e2ffdb56..da8ba26f5d613 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -922,6 +922,8 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(AlterParallelismResponse {})) } + /// Auto schema change for cdc sources, + /// called by the source parser when a schema change is detected. async fn auto_schema_change( &self, request: Request, @@ -931,13 +933,6 @@ impl DdlService for DdlServiceImpl { let schema_change = req.schema_change.unwrap(); let cdc_table_name = schema_change.cdc_table_name.clone(); - // get the table catalog corresponding to the - let tables: Vec
= self - .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) - .await?; - - // send a request to the frontend to get the ReplaceTablePlan let mut workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) @@ -948,7 +943,14 @@ impl DdlService for DdlServiceImpl { .ok_or_else(|| anyhow!("no frontend worker available"))?; let client = self.env.frontend_client_pool().get(worker).await?; + // get the table catalog corresponding to the + let tables: Vec
= self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) + .await?; + for table in tables { + // send a request to the frontend to get the ReplaceTablePlan let resp = client .get_table_replace_plan(GetTableReplacePlanRequest { database_id: table.database_id, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index db24711ba2377..35ef5f44475b9 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -549,6 +549,14 @@ impl MetaClient { Ok(resp.version) } + pub async fn auto_schema_change(&self, schema_change: SchemaChangeEnvelope) -> Result<()> { + let request = AutoSchemaChangeRequest { + schema_change: Some(schema_change), + }; + let _ = self.inner.auto_schema_change(request).await?; + Ok(()) + } + pub async fn create_view(&self, view: PbView) -> Result { let request = CreateViewRequest { view: Some(view) }; let resp = self.inner.create_view(request).await?; @@ -2019,6 +2027,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, comment_on, CommentOnRequest, CommentOnResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } ,{ ddl_client, wait, WaitRequest, WaitResponse } + ,{ ddl_client, auto_schema_change, AutoSchemaChangeRequest, AutoSchemaChangeResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } From 850f1d05ca61f530fb3d9b6de52556fb963d5b31 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 15:47:28 +0800 Subject: [PATCH 14/75] refine source -> meta -> frontend rpc --- proto/ddl_service.proto | 2 +- proto/frontend_service.proto | 5 +- .../src/parser/debezium/schema_change.rs | 31 ++++++--- src/frontend/src/rpc/mod.rs | 16 ++--- src/meta/service/src/ddl_service.rs | 66 +++++++++++-------- .../src/delete_range_runner.rs | 1 + 6 files changed, 72 insertions(+), 49 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 51a15ba7916de..3b2e2a9bd2b9f 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -437,7 +437,7 @@ message CommentOnResponse { message TableSchemaChange { string cdc_table_name = 1; - repeated plan_common.ColumnCatalog column_descs = 2; + repeated plan_common.ColumnCatalog columns = 2; } message SchemaChangeEnvelope { diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 247c580cb392e..8fb49e1bedb3a 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -12,12 +12,11 @@ message GetTableReplacePlanRequest { uint32 database_id = 1; uint32 owner = 2; string table_name = 3; - ddl_service.SchemaChangeEnvelope schema_change = 4; + ddl_service.TableSchemaChange table_change = 4; } - message GetTableReplacePlanResponse { - ddl_service.ReplaceTablePlan table_plan = 1; + ddl_service.ReplaceTablePlan replace_plan = 1; } // Frontend will start a background worker to diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 66258c4a41ef5..07efd099462b3 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -13,8 +13,9 @@ // limitations under the License. use risingwave_common::catalog::ColumnCatalog; - -use risingwave_pb::ddl_service::SchemaChangeEnvelope as PbSchemaChangeEnvelope; +use risingwave_pb::ddl_service::{ + SchemaChangeEnvelope as PbSchemaChangeEnvelope, TableSchemaChange as PbTableSchemaChange, +}; #[derive(Debug)] pub struct SchemaChangeEnvelope { @@ -28,13 +29,23 @@ pub struct TableSchemaChange { } impl SchemaChangeEnvelope { - pub fn to_protobuf(&self) -> Vec { - - PbSchemaChangeEnvelope { - cdc_table_name: self.c - column_descs: vec![], - } - - + pub fn to_protobuf(&self) -> PbSchemaChangeEnvelope { + let table_changes = self + .table_changes + .iter() + .map(|table_change| { + let columns = table_change + .columns + .iter() + .map(|column| column.to_protobuf()) + .collect(); + PbTableSchemaChange { + cdc_table_name: table_change.cdc_table_name.clone(), + columns, + } + }) + .collect(); + + PbSchemaChangeEnvelope { table_changes } } } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index 8651a19165db6..784c46a479b9d 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. use pgwire::pg_server::{BoxedError, SessionManager}; -use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; +use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope, TableSchemaChange}; use risingwave_pb::frontend_service::frontend_service_server::FrontendService; use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; use risingwave_rpc_client::error::ToTonicStatus; @@ -62,20 +62,18 @@ impl FrontendService for FrontendServiceImpl { request: RpcRequest, ) -> Result, Status> { let req = request.into_inner(); - - let change = req - .schema_change - .expect("schema change message is required"); + let table_change = req.table_change.expect("schema change message is required"); let replace_plan = - get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; + get_new_table_plan(table_change, req.table_name, req.database_id, req.owner).await?; + Ok(RpcResponse::new(GetTableReplacePlanResponse { - table_plan: Some(replace_plan), + replace_plan: Some(replace_plan), })) } } async fn get_new_table_plan( - change: SchemaChangeEnvelope, + table_change: TableSchemaChange, table_name: String, database_id: u32, owner: u32, @@ -88,7 +86,7 @@ async fn get_new_table_plan( let session = session_mgr.get_session(database_id, owner)?; // call the handle alter method - let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); + let new_columns = table_change.columns.into_iter().map(|c| c.into()).collect(); let table_name = ObjectName::from(vec![table_name.as_str().into()]); let (new_table_definition, original_catalog) = get_new_table_definition_for_cdc_table(&session, table_name.clone(), new_columns).await?; diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index da8ba26f5d613..69b443a8e73d7 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -20,6 +20,7 @@ use rand::seq::SliceRandom; use rand::thread_rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_connector::sink::catalog::SinkId; +use risingwave_meta::error::MetaErrorInner; use risingwave_meta::manager::MetadataManager; use risingwave_meta::rpc::ddl_controller::fill_table_stream_graph_info; use risingwave_pb::catalog::connection::private_link_service::{ @@ -930,9 +931,6 @@ impl DdlService for DdlServiceImpl { ) -> Result, Status> { let req = request.into_inner(); - let schema_change = req.schema_change.unwrap(); - let cdc_table_name = schema_change.cdc_table_name.clone(); - let mut workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) @@ -940,33 +938,49 @@ impl DdlService for DdlServiceImpl { workers.shuffle(&mut thread_rng()); let worker = workers .first() - .ok_or_else(|| anyhow!("no frontend worker available"))?; - let client = self.env.frontend_client_pool().get(worker).await?; + .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; + let client = self + .env + .frontend_client_pool() + .get(worker) + .await + .map_err(|err| MetaError::from(err))?; + + let Some(schema_change) = req.schema_change else { + return Err(Status::invalid_argument( + "schema change message is required", + )); + }; - // get the table catalog corresponding to the - let tables: Vec
= self - .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) - .await?; + for table_change in schema_change.table_changes { + let cdc_table_name = table_change.cdc_table_name.clone(); - for table in tables { - // send a request to the frontend to get the ReplaceTablePlan - let resp = client - .get_table_replace_plan(GetTableReplacePlanRequest { - database_id: table.database_id, - owner: table.owner, - table_name: table.name, - schema_change: Some(schema_change), - }) + // get the table catalog corresponding to the cdc table + let tables: Vec
= self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) .await?; - if let Some(plan) = resp.table_plan { - // start the schema change procedure - self.ddl_controller - .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( - plan, - ))) - .await?; + for table in tables { + // send a request to the frontend to get the ReplaceTablePlan + let resp = client + .get_table_replace_plan(GetTableReplacePlanRequest { + database_id: table.database_id, + owner: table.owner, + table_name: table.name, + table_change: Some(table_change.clone()), + }) + .await + .map_err(|err| MetaError::from(err))?; + + if let Some(plan) = resp.replace_plan { + // start the schema change procedure + self.ddl_controller + .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( + plan, + ))) + .await?; + } } } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 5d6c6ff7e70d1..9f46e6be5d6fe 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -158,6 +158,7 @@ async fn compaction_test( incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, + cdc_table_name: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; From 6ffeb3b96e0887e6447b1e5b94b058102dc48568 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 19:26:03 +0800 Subject: [PATCH 15/75] submit schema change to spawned task in source exec --- src/batch/src/executor/source.rs | 1 + src/connector/src/parser/mod.rs | 5 ++++ src/connector/src/source/base.rs | 10 +++++++ src/connector/src/source/mod.rs | 1 + src/stream/src/executor/actor.rs | 7 +++++ .../src/executor/source/fetch_executor.rs | 1 + .../src/executor/source/fs_source_executor.rs | 1 + .../source/source_backfill_executor.rs | 1 + .../src/executor/source/source_executor.rs | 30 ++++++++++++++++++- src/stream/src/task/stream_manager.rs | 1 + 10 files changed, 57 insertions(+), 1 deletion(-) diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 51d8da9d14d9c..655030e3d510d 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -175,6 +175,7 @@ impl SourceExecutor { rate_limit: None, }, ConnectorProperties::default(), + None, )); let stream = self .source diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 1f263ac5adde4..38eaef6fbd757 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -841,6 +841,11 @@ async fn into_chunk_stream_inner( // 1. block source executor // 2. send schema change to Meta // 3. wait for Meta to finish schema change + let (oneshot_tx, oneshot_rx) = tokio::sync::oneshot::channel(); + if let Some(ref tx) = parser.source_ctx().schema_change_tx { + tx.send((schema_change, oneshot_tx)).await.unwrap(); + oneshot_rx.await.unwrap(); + } } } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 4c44f9610bd17..598d59c1f9436 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -32,6 +32,7 @@ use risingwave_pb::catalog::{PbSource, PbStreamSourceInfo}; use risingwave_pb::plan_common::ExternalTableDesc; use risingwave_pb::source::ConnectorSplit; use serde::de::DeserializeOwned; +use tokio::sync::mpsc; use super::cdc::DebeziumCdcMeta; use super::datagen::DatagenMeta; @@ -42,6 +43,7 @@ use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; use crate::error::ConnectorResult as Result; +use crate::parser::schema_change::SchemaChangeEnvelope; use crate::parser::ParserConfig; use crate::source::filesystem::FsPageItem; use crate::source::monitor::EnumeratorMetrics; @@ -178,6 +180,9 @@ pub struct SourceContext { pub metrics: Arc, pub source_ctrl_opts: SourceCtrlOpts, pub connector_props: ConnectorProperties, + // source parser put schema change event into this channel + pub schema_change_tx: + Option)>>, } impl SourceContext { @@ -189,6 +194,9 @@ impl SourceContext { metrics: Arc, source_ctrl_opts: SourceCtrlOpts, connector_props: ConnectorProperties, + schema_change_channel: Option< + mpsc::Sender<(SchemaChangeEnvelope, tokio::sync::oneshot::Sender<()>)>, + >, ) -> Self { Self { actor_id, @@ -198,6 +206,7 @@ impl SourceContext { metrics, source_ctrl_opts, connector_props, + schema_change_tx: schema_change_channel, } } @@ -215,6 +224,7 @@ impl SourceContext { rate_limit: None, }, ConnectorProperties::default(), + None, ) } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ed8842e70825f..46124cca1f84b 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -42,6 +42,7 @@ pub mod test_source; pub use manager::{SourceColumnDesc, SourceColumnType}; use risingwave_common::array::{Array, ArrayRef}; +use risingwave_pb::ddl_service::SchemaChangeEnvelope; use thiserror_ext::AsReport; pub use crate::source::filesystem::opendal_source::{ diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index 1c73a3aeddad6..a94e07949f0f6 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -28,6 +28,7 @@ use risingwave_expr::expr_context::{expr_context_scope, FRAGMENT_ID}; use risingwave_expr::ExprError; use risingwave_pb::plan_common::ExprContext; use risingwave_pb::stream_plan::PbStreamActor; +use risingwave_rpc_client::MetaClient; use thiserror_ext::AsReport; use tokio_stream::StreamExt; use tracing::Instrument; @@ -55,6 +56,9 @@ pub struct ActorContext { pub initial_dispatch_num: usize, // mv_table_id to subscription id pub related_subscriptions: HashMap>, + + // Meta client. currently used for auto schema change + pub meta_client: Option, } pub type ActorContextRef = Arc; @@ -72,6 +76,7 @@ impl ActorContext { // Set 1 for test to enable sanity check on table initial_dispatch_num: 1, related_subscriptions: HashMap::new(), + meta_client: None, }) } @@ -81,6 +86,7 @@ impl ActorContext { streaming_metrics: Arc, initial_dispatch_num: usize, related_subscriptions: HashMap>, + meta_client: Option, ) -> ActorContextRef { Arc::new(Self { id: stream_actor.actor_id, @@ -92,6 +98,7 @@ impl ActorContext { streaming_metrics, initial_dispatch_num, related_subscriptions, + meta_client, }) } diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 788a9a45662cd..13580ca49e001 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -177,6 +177,7 @@ impl FsFetchExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ) } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 32d5d533d904a..6754570c4930b 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -108,6 +108,7 @@ impl FsSourceExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 4a34eabe97e16..8351e48023a05 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -216,6 +216,7 @@ impl SourceBackfillExecutorInner { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 8e5c3f9726c28..9ad5359b01aa5 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -24,6 +24,7 @@ use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::{Epoch, EpochPair}; +use risingwave_connector::parser::schema_change::SchemaChangeEnvelope; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::reader::reader::SourceReader; use risingwave_connector::source::{ @@ -32,8 +33,8 @@ use risingwave_connector::source::{ }; use risingwave_hummock_sdk::HummockReadEpoch; use thiserror_ext::AsReport; -use tokio::sync::mpsc; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; +use tokio::sync::{mpsc, oneshot}; use tokio::time::Instant; use super::executor_core::StreamSourceCore; @@ -122,6 +123,32 @@ impl SourceExecutor { .iter() .map(|column_desc| column_desc.column_id) .collect_vec(); + + let (schema_change_tx, rx) = + tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(32); + let meta_client = self.actor_ctx.meta_client.clone(); + let _ = tokio::task::spawn(async move { + let mut schema_change_rx = rx; + while let Some((schema_change, parser_tx)) = schema_change_rx.recv().await { + // handle schema change + if let Some(ref meta_client) = meta_client { + match meta_client + .auto_schema_change(schema_change.to_protobuf()) + .await + { + Ok(_) => { + tracing::info!("schema change success"); + parser_tx.send(()).unwrap(); + } + Err(e) => { + tracing::error!(error = ?e.as_report(), "schema change error"); + parser_tx.send(()).unwrap(); + } + } + } + } + }); + let source_ctx = SourceContext::new( self.actor_ctx.id, self.stream_source_core.as_ref().unwrap().source_id, @@ -137,6 +164,7 @@ impl SourceExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + Some(schema_change_tx), ); let stream = source_desc .source diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index e7105347c9356..5ed64249bc350 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -579,6 +579,7 @@ impl StreamActorManager { ) }) .collect(), + self.env.meta_client().clone(), ); let vnode_bitmap = actor.vnode_bitmap.as_ref().map(|b| b.into()); let expr_context = actor.expr_context.clone().unwrap(); From 266354b696def27dc2d3455cd3a6a1aa1185d9f6 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 21 Jul 2024 19:16:03 +0800 Subject: [PATCH 16/75] meta->frontend rpc skeleton --- proto/ddl_service.proto | 14 ++ proto/frontend_service.proto | 39 +++++ .../src/parser/debezium/schema_change.rs | 20 +++ src/connector/src/parser/mod.rs | 7 +- .../src/schema/auto_schema_change/mod.rs | 25 +++ src/connector/src/schema/mod.rs | 1 + src/frontend/src/lib.rs | 10 +- src/frontend/src/rpc/mod.rs | 57 ++++++ src/frontend/src/session.rs | 164 +++++++++++------- src/frontend/src/test_utils.rs | 8 + src/meta/service/src/ddl_service.rs | 15 ++ src/prost/build.rs | 1 + src/prost/src/lib.rs | 3 + src/utils/pgwire/src/pg_server.rs | 18 +- 14 files changed, 314 insertions(+), 68 deletions(-) create mode 100644 proto/frontend_service.proto create mode 100644 src/connector/src/schema/auto_schema_change/mod.rs create mode 100644 src/frontend/src/rpc/mod.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 1b4f4e423949e..1210958861632 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -6,6 +6,7 @@ import "catalog.proto"; import "common.proto"; import "meta.proto"; import "stream_plan.proto"; +import "plan_common.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; @@ -444,6 +445,18 @@ message CommentOnResponse { uint64 version = 2; } +message SchemaChangeEnvelope { + string up_table_name = 1; + repeated plan_common.ColumnCatalog column_descs = 2; +} + +message AutoSchemaChangeRequest { + SchemaChangeEnvelope schema_change = 1; +} + +message AutoSchemaChangeResponse {} + + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -482,4 +495,5 @@ service DdlService { rpc GetTables(GetTablesRequest) returns (GetTablesResponse); rpc Wait(WaitRequest) returns (WaitResponse); rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); + rpc AutoSchemaChange(AutoSchemaChangeRequest) returns (AutoSchemaChangeResponse); } diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto new file mode 100644 index 0000000000000..81666cc3f6d81 --- /dev/null +++ b/proto/frontend_service.proto @@ -0,0 +1,39 @@ +syntax = "proto3"; + +package frontend_service; + +import "ddl_service.proto"; + +option java_package = "com.risingwave.proto"; +option optimize_for = SPEED; + + +message SchemaChangeRequest { + message GetReplaceTablePlanRequest { + uint32 database_id = 1; + string owner = 2; + ddl_service.SchemaChangeEnvelope schema_change = 3; + } + + oneof request { + GetReplaceTablePlanRequest replace_table_plan = 1; + } + +} + +message SchemaChangeResponse { + message GetReplaceTablePlanResponse { + ddl_service.ReplaceTablePlan table_plan = 1; + } + + oneof response { + GetReplaceTablePlanResponse replace_table_plan = 1; + } + +} + + +// Frontend will start a background worker to +service SchemaChangeService { + rpc GetNewTableStreamingGraph(SchemaChangeRequest) returns (SchemaChangeResponse); +} diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 0881cd26cc503..958748d04bb31 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -1,3 +1,17 @@ +// 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 risingwave_common::catalog::ColumnCatalog; #[derive(Debug)] @@ -10,3 +24,9 @@ pub struct TableSchemaChange { pub(crate) up_table_full_name: String, pub(crate) columns: Vec, } + +impl SchemaChangeEnvelope { + pub fn to_protobuf(&self) { + todo!() + } +} diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 195c285169bfc..f521032dfe993 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -836,7 +836,12 @@ async fn into_chunk_stream_inner( } }, - Ok(ParseResult::SchemaChange(_)) => todo!(), + Ok(ParseResult::SchemaChange(schema_change)) => { + // TODO: + // 1. block source executor + // 2. send schema change to Meta + // 3. wait for Meta to finish schema change + } } } diff --git a/src/connector/src/schema/auto_schema_change/mod.rs b/src/connector/src/schema/auto_schema_change/mod.rs new file mode 100644 index 0000000000000..44b9c88f9e347 --- /dev/null +++ b/src/connector/src/schema/auto_schema_change/mod.rs @@ -0,0 +1,25 @@ +// 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 crate::parser::schema_change::SchemaChangeEnvelope; + +/// client for auto schema change +/// Can be a global client +pub struct AutoSchemaChangeClient {} + +impl AutoSchemaChangeClient { + pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { + Ok(()) + } +} diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 9b3757e29c094..b7a50f72a1817 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -14,6 +14,7 @@ use crate::error::ConnectorError; +pub mod auto_schema_change; pub mod avro; mod loader; pub mod protobuf; diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 185d65cb567a0..c43074cbac7d8 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -37,6 +37,7 @@ #![feature(used_with_arg)] #![feature(entry_insert)] #![recursion_limit = "256"] +#![feature(once_cell_try)] #[cfg(test)] risingwave_expr_impl::enable!(); @@ -74,6 +75,7 @@ mod user; pub mod health_service; mod monitor; +pub mod rpc; mod telemetry; use std::ffi::OsString; @@ -175,6 +177,8 @@ use std::pin::Pin; use pgwire::pg_protocol::TlsConfig; +use crate::session::SESSION_MANAGER; + /// Start frontend pub fn start( opts: FrontendOpts, @@ -184,7 +188,9 @@ pub fn start( // slow compile in release mode. Box::pin(async move { let listen_addr = opts.listen_addr.clone(); - let session_mgr = SessionManagerImpl::new(opts).await.unwrap(); + let session_mgr = SESSION_MANAGER + .get_or_init(|| async { Arc::new(SessionManagerImpl::new(opts).await.unwrap()) }) + .await; let redact_sql_option_keywords = Arc::new( session_mgr .env() @@ -197,7 +203,7 @@ pub fn start( pg_serve( &listen_addr, - session_mgr, + session_mgr.clone(), TlsConfig::new_default(), Some(redact_sql_option_keywords), shutdown, diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs new file mode 100644 index 0000000000000..f312f9a31c61d --- /dev/null +++ b/src/frontend/src/rpc/mod.rs @@ -0,0 +1,57 @@ +// 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 pgwire::pg_server::SessionManager; +use risingwave_pb::frontend_service::schema_change_request::Request; +use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; +use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; + +use crate::session::SESSION_MANAGER; + +#[derive(Default)] +pub struct SchemaChangeServiceImpl {} + +impl SchemaChangeServiceImpl { + pub fn new() -> Self { + Self {} + } +} + +#[async_trait::async_trait] +impl SchemaChangeService for SchemaChangeServiceImpl { + async fn get_new_table_streaming_graph( + &self, + request: RpcRequest, + ) -> Result, Status> { + let req = request.into_inner(); + + match req.request.unwrap() { + Request::ReplaceTablePlan(req) => { + let change = req.schema_change.expect("schema_change"); + // get a session object + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + let _session = session_mgr + .get_session(req.database_id, &req.owner) + .map_err(|e| Status::internal(format!("Failed to get session: {}", e)))?; + + // call the handle alter method + } + }; + + Ok(RpcResponse::new(SchemaChangeResponse { response: None })) + } +} diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 8266fd48fcbf1..56845fc70c087 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -14,7 +14,6 @@ use std::collections::HashMap; use std::io::{Error, ErrorKind}; -#[cfg(test)] use std::net::{IpAddr, Ipv4Addr, SocketAddr}; use std::sync::atomic::{AtomicI32, Ordering}; use std::sync::{Arc, Weak}; @@ -80,6 +79,7 @@ use tokio::sync::oneshot::Sender; use tokio::sync::watch; use tokio::task::JoinHandle; use tracing::info; +use tracing::log::error; use self::cursor_manager::CursorManager; use crate::binder::{Binder, BoundStatement, ResolveQualifiedNameError}; @@ -372,6 +372,7 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) + // .add_service() .serve(host.parse().unwrap()) .await .unwrap(); @@ -1116,6 +1117,9 @@ impl SessionImpl { } } +pub static SESSION_MANAGER: tokio::sync::OnceCell> = + tokio::sync::OnceCell::const_new(); + pub struct SessionManagerImpl { env: FrontendEnv, _join_handles: Vec>, @@ -1126,6 +1130,18 @@ pub struct SessionManagerImpl { impl SessionManager for SessionManagerImpl { type Session = SessionImpl; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> std::result::Result, BoxedError> { + let dumb_addr = Address::Tcp(SocketAddr::new( + IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), + 5691, // port of meta + )); + self.connect_inner(database_id, user_name, Arc::new(dumb_addr)) + } + fn connect( &self, database: &str, @@ -1143,6 +1159,89 @@ impl SessionManager for SessionManagerImpl { )) })? .id(); + + self.connect_inner(database_id, user_name, peer_addr) + } + + /// Used when cancel request happened. + fn cancel_queries_in_session(&self, session_id: SessionId) { + self.env.cancel_queries_in_session(session_id); + } + + fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { + self.env.cancel_creating_jobs_in_session(session_id); + } + + fn end_session(&self, session: &Self::Session) { + self.delete_session(&session.session_id()); + } + + async fn shutdown(&self) { + // Clean up the session map. + self.env.sessions_map().write().clear(); + // Unregister from the meta service. + self.env.meta_client().try_unregister().await; + } +} + +impl SessionManagerImpl { + pub async fn new(opts: FrontendOpts) -> Result { + // TODO(shutdown): only save join handles that **need** to be shutdown + let (env, join_handles, shutdown_senders) = FrontendEnv::init(opts).await?; + Ok(Self { + env, + _join_handles: join_handles, + _shutdown_senders: shutdown_senders, + number: AtomicI32::new(0), + }) + } + + pub fn env(&self) -> &FrontendEnv { + &self.env + } + + fn insert_session(&self, session: Arc) { + let active_sessions = { + let mut write_guard = self.env.sessions_map.write(); + write_guard.insert(session.id(), session); + write_guard.len() + }; + self.env + .frontend_metrics + .active_sessions + .set(active_sessions as i64); + } + + fn delete_session(&self, session_id: &SessionId) { + let active_sessions = { + let mut write_guard = self.env.sessions_map.write(); + write_guard.remove(session_id); + write_guard.len() + }; + self.env + .frontend_metrics + .active_sessions + .set(active_sessions as i64); + } + + fn connect_inner( + &self, + database_id: u32, + user_name: &str, + peer_addr: AddressRef, + ) -> std::result::Result, BoxedError> { + let catalog_reader = self.env.catalog_reader(); + let reader = catalog_reader.read_guard(); + let database_name = reader + .get_database_by_id(&database_id) + .map_err(|_| { + Box::new(Error::new( + ErrorKind::InvalidInput, + format!("database \"{}\" does not exist", database_id), + )) + })? + .name(); + let user_reader = self.env.user_info_reader(); let reader = user_reader.read_guard(); if let Some(user) = reader.get_user_by_name(user_name) { @@ -1197,7 +1296,7 @@ impl SessionManager for SessionManagerImpl { let session_impl: Arc = SessionImpl::new( self.env.clone(), Arc::new(AuthContext::new( - database.to_string(), + database_name.to_string(), user_name.to_string(), user.id, )), @@ -1217,67 +1316,6 @@ impl SessionManager for SessionManagerImpl { ))) } } - - /// Used when cancel request happened. - fn cancel_queries_in_session(&self, session_id: SessionId) { - self.env.cancel_queries_in_session(session_id); - } - - fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { - self.env.cancel_creating_jobs_in_session(session_id); - } - - fn end_session(&self, session: &Self::Session) { - self.delete_session(&session.session_id()); - } - - async fn shutdown(&self) { - // Clean up the session map. - self.env.sessions_map().write().clear(); - // Unregister from the meta service. - self.env.meta_client().try_unregister().await; - } -} - -impl SessionManagerImpl { - pub async fn new(opts: FrontendOpts) -> Result { - // TODO(shutdown): only save join handles that **need** to be shutdown - let (env, join_handles, shutdown_senders) = FrontendEnv::init(opts).await?; - Ok(Self { - env, - _join_handles: join_handles, - _shutdown_senders: shutdown_senders, - number: AtomicI32::new(0), - }) - } - - pub fn env(&self) -> &FrontendEnv { - &self.env - } - - fn insert_session(&self, session: Arc) { - let active_sessions = { - let mut write_guard = self.env.sessions_map.write(); - write_guard.insert(session.id(), session); - write_guard.len() - }; - self.env - .frontend_metrics - .active_sessions - .set(active_sessions as i64); - } - - fn delete_session(&self, session_id: &SessionId) { - let active_sessions = { - let mut write_guard = self.env.sessions_map.write(); - write_guard.remove(session_id); - write_guard.len() - }; - self.env - .frontend_metrics - .active_sessions - .set(active_sessions as i64); - } } impl Session for SessionImpl { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 24453f766c72c..6c6d9958210d6 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -85,6 +85,14 @@ pub struct LocalFrontend { impl SessionManager for LocalFrontend { type Session = SessionImpl; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> std::result::Result, BoxedError> { + todo!() + } + fn connect( &self, _database: &str, diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 0eec9add5b185..560c6b369e946 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -916,6 +916,21 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(AlterParallelismResponse {})) } + + async fn auto_schema_change( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + + let schema_change = req.schema_change.unwrap(); + + // TODO: can we build a ReplaceTablePlan using the info in schema_change? + + // send a request to the frontend to get the ReplaceTablePlan + + Ok(Response::new(AutoSchemaChangeResponse {})) + } } impl DdlServiceImpl { diff --git a/src/prost/build.rs b/src/prost/build.rs index f8e651e7103a3..96dbf27a721d1 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -54,6 +54,7 @@ fn main() -> Result<(), Box> { "telemetry", "user", "secret", + "frontend_service", ]; let protos: Vec = proto_files .iter() diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index f26f8c9f38d91..9ae9448b645e4 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -83,6 +83,9 @@ pub mod monitor_service; #[cfg_attr(madsim, path = "sim/backup_service.rs")] pub mod backup_service; #[rustfmt::skip] +#[cfg_attr(madsim, path = "sim/frontend_service.rs")] +pub mod frontend_service; +#[rustfmt::skip] #[cfg_attr(madsim, path = "sim/java_binding.rs")] pub mod java_binding; #[rustfmt::skip] diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 840f21dda1be2..e785f5409f0dd 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -47,6 +47,12 @@ pub type SessionId = (ProcessId, SecretKey); pub trait SessionManager: Send + Sync + 'static { type Session: Session; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> Result, BoxedError>; + fn connect( &self, database: &str, @@ -257,7 +263,7 @@ impl UserAuthenticator { /// Returns when the `shutdown` token is triggered. pub async fn pg_serve( addr: &str, - session_mgr: impl SessionManager, + session_mgr: Arc, tls_config: Option, redact_sql_option_keywords: Option, shutdown: CancellationToken, @@ -280,7 +286,7 @@ pub async fn pg_serve( #[cfg(madsim)] let worker_runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); - let session_mgr = Arc::new(session_mgr); + // let session_mgr = Arc::new(session_mgr); let session_mgr_clone = session_mgr.clone(); let f = async move { loop { @@ -380,6 +386,14 @@ mod tests { impl SessionManager for MockSessionManager { type Session = MockSession; + fn get_session( + &self, + database_id: u32, + user_name: &str, + ) -> Result, BoxedError> { + todo!() + } + fn connect( &self, _database: &str, From 32b93771c162f24f9a3fc279887dde95d759012f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 17:12:31 +0800 Subject: [PATCH 17/75] call frontend to generate new table plan --- proto/frontend_service.proto | 11 +- .../src/handler/alter_table_column.rs | 125 ++++++++++++++++-- src/frontend/src/handler/mod.rs | 2 + src/frontend/src/rpc/mod.rs | 71 ++++++++-- src/rpc_client/src/error.rs | 2 +- 5 files changed, 185 insertions(+), 26 deletions(-) diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 81666cc3f6d81..ce5f49a49be27 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -9,25 +9,26 @@ option optimize_for = SPEED; message SchemaChangeRequest { - message GetReplaceTablePlanRequest { + message GetNewTablePlanRequest { uint32 database_id = 1; string owner = 2; - ddl_service.SchemaChangeEnvelope schema_change = 3; + string table_name = 3; + ddl_service.SchemaChangeEnvelope schema_change = 4; } oneof request { - GetReplaceTablePlanRequest replace_table_plan = 1; + GetNewTablePlanRequest get_new_table_plan = 1; } } message SchemaChangeResponse { - message GetReplaceTablePlanResponse { + message GetNewTablePlanResponse { ddl_service.ReplaceTablePlan table_plan = 1; } oneof response { - GetReplaceTablePlanResponse replace_table_plan = 1; + GetNewTablePlanResponse replace_table_plan = 1; } } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index bd102f88553b6..bccf9e9797521 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -14,13 +14,19 @@ use std::sync::Arc; -use anyhow::Context; +use anyhow::{anyhow, Context}; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::catalog::ColumnCatalog; +use risingwave_common::types::DataType; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::{bail, bail_not_implemented}; +use risingwave_pb::catalog::{Source, Table}; +use risingwave_pb::ddl_service::TableJobType; +use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - AlterTableOperation, ColumnOption, ConnectorSchema, Encode, ObjectName, Statement, + AlterTableOperation, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Encode, + ObjectName, Statement, StructField, }; use risingwave_sqlparser::parser::Parser; @@ -30,7 +36,7 @@ use super::util::SourceSchemaCompatExt; use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; -use crate::error::{ErrorCode, Result}; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::ExprImpl; use crate::session::SessionImpl; use crate::{Binder, TableCatalog, WithOptions}; @@ -42,6 +48,112 @@ pub async fn replace_table_with_definition( original_catalog: &Arc, source_schema: Option, ) -> Result<()> { + let (source, table, graph, col_index_mapping, job_type) = get_replace_table_plan( + session, + table_name, + definition, + original_catalog, + source_schema, + ) + .await?; + + let catalog_writer = session.catalog_writer()?; + + catalog_writer + .replace_table(source, table, graph, col_index_mapping, job_type) + .await?; + Ok(()) +} + +pub async fn get_new_table_definition_for_cdc_table( + session: &Arc, + table_name: ObjectName, + new_columns: Vec, +) -> Result<(Statement, Arc)> { + let original_catalog = fetch_table_catalog_for_alter(session.as_ref(), &table_name)?; + + // Retrieve the original table definition and parse it to AST. + let [mut definition]: [_; 1] = Parser::parse_sql(&original_catalog.definition) + .context("unable to parse original table definition")? + .try_into() + .unwrap(); + let Statement::CreateTable { + columns: original_columns, + source_schema, + .. + } = &mut definition + else { + panic!("unexpected statement: {:?}", definition); + }; + + assert!( + source_schema.is_none(), + "source schema should be None for CDC table" + ); + if original_columns.is_empty() { + Err(ErrorCode::NotSupported( + "alter a table with empty column definitions".to_string(), + "Please recreate the table with column definitions.".to_string(), + ))? + } + + // since the DDL is committed on upstream, so we can safely replace the original columns with new columns + // replace original columns with new columns + let mut new_column_defs = vec![]; + for col in new_columns.into_iter() { + let ty = to_ast_data_type(col.data_type())?; + new_column_defs.push(ColumnDef::new(col.name().into(), ty, None, vec![])); + } + *original_columns = new_column_defs; + + Ok((definition, original_catalog)) +} + +fn to_ast_data_type(ty: &DataType) -> Result { + match ty { + DataType::Boolean => Ok(AstDataType::Boolean), + DataType::Int16 => Ok(AstDataType::SmallInt), + DataType::Int32 => Ok(AstDataType::Int), + DataType::Int64 => Ok(AstDataType::BigInt), + DataType::Float32 => Ok(AstDataType::Real), + DataType::Float64 => Ok(AstDataType::Double), + DataType::Date => Ok(AstDataType::Date), + DataType::Time => Ok(AstDataType::Time(false)), + DataType::Timestamp => Ok(AstDataType::Timestamp(false)), + DataType::Timestamptz => Ok(AstDataType::Timestamp(true)), + DataType::Interval => Ok(AstDataType::Interval), + DataType::Jsonb => Ok(AstDataType::Jsonb), + DataType::Bytea => Ok(AstDataType::Bytea), + DataType::List(item_ty) => Ok(AstDataType::Array(Box::new(to_ast_data_type(item_ty)?))), + DataType::Struct(fields) => { + let fields = fields + .iter() + .map(|(name, ty)| { + Ok::(StructField { + name: name.into(), + data_type: to_ast_data_type(ty)?, + }) + }) + .try_collect()?; + Ok(AstDataType::Struct(fields)) + } + _ => Err(anyhow!("unsupported data type: {:?}", ty).context("to_ast_data_type"))?, + } +} + +pub async fn get_replace_table_plan( + session: &Arc, + table_name: ObjectName, + definition: Statement, + original_catalog: &Arc, + source_schema: Option, +) -> Result<( + Option, + Table, + StreamFragmentGraph, + ColIndexMapping, + TableJobType, +)> { // Create handler args as if we're creating a new table with the altered definition. let handler_args = HandlerArgs::new(session.clone(), &definition, Arc::from(""))?; let col_id_gen = ColumnIdGenerator::new_alter(original_catalog); @@ -92,12 +204,7 @@ pub async fn replace_table_with_definition( table.columns.len(), ); - let catalog_writer = session.catalog_writer()?; - - catalog_writer - .replace_table(source, table, graph, col_index_mapping, job_type) - .await?; - Ok(()) + Ok((source, table, graph, col_index_mapping, job_type)) } /// Handle `ALTER TABLE [ADD|DROP] COLUMN` statements. The `operation` must be either `AddColumn` or diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index f8beeedb19438..73fb4f08ff09a 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -100,6 +100,8 @@ pub mod util; pub mod variable; mod wait; +pub use alter_table_column::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; + /// The [`PgResponseBuilder`] used by RisingWave. pub type RwPgResponseBuilder = PgResponseBuilder; diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index f312f9a31c61d..af5be7b3df118 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -12,14 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. -use pgwire::pg_server::SessionManager; +use futures::TryStreamExt; +use pgwire::pg_server::{BoxedError, SessionManager}; +use risingwave_pb::ddl_service::SchemaChangeEnvelope; use risingwave_pb::frontend_service::schema_change_request::Request; use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use risingwave_rpc_client::error::ToTonicStatus; +use risingwave_sqlparser::ast::ObjectName; use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; +use crate::error::RwError; +use crate::handler::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; use crate::session::SESSION_MANAGER; +#[derive(thiserror::Error, Debug)] +pub enum AutoSchemaChangeError { + #[error("frontend error")] + FrontendError( + #[from] + #[backtrace] + RwError, + ), +} + +impl From for AutoSchemaChangeError { + fn from(err: BoxedError) -> Self { + AutoSchemaChangeError::FrontendError(RwError::from(err)) + } +} + +impl From for tonic::Status { + fn from(err: AutoSchemaChangeError) -> Self { + err.to_status(tonic::Code::Internal, "frontend") + } +} + #[derive(Default)] pub struct SchemaChangeServiceImpl {} @@ -38,20 +66,41 @@ impl SchemaChangeService for SchemaChangeServiceImpl { let req = request.into_inner(); match req.request.unwrap() { - Request::ReplaceTablePlan(req) => { + Request::GetNewTablePlan(req) => { let change = req.schema_change.expect("schema_change"); - // get a session object - let session_mgr = SESSION_MANAGER - .get() - .expect("session manager has been initialized"); - let _session = session_mgr - .get_session(req.database_id, &req.owner) - .map_err(|e| Status::internal(format!("Failed to get session: {}", e)))?; - - // call the handle alter method + get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; } }; Ok(RpcResponse::new(SchemaChangeResponse { response: None })) } } + +async fn get_new_table_plan( + change: SchemaChangeEnvelope, + table_name: String, + database_id: u32, + owner: String, +) -> Result<(), AutoSchemaChangeError> { + // get a session object + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + let session = session_mgr.get_session(database_id, &owner)?; + + // call the handle alter method + let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); + let table_name = ObjectName::from(vec![table_name.as_str().into()]); + let (new_table_definition, original_catalog) = + get_new_table_definition_for_cdc_table(&session, table_name.clone(), new_columns).await?; + let (_, table, graph, col_index_mapping, job_type) = get_replace_table_plan( + &session, + table_name, + new_table_definition, + &original_catalog, + None, + ) + .await?; + + Ok(()) +} diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index c5c5613a32a4b..7ffab0736179e 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -76,4 +76,4 @@ macro_rules! impl_from_status { }; } -impl_from_status!(stream, batch, meta, compute, compactor, connector); +impl_from_status!(stream, batch, meta, compute, compactor, connector, frontend); From 253ea4173255f51be6529c96416d1e8d930cd8d3 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 17:45:12 +0800 Subject: [PATCH 18/75] minor --- src/frontend/src/rpc/mod.rs | 37 +++++++++++++++++++++++++------------ 1 file changed, 25 insertions(+), 12 deletions(-) diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index af5be7b3df118..c7c4162ac1591 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use futures::TryStreamExt; use pgwire::pg_server::{BoxedError, SessionManager}; -use risingwave_pb::ddl_service::SchemaChangeEnvelope; +use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; use risingwave_pb::frontend_service::schema_change_request::Request; +use risingwave_pb::frontend_service::schema_change_response::{GetNewTablePlanResponse, Response}; use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; use risingwave_rpc_client::error::ToTonicStatus; @@ -65,14 +65,20 @@ impl SchemaChangeService for SchemaChangeServiceImpl { ) -> Result, Status> { let req = request.into_inner(); - match req.request.unwrap() { - Request::GetNewTablePlan(req) => { - let change = req.schema_change.expect("schema_change"); + if let Some(Request::GetNewTablePlan(req)) = req.request { + let change = req + .schema_change + .expect("schema change message is required"); + let replace_plan = get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; - } - }; - - Ok(RpcResponse::new(SchemaChangeResponse { response: None })) + Ok(RpcResponse::new(SchemaChangeResponse { + response: Some(Response::ReplaceTablePlan(GetNewTablePlanResponse { + table_plan: Some(replace_plan), + })), + })) + } else { + Err(Status::invalid_argument("invalid schema change request")) + } } } @@ -81,11 +87,12 @@ async fn get_new_table_plan( table_name: String, database_id: u32, owner: String, -) -> Result<(), AutoSchemaChangeError> { - // get a session object +) -> Result { let session_mgr = SESSION_MANAGER .get() .expect("session manager has been initialized"); + + // get a session object for the corresponding user and database let session = session_mgr.get_session(database_id, &owner)?; // call the handle alter method @@ -102,5 +109,11 @@ async fn get_new_table_plan( ) .await?; - Ok(()) + Ok(ReplaceTablePlan { + table: Some(table), + fragment_graph: Some(graph), + table_col_index_mapping: Some(col_index_mapping.to_protobuf()), + source: None, + job_type: job_type as _, + }) } From e6cfb506d51929d87f24b39f691d5c611ae2108f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 22 Jul 2024 18:11:40 +0800 Subject: [PATCH 19/75] launch rpc service --- src/frontend/src/session.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 56845fc70c087..e1cd2523b12ad 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -67,6 +67,7 @@ use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_connector::source::monitor::{SourceMetrics, GLOBAL_SOURCE_METRICS}; use risingwave_pb::common::WorkerType; +use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeServiceServer; use risingwave_pb::health::health_server::HealthServer; use risingwave_pb::user::auth_info::EncryptionType; use risingwave_pb::user::grant_privilege::Object; @@ -105,6 +106,7 @@ use crate::health_service::HealthServiceImpl; use crate::meta_client::{FrontendMetaClient, FrontendMetaClientImpl}; use crate::monitor::{FrontendMetrics, GLOBAL_FRONTEND_METRICS}; use crate::observer::FrontendObserverNode; +use crate::rpc::SchemaChangeServiceImpl; use crate::scheduler::streaming_manager::{StreamingJobTracker, StreamingJobTrackerRef}; use crate::scheduler::{ DistributedQueryMetrics, HummockSnapshotManager, HummockSnapshotManagerRef, QueryManager, @@ -352,6 +354,7 @@ impl FrontendEnv { } let health_srv = HealthServiceImpl::new(); + let schema_change_srv = SchemaChangeServiceImpl::new(); let host = opts.health_check_listener_addr.clone(); let telemetry_manager = TelemetryManager::new( @@ -372,13 +375,13 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) - // .add_service() + .add_service(SchemaChangeServiceServer::new(schema_change_srv)) .serve(host.parse().unwrap()) .await .unwrap(); }); info!( - "Health Check RPC Listener is set up on {}", + "Frontend RPC Listener is set up on {}", opts.health_check_listener_addr.clone() ); From f1c41b011012e756da3650079891d6f71f2b6218 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 11:11:48 +0800 Subject: [PATCH 20/75] refactor frontend service --- proto/frontend_service.proto | 32 +++++++--------------- src/frontend/src/rpc/mod.rs | 42 ++++++++++++----------------- src/frontend/src/session.rs | 8 +++--- src/meta/service/src/ddl_service.rs | 1 + 4 files changed, 31 insertions(+), 52 deletions(-) diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index ce5f49a49be27..d633734bd58b5 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -8,33 +8,19 @@ option java_package = "com.risingwave.proto"; option optimize_for = SPEED; -message SchemaChangeRequest { - message GetNewTablePlanRequest { - uint32 database_id = 1; - string owner = 2; - string table_name = 3; - ddl_service.SchemaChangeEnvelope schema_change = 4; - } - - oneof request { - GetNewTablePlanRequest get_new_table_plan = 1; - } - +message GetTableReplacePlanRequest { + uint32 database_id = 1; + string owner = 2; + string table_name = 3; + ddl_service.SchemaChangeEnvelope schema_change = 4; } -message SchemaChangeResponse { - message GetNewTablePlanResponse { - ddl_service.ReplaceTablePlan table_plan = 1; - } - - oneof response { - GetNewTablePlanResponse replace_table_plan = 1; - } +message GetTableReplacePlanResponse { + ddl_service.ReplaceTablePlan table_plan = 1; } - // Frontend will start a background worker to -service SchemaChangeService { - rpc GetNewTableStreamingGraph(SchemaChangeRequest) returns (SchemaChangeResponse); +service FrontendService { + rpc GetTableReplacePlan(GetTableReplacePlanRequest) returns (GetTableReplacePlanResponse); } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index c7c4162ac1591..acf37d7cab323 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -14,13 +14,11 @@ use pgwire::pg_server::{BoxedError, SessionManager}; use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; -use risingwave_pb::frontend_service::schema_change_request::Request; -use risingwave_pb::frontend_service::schema_change_response::{GetNewTablePlanResponse, Response}; -use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeService; -use risingwave_pb::frontend_service::{SchemaChangeRequest, SchemaChangeResponse}; +use risingwave_pb::frontend_service::frontend_service_server::FrontendService; +use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; use risingwave_rpc_client::error::ToTonicStatus; use risingwave_sqlparser::ast::ObjectName; -use tonic::{Request as RpcRequest, Response as RpcResponse, Status}; +use tonic::{Request as RpcRequest, Response as RpcResponse, Response, Status}; use crate::error::RwError; use crate::handler::{get_new_table_definition_for_cdc_table, get_replace_table_plan}; @@ -49,36 +47,30 @@ impl From for tonic::Status { } #[derive(Default)] -pub struct SchemaChangeServiceImpl {} +pub struct FrontendServiceImpl {} -impl SchemaChangeServiceImpl { +impl FrontendServiceImpl { pub fn new() -> Self { Self {} } } #[async_trait::async_trait] -impl SchemaChangeService for SchemaChangeServiceImpl { - async fn get_new_table_streaming_graph( +impl FrontendService for FrontendServiceImpl { + async fn get_table_replace_plan( &self, - request: RpcRequest, - ) -> Result, Status> { + request: RpcRequest, + ) -> Result, Status> { let req = request.into_inner(); - if let Some(Request::GetNewTablePlan(req)) = req.request { - let change = req - .schema_change - .expect("schema change message is required"); - let replace_plan = - get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; - Ok(RpcResponse::new(SchemaChangeResponse { - response: Some(Response::ReplaceTablePlan(GetNewTablePlanResponse { - table_plan: Some(replace_plan), - })), - })) - } else { - Err(Status::invalid_argument("invalid schema change request")) - } + let change = req + .schema_change + .expect("schema change message is required"); + let replace_plan = + get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; + Ok(RpcResponse::new(GetTableReplacePlanResponse { + table_plan: Some(replace_plan), + })) } } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index e1cd2523b12ad..62e1624efdcb9 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -67,7 +67,7 @@ use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_connector::source::monitor::{SourceMetrics, GLOBAL_SOURCE_METRICS}; use risingwave_pb::common::WorkerType; -use risingwave_pb::frontend_service::schema_change_service_server::SchemaChangeServiceServer; +use risingwave_pb::frontend_service::frontend_service_server::FrontendServiceServer; use risingwave_pb::health::health_server::HealthServer; use risingwave_pb::user::auth_info::EncryptionType; use risingwave_pb::user::grant_privilege::Object; @@ -106,7 +106,7 @@ use crate::health_service::HealthServiceImpl; use crate::meta_client::{FrontendMetaClient, FrontendMetaClientImpl}; use crate::monitor::{FrontendMetrics, GLOBAL_FRONTEND_METRICS}; use crate::observer::FrontendObserverNode; -use crate::rpc::SchemaChangeServiceImpl; +use crate::rpc::FrontendServiceImpl; use crate::scheduler::streaming_manager::{StreamingJobTracker, StreamingJobTrackerRef}; use crate::scheduler::{ DistributedQueryMetrics, HummockSnapshotManager, HummockSnapshotManagerRef, QueryManager, @@ -354,7 +354,7 @@ impl FrontendEnv { } let health_srv = HealthServiceImpl::new(); - let schema_change_srv = SchemaChangeServiceImpl::new(); + let frontend_srv = FrontendServiceImpl::new(); let host = opts.health_check_listener_addr.clone(); let telemetry_manager = TelemetryManager::new( @@ -375,7 +375,7 @@ impl FrontendEnv { tokio::spawn(async move { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) - .add_service(SchemaChangeServiceServer::new(schema_change_srv)) + .add_service(FrontendServiceServer::new(frontend_srv)) .serve(host.parse().unwrap()) .await .unwrap(); diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 560c6b369e946..6fdb622e3e096 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -49,6 +49,7 @@ pub struct DdlServiceImpl { sink_manager: SinkCoordinatorManager, ddl_controller: DdlController, aws_client: Arc>, + // frontend_client: } impl DdlServiceImpl { From dcb359e26fb634e3cf898a2c302803b74f5f898e Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 18:07:51 +0800 Subject: [PATCH 21/75] add cdc table name to PbTable --- proto/catalog.proto | 2 ++ src/frontend/src/catalog/table_catalog.rs | 4 ++++ src/frontend/src/handler/create_table.rs | 4 +++- src/frontend/src/optimizer/mod.rs | 2 ++ src/frontend/src/optimizer/plan_node/stream_materialize.rs | 6 +++++- src/frontend/src/optimizer/plan_node/utils.rs | 1 + src/meta/model_v2/src/table.rs | 2 ++ src/meta/src/controller/mod.rs | 1 + 8 files changed, 20 insertions(+), 2 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index b18275e32d4ce..3c3471d2fffbe 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -406,6 +406,8 @@ message Table { // conflict" operations. optional uint32 version_column_index = 38; + optional string cdc_table_name = 39; + // Per-table catalog version, used by schema change. `None` for internal // tables and tests. Not to be confused with the global catalog version for // notification service. diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 29c74a64e2551..5d5e7e9060872 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -168,6 +168,8 @@ pub struct TableCatalog { pub created_at_cluster_version: Option, pub initialized_at_cluster_version: Option, + + pub cdc_table_name: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -425,6 +427,7 @@ impl TableCatalog { created_at_cluster_version: self.created_at_cluster_version.clone(), initialized_at_cluster_version: self.initialized_at_cluster_version.clone(), retention_seconds: self.retention_seconds, + cdc_table_name: self.cdc_table_name.clone(), } } @@ -580,6 +583,7 @@ impl From for TableCatalog { .into_iter() .map(TableId::from) .collect_vec(), + cdc_table_name: tb.cdc_table_name, } } } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5dce6028d0cc0..6d5fae241a02a 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -727,6 +727,7 @@ fn gen_table_plan_inner( version, is_external_source, retention_seconds, + None, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -810,7 +811,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_table( let options = CdcScanOptions::from_with_options(context.with_options())?; let logical_scan = LogicalCdcScan::create( - external_table_name, + external_table_name.clone(), Rc::new(cdc_table_desc), context.clone(), options, @@ -840,6 +841,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_table( Some(col_id_gen.into_version()), true, None, + Some(external_table_name), )?; let mut table = materialize.table().to_prost(schema_id, database_id); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index f59c0635b8bd2..308459208b743 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -631,6 +631,7 @@ impl PlanRoot { version: Option, with_external_source: bool, retention_seconds: Option, + cdc_table_name: Option, ) -> Result { assert_eq!(self.phase, PlanPhase::Logical); assert_eq!(self.plan.convention(), Convention::Logical); @@ -860,6 +861,7 @@ impl PlanRoot { row_id_index, version, retention_seconds, + cdc_table_name, ) } diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 865dc71191b46..41b3aca4269e4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -133,10 +133,11 @@ impl StreamMaterialize { row_id_index: Option, version: Option, retention_seconds: Option, + cdc_table_name: Option, ) -> Result { let input = Self::rewrite_input(input, user_distributed_by, TableType::Table)?; - let table = Self::derive_table_catalog( + let mut table = Self::derive_table_catalog( input.clone(), name, user_order_by, @@ -153,6 +154,8 @@ impl StreamMaterialize { CreateType::Foreground, )?; + table.cdc_table_name = cdc_table_name; + Ok(Self::new(input, table)) } @@ -279,6 +282,7 @@ impl StreamMaterialize { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: retention_seconds.map(|i| i.into()), + cdc_table_name: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 2ac317d597bad..85327dc8de098 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -178,6 +178,7 @@ impl TableCatalogBuilder { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: None, + cdc_table_name: None, } } diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 75fb66fcb2a5e..5696a9a91caca 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -132,6 +132,7 @@ pub struct Model { pub version: Option, pub retention_seconds: Option, pub incoming_sinks: I32Array, + pub cdc_table_name: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -243,6 +244,7 @@ impl From for ActiveModel { version: Set(pb_table.version.as_ref().map(|v| v.into())), retention_seconds: Set(pb_table.retention_seconds.map(|i| i as _)), incoming_sinks: Set(pb_table.incoming_sinks.into()), + cdc_table_name: Set(pb_table.cdc_table_name), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 5ace222739d52..5dcc936e0778b 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -161,6 +161,7 @@ impl From> for PbTable { initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, retention_seconds: value.0.retention_seconds.map(|id| id as u32), + cdc_table_name: value.0.cdc_table_name, } } } From 0539172bea601bb66dd083cd7a2c88f82c42421b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 23 Jul 2024 19:41:38 +0800 Subject: [PATCH 22/75] wip: meta->frontend rpc implementation --- proto/ddl_service.proto | 2 +- proto/frontend_service.proto | 2 +- src/frontend/src/rpc/mod.rs | 4 +- src/frontend/src/session.rs | 13 +++- src/frontend/src/test_utils.rs | 2 +- src/meta/service/src/ddl_service.rs | 44 +++++++++++++- src/meta/src/controller/catalog.rs | 16 +++++ src/meta/src/manager/catalog/database.rs | 9 +++ src/meta/src/manager/catalog/mod.rs | 8 +++ src/meta/src/manager/env.rs | 14 ++++- src/meta/src/manager/metadata.rs | 16 +++++ src/rpc_client/src/frontend_client.rs | 76 ++++++++++++++++++++++++ src/rpc_client/src/lib.rs | 19 ++++++ src/utils/pgwire/src/pg_server.rs | 9 +-- 14 files changed, 217 insertions(+), 17 deletions(-) create mode 100644 src/rpc_client/src/frontend_client.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 1210958861632..0b813ec7197c9 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -446,7 +446,7 @@ message CommentOnResponse { } message SchemaChangeEnvelope { - string up_table_name = 1; + string cdc_table_name = 1; repeated plan_common.ColumnCatalog column_descs = 2; } diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index d633734bd58b5..247c580cb392e 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -10,7 +10,7 @@ option optimize_for = SPEED; message GetTableReplacePlanRequest { uint32 database_id = 1; - string owner = 2; + uint32 owner = 2; string table_name = 3; ddl_service.SchemaChangeEnvelope schema_change = 4; } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index acf37d7cab323..8651a19165db6 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -78,14 +78,14 @@ async fn get_new_table_plan( change: SchemaChangeEnvelope, table_name: String, database_id: u32, - owner: String, + owner: u32, ) -> Result { let session_mgr = SESSION_MANAGER .get() .expect("session manager has been initialized"); // get a session object for the corresponding user and database - let session = session_mgr.get_session(database_id, &owner)?; + let session = session_mgr.get_session(database_id, owner)?; // call the handle alter method let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 62e1624efdcb9..794471fc7d74b 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1136,13 +1136,22 @@ impl SessionManager for SessionManagerImpl { fn get_session( &self, database_id: u32, - user_name: &str, + user_id: u32, ) -> std::result::Result, BoxedError> { let dumb_addr = Address::Tcp(SocketAddr::new( IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691, // port of meta )); - self.connect_inner(database_id, user_name, Arc::new(dumb_addr)) + let user_reader = self.env.user_info_reader(); + let reader = user_reader.read_guard(); + if let Some(user_name) = reader.get_user_name_by_id(user_id) { + self.connect_inner(database_id, user_name.as_str(), Arc::new(dumb_addr)) + } else { + Err(Box::new(Error::new( + ErrorKind::InvalidInput, + format!("Role id {} does not exist", user_id), + ))) + } } fn connect( diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 6c6d9958210d6..0f4a656e7dc9b 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -88,7 +88,7 @@ impl SessionManager for LocalFrontend { fn get_session( &self, database_id: u32, - user_name: &str, + user_name: u32, ) -> std::result::Result, BoxedError> { todo!() } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 6fdb622e3e096..59294e2ffdb56 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -16,6 +16,8 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::anyhow; +use rand::seq::SliceRandom; +use rand::thread_rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_connector::sink::catalog::SinkId; use risingwave_meta::manager::MetadataManager; @@ -25,10 +27,13 @@ use risingwave_pb::catalog::connection::private_link_service::{ }; use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, Secret}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, Secret, Table}; +use risingwave_pb::common::worker_node::State; +use risingwave_pb::common::WorkerType; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; +use risingwave_pb::frontend_service::GetTableReplacePlanRequest; use tonic::{Request, Response, Status}; use crate::barrier::BarrierManagerRef; @@ -49,7 +54,6 @@ pub struct DdlServiceImpl { sink_manager: SinkCoordinatorManager, ddl_controller: DdlController, aws_client: Arc>, - // frontend_client: } impl DdlServiceImpl { @@ -925,10 +929,44 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); let schema_change = req.schema_change.unwrap(); + let cdc_table_name = schema_change.cdc_table_name.clone(); - // TODO: can we build a ReplaceTablePlan using the info in schema_change? + // get the table catalog corresponding to the + let tables: Vec
= self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) + .await?; // send a request to the frontend to get the ReplaceTablePlan + let mut workers = self + .metadata_manager + .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) + .await?; + workers.shuffle(&mut thread_rng()); + let worker = workers + .first() + .ok_or_else(|| anyhow!("no frontend worker available"))?; + let client = self.env.frontend_client_pool().get(worker).await?; + + for table in tables { + let resp = client + .get_table_replace_plan(GetTableReplacePlanRequest { + database_id: table.database_id, + owner: table.owner, + table_name: table.name, + schema_change: Some(schema_change), + }) + .await?; + + if let Some(plan) = resp.table_plan { + // start the schema change procedure + self.ddl_controller + .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( + plan, + ))) + .await?; + } + } Ok(Response::new(AutoSchemaChangeResponse {})) } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index cb5dc2bf41b7e..c818156a80f2a 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2838,6 +2838,22 @@ impl CatalogController { .collect()) } + pub async fn get_table_by_cdc_table_name( + &self, + cdc_table_name: String, + ) -> MetaResult> { + let inner = self.inner.read().await; + let table_objs = Table::find() + .find_also_related(Object) + .filter(table::Column::CdcTableName.eq(cdc_table_name)) + .all(&inner.db) + .await?; + Ok(table_objs + .into_iter() + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + pub async fn get_created_table_ids(&self) -> MetaResult> { let inner = self.inner.read().await; diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index e3b2e0d02cae7..2c977981c4676 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -248,6 +248,15 @@ impl DatabaseManager { .collect() } + pub fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ + let cdc_table_name = Some(cdc_table_name); + self.tables + .values() + .filter(|t| t.cdc_table_name == cdc_table_name) + .cloned() + .collect() + } + pub fn check_relation_name_duplicated(&self, relation_key: &RelationKey) -> MetaResult<()> { if let Some(t) = self.tables.values().find(|x| { x.database_id == relation_key.0 diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 02cc9ee8de0bf..c6ce0334dcfa9 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -4063,6 +4063,14 @@ impl CatalogManager { .get_table_name_and_type_mapping() } + pub async fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ + self.core + .lock() + .await + .database + .get_table_by_cdc_table_name(cdc_table_name) + } + /// `list_stream_job_ids` returns all running and creating stream job ids, this is for recovery /// clean up progress. pub async fn list_stream_job_ids(&self) -> MetaResult> { diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 5bb12eec9c7bf..143d02f1bf93b 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -23,7 +23,9 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; -use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; +use risingwave_rpc_client::{ + FrontendClientPool, FrontendClientPoolRef, StreamClientPool, StreamClientPoolRef, +}; use sea_orm::EntityTrait; use super::{ @@ -123,6 +125,9 @@ pub struct MetaSrvEnv { /// stream client pool memorization. stream_client_pool: StreamClientPoolRef, + /// rpc client pool for frontend nodes. + frontend_client_pool: FrontendClientPoolRef, + /// idle status manager. idle_manager: IdleManagerRef, @@ -389,6 +394,7 @@ impl MetaSrvEnv { ) -> MetaResult { let idle_manager = Arc::new(IdleManager::new(opts.max_idle_ms)); let stream_client_pool = Arc::new(StreamClientPool::new(1)); // typically no need for plural clients + let frontend_client_pool = Arc::new(FrontendClientPool::default()); let event_log_manager = Arc::new(start_event_log_manager( opts.event_log_enabled, opts.event_log_channel_max_size, @@ -444,6 +450,7 @@ impl MetaSrvEnv { meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, + frontend_client_pool, idle_manager, event_log_manager, cluster_id, @@ -499,6 +506,7 @@ impl MetaSrvEnv { meta_store_impl: meta_store_impl.clone(), notification_manager, stream_client_pool, + frontend_client_pool, idle_manager, event_log_manager, cluster_id, @@ -563,6 +571,10 @@ impl MetaSrvEnv { self.stream_client_pool.deref() } + pub fn frontend_client_pool(&self) -> &FrontendClientPool { + self.frontend_client_pool.deref() + } + pub fn cluster_id(&self) -> &ClusterId { &self.cluster_id } diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index cb90f2326d20d..fd3c24d69f30f 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -554,6 +554,22 @@ impl MetadataManager { } } + pub async fn get_table_catalog_by_cdc_table_name( + &self, + name: String, + ) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => { + Ok(mgr.catalog_manager.get_table_by_cdc_table_name(name).await) + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_table_by_cdc_table_name(name) + .await + } + } + } + pub async fn get_downstream_chain_fragments( &self, job_id: u32, diff --git a/src/rpc_client/src/frontend_client.rs b/src/rpc_client/src/frontend_client.rs new file mode 100644 index 0000000000000..088cf74e33e76 --- /dev/null +++ b/src/rpc_client/src/frontend_client.rs @@ -0,0 +1,76 @@ +// 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::sync::Arc; +use std::time::Duration; + +use anyhow::anyhow; +use async_trait::async_trait; +use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; +use risingwave_common::monitor::{EndpointExt, TcpConfig}; +use risingwave_common::util::addr::HostAddr; +use risingwave_pb::frontend_service::frontend_service_client::FrontendServiceClient; +use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; +use tonic::transport::Endpoint; + +use crate::error::{Result, RpcError}; +use crate::tracing::{Channel, TracingInjectedChannelExt}; +use crate::{frontend_rpc_client_method_impl, RpcClient, RpcClientPool}; + +#[derive(Clone)] +pub struct FrontendClient(FrontendServiceClient); + +#[async_trait] +impl RpcClient for FrontendClient { + async fn new_client(host_addr: HostAddr) -> Result { + Self::new(host_addr).await + } +} + +impl FrontendClient { + async fn new(host_addr: HostAddr) -> Result { + let channel = Endpoint::from_shared(format!("http://{}", &host_addr))? + .initial_connection_window_size(MAX_CONNECTION_WINDOW_SIZE) + .connect_timeout(Duration::from_secs(5)) + .monitored_connect( + "grpc-frontend-client", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + ) + .await? + .tracing_injected(); + + Ok(Self( + FrontendServiceClient::new(channel).max_decoding_message_size(usize::MAX), + )) + } +} + +// similar to the stream_client used in the Meta node +pub type FrontendClientPool = RpcClientPool; +pub type FrontendClientPoolRef = Arc; + +macro_rules! for_all_frontend_rpc { + ($macro:ident) => { + $macro! { + { 0, get_table_replace_plan, GetTableReplacePlanRequest, GetTableReplacePlanResponse } + } + }; +} + +impl FrontendClient { + for_all_frontend_rpc! { frontend_rpc_client_method_impl } +} diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index bb1d90dcffbf4..037ae44972fc8 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -53,6 +53,7 @@ use error::Result; mod compactor_client; mod compute_client; mod connector_client; +mod frontend_client; mod hummock_meta_client; mod meta_client; mod sink_coordinate_client; @@ -62,6 +63,7 @@ mod tracing; pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; +pub use frontend_client::{FrontendClient, FrontendClientPool, FrontendClientPoolRef}; pub use hummock_meta_client::{CompactionEventItem, HummockMetaClient}; pub use meta_client::{MetaClient, SinkCoordinationRpcClient}; use rw_futures_util::await_future_with_monitor_error_stream; @@ -196,6 +198,23 @@ macro_rules! meta_rpc_client_method_impl { } } +#[macro_export] +macro_rules! frontend_rpc_client_method_impl { + ($( { $client:tt, $fn_name:ident, $req:ty, $resp:ty }),*) => { + $( + pub async fn $fn_name(&self, request: $req) -> $crate::Result<$resp> { + Ok(self + .$client + .to_owned() + .$fn_name(request) + .await + .map_err($crate::error::RpcError::from_frontend_status)? + .into_inner()) + } + )* + } +} + pub const DEFAULT_BUFFER_SIZE: usize = 16; pub struct BidiStreamSender { diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index e785f5409f0dd..b7fea536f9053 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -47,11 +47,8 @@ pub type SessionId = (ProcessId, SecretKey); pub trait SessionManager: Send + Sync + 'static { type Session: Session; - fn get_session( - &self, - database_id: u32, - user_name: &str, - ) -> Result, BoxedError>; + fn get_session(&self, database_id: u32, user_id: u32) + -> Result, BoxedError>; fn connect( &self, @@ -389,7 +386,7 @@ mod tests { fn get_session( &self, database_id: u32, - user_name: &str, + user_name: u32, ) -> Result, BoxedError> { todo!() } From f1fff64720e63d37f63b87f790bd8b54d90d7cb9 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 14:03:55 +0800 Subject: [PATCH 23/75] wip: rpc source parser->meta --- proto/ddl_service.proto | 6 +++- .../src/parser/debezium/schema_change.rs | 14 +++++++-- src/connector/src/parser/unified/debezium.rs | 2 +- .../src/schema/auto_schema_change/client.rs | 31 +++++++++++++++++++ .../src/schema/auto_schema_change/mod.rs | 12 +------ src/meta/service/src/ddl_service.rs | 16 +++++----- src/rpc_client/src/meta_client.rs | 9 ++++++ 7 files changed, 67 insertions(+), 23 deletions(-) create mode 100644 src/connector/src/schema/auto_schema_change/client.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 0b813ec7197c9..29a2d910f47e1 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -445,11 +445,15 @@ message CommentOnResponse { uint64 version = 2; } -message SchemaChangeEnvelope { +message TableSchemaChange { string cdc_table_name = 1; repeated plan_common.ColumnCatalog column_descs = 2; } +message SchemaChangeEnvelope { + repeated TableSchemaChange table_changes = 1; +} + message AutoSchemaChangeRequest { SchemaChangeEnvelope schema_change = 1; } diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 958748d04bb31..66258c4a41ef5 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -14,6 +14,8 @@ use risingwave_common::catalog::ColumnCatalog; +use risingwave_pb::ddl_service::SchemaChangeEnvelope as PbSchemaChangeEnvelope; + #[derive(Debug)] pub struct SchemaChangeEnvelope { pub table_changes: Vec, @@ -21,12 +23,18 @@ pub struct SchemaChangeEnvelope { #[derive(Debug)] pub struct TableSchemaChange { - pub(crate) up_table_full_name: String, + pub(crate) cdc_table_name: String, pub(crate) columns: Vec, } impl SchemaChangeEnvelope { - pub fn to_protobuf(&self) { - todo!() + pub fn to_protobuf(&self) -> Vec { + + PbSchemaChangeEnvelope { + cdc_table_name: self.c + column_descs: vec![], + } + + } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index af74a80fc13ab..9ce8ae993fa24 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -198,7 +198,7 @@ pub fn parse_schema_change( } } schema_changes.push(TableSchemaChange { - up_table_full_name: id, + cdc_table_name: id, columns: column_descs .into_iter() .map(|column_desc| ColumnCatalog { diff --git a/src/connector/src/schema/auto_schema_change/client.rs b/src/connector/src/schema/auto_schema_change/client.rs new file mode 100644 index 0000000000000..ac7afe0bd53e5 --- /dev/null +++ b/src/connector/src/schema/auto_schema_change/client.rs @@ -0,0 +1,31 @@ +// 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 risingwave_rpc_client::MetaClient; + +use crate::parser::schema_change::SchemaChangeEnvelope; + +/// client for auto schema change +/// we may collect some metrics here +pub struct AutoSchemaChangeClient { + meta_client: MetaClient, +} + +impl AutoSchemaChangeClient { + pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { + // TODO: + + Ok(()) + } +} diff --git a/src/connector/src/schema/auto_schema_change/mod.rs b/src/connector/src/schema/auto_schema_change/mod.rs index 44b9c88f9e347..f2b2f45ae6a2e 100644 --- a/src/connector/src/schema/auto_schema_change/mod.rs +++ b/src/connector/src/schema/auto_schema_change/mod.rs @@ -12,14 +12,4 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::parser::schema_change::SchemaChangeEnvelope; - -/// client for auto schema change -/// Can be a global client -pub struct AutoSchemaChangeClient {} - -impl AutoSchemaChangeClient { - pub fn submit_schema_change(&self, schema_change: SchemaChangeEnvelope) -> anyhow::Result<()> { - Ok(()) - } -} +mod client; diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 59294e2ffdb56..da8ba26f5d613 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -922,6 +922,8 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(AlterParallelismResponse {})) } + /// Auto schema change for cdc sources, + /// called by the source parser when a schema change is detected. async fn auto_schema_change( &self, request: Request, @@ -931,13 +933,6 @@ impl DdlService for DdlServiceImpl { let schema_change = req.schema_change.unwrap(); let cdc_table_name = schema_change.cdc_table_name.clone(); - // get the table catalog corresponding to the - let tables: Vec
= self - .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) - .await?; - - // send a request to the frontend to get the ReplaceTablePlan let mut workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) @@ -948,7 +943,14 @@ impl DdlService for DdlServiceImpl { .ok_or_else(|| anyhow!("no frontend worker available"))?; let client = self.env.frontend_client_pool().get(worker).await?; + // get the table catalog corresponding to the + let tables: Vec
= self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) + .await?; + for table in tables { + // send a request to the frontend to get the ReplaceTablePlan let resp = client .get_table_replace_plan(GetTableReplacePlanRequest { database_id: table.database_id, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 25cf2222b4407..703783e3da2b6 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -551,6 +551,14 @@ impl MetaClient { Ok(resp.version) } + pub async fn auto_schema_change(&self, schema_change: SchemaChangeEnvelope) -> Result<()> { + let request = AutoSchemaChangeRequest { + schema_change: Some(schema_change), + }; + let _ = self.inner.auto_schema_change(request).await?; + Ok(()) + } + pub async fn create_view(&self, view: PbView) -> Result { let request = CreateViewRequest { view: Some(view) }; let resp = self.inner.create_view(request).await?; @@ -2021,6 +2029,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, comment_on, CommentOnRequest, CommentOnResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } ,{ ddl_client, wait, WaitRequest, WaitResponse } + ,{ ddl_client, auto_schema_change, AutoSchemaChangeRequest, AutoSchemaChangeResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } From 5d97d734bc2db0407f98ff9113714df053e1fc4c Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 15:47:28 +0800 Subject: [PATCH 24/75] refine source -> meta -> frontend rpc --- proto/ddl_service.proto | 2 +- proto/frontend_service.proto | 5 +- .../src/parser/debezium/schema_change.rs | 31 ++++++--- src/frontend/src/rpc/mod.rs | 16 ++--- src/meta/service/src/ddl_service.rs | 66 +++++++++++-------- .../src/delete_range_runner.rs | 1 + 6 files changed, 72 insertions(+), 49 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 29a2d910f47e1..091daa9bb6869 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -447,7 +447,7 @@ message CommentOnResponse { message TableSchemaChange { string cdc_table_name = 1; - repeated plan_common.ColumnCatalog column_descs = 2; + repeated plan_common.ColumnCatalog columns = 2; } message SchemaChangeEnvelope { diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 247c580cb392e..8fb49e1bedb3a 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -12,12 +12,11 @@ message GetTableReplacePlanRequest { uint32 database_id = 1; uint32 owner = 2; string table_name = 3; - ddl_service.SchemaChangeEnvelope schema_change = 4; + ddl_service.TableSchemaChange table_change = 4; } - message GetTableReplacePlanResponse { - ddl_service.ReplaceTablePlan table_plan = 1; + ddl_service.ReplaceTablePlan replace_plan = 1; } // Frontend will start a background worker to diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 66258c4a41ef5..07efd099462b3 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -13,8 +13,9 @@ // limitations under the License. use risingwave_common::catalog::ColumnCatalog; - -use risingwave_pb::ddl_service::SchemaChangeEnvelope as PbSchemaChangeEnvelope; +use risingwave_pb::ddl_service::{ + SchemaChangeEnvelope as PbSchemaChangeEnvelope, TableSchemaChange as PbTableSchemaChange, +}; #[derive(Debug)] pub struct SchemaChangeEnvelope { @@ -28,13 +29,23 @@ pub struct TableSchemaChange { } impl SchemaChangeEnvelope { - pub fn to_protobuf(&self) -> Vec { - - PbSchemaChangeEnvelope { - cdc_table_name: self.c - column_descs: vec![], - } - - + pub fn to_protobuf(&self) -> PbSchemaChangeEnvelope { + let table_changes = self + .table_changes + .iter() + .map(|table_change| { + let columns = table_change + .columns + .iter() + .map(|column| column.to_protobuf()) + .collect(); + PbTableSchemaChange { + cdc_table_name: table_change.cdc_table_name.clone(), + columns, + } + }) + .collect(); + + PbSchemaChangeEnvelope { table_changes } } } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index 8651a19165db6..784c46a479b9d 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. use pgwire::pg_server::{BoxedError, SessionManager}; -use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope}; +use risingwave_pb::ddl_service::{ReplaceTablePlan, SchemaChangeEnvelope, TableSchemaChange}; use risingwave_pb::frontend_service::frontend_service_server::FrontendService; use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; use risingwave_rpc_client::error::ToTonicStatus; @@ -62,20 +62,18 @@ impl FrontendService for FrontendServiceImpl { request: RpcRequest, ) -> Result, Status> { let req = request.into_inner(); - - let change = req - .schema_change - .expect("schema change message is required"); + let table_change = req.table_change.expect("schema change message is required"); let replace_plan = - get_new_table_plan(change, req.table_name, req.database_id, req.owner).await?; + get_new_table_plan(table_change, req.table_name, req.database_id, req.owner).await?; + Ok(RpcResponse::new(GetTableReplacePlanResponse { - table_plan: Some(replace_plan), + replace_plan: Some(replace_plan), })) } } async fn get_new_table_plan( - change: SchemaChangeEnvelope, + table_change: TableSchemaChange, table_name: String, database_id: u32, owner: u32, @@ -88,7 +86,7 @@ async fn get_new_table_plan( let session = session_mgr.get_session(database_id, owner)?; // call the handle alter method - let new_columns = change.column_descs.into_iter().map(|c| c.into()).collect(); + let new_columns = table_change.columns.into_iter().map(|c| c.into()).collect(); let table_name = ObjectName::from(vec![table_name.as_str().into()]); let (new_table_definition, original_catalog) = get_new_table_definition_for_cdc_table(&session, table_name.clone(), new_columns).await?; diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index da8ba26f5d613..69b443a8e73d7 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -20,6 +20,7 @@ use rand::seq::SliceRandom; use rand::thread_rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_connector::sink::catalog::SinkId; +use risingwave_meta::error::MetaErrorInner; use risingwave_meta::manager::MetadataManager; use risingwave_meta::rpc::ddl_controller::fill_table_stream_graph_info; use risingwave_pb::catalog::connection::private_link_service::{ @@ -930,9 +931,6 @@ impl DdlService for DdlServiceImpl { ) -> Result, Status> { let req = request.into_inner(); - let schema_change = req.schema_change.unwrap(); - let cdc_table_name = schema_change.cdc_table_name.clone(); - let mut workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) @@ -940,33 +938,49 @@ impl DdlService for DdlServiceImpl { workers.shuffle(&mut thread_rng()); let worker = workers .first() - .ok_or_else(|| anyhow!("no frontend worker available"))?; - let client = self.env.frontend_client_pool().get(worker).await?; + .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; + let client = self + .env + .frontend_client_pool() + .get(worker) + .await + .map_err(|err| MetaError::from(err))?; + + let Some(schema_change) = req.schema_change else { + return Err(Status::invalid_argument( + "schema change message is required", + )); + }; - // get the table catalog corresponding to the - let tables: Vec
= self - .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) - .await?; + for table_change in schema_change.table_changes { + let cdc_table_name = table_change.cdc_table_name.clone(); - for table in tables { - // send a request to the frontend to get the ReplaceTablePlan - let resp = client - .get_table_replace_plan(GetTableReplacePlanRequest { - database_id: table.database_id, - owner: table.owner, - table_name: table.name, - schema_change: Some(schema_change), - }) + // get the table catalog corresponding to the cdc table + let tables: Vec
= self + .metadata_manager + .get_table_catalog_by_cdc_table_name(cdc_table_name) .await?; - if let Some(plan) = resp.table_plan { - // start the schema change procedure - self.ddl_controller - .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( - plan, - ))) - .await?; + for table in tables { + // send a request to the frontend to get the ReplaceTablePlan + let resp = client + .get_table_replace_plan(GetTableReplacePlanRequest { + database_id: table.database_id, + owner: table.owner, + table_name: table.name, + table_change: Some(table_change.clone()), + }) + .await + .map_err(|err| MetaError::from(err))?; + + if let Some(plan) = resp.replace_plan { + // start the schema change procedure + self.ddl_controller + .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( + plan, + ))) + .await?; + } } } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 5d6c6ff7e70d1..9f46e6be5d6fe 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -158,6 +158,7 @@ async fn compaction_test( incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, + cdc_table_name: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; From ead027ba167a712e748192469126eeafb3afb380 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 24 Jul 2024 19:26:03 +0800 Subject: [PATCH 25/75] submit schema change to spawned task in source exec --- src/batch/src/executor/source.rs | 1 + src/connector/src/parser/mod.rs | 5 ++++ src/connector/src/source/base.rs | 10 +++++++ src/connector/src/source/mod.rs | 1 + src/stream/src/executor/actor.rs | 7 +++++ .../src/executor/source/fetch_executor.rs | 1 + .../src/executor/source/fs_source_executor.rs | 1 + .../source/source_backfill_executor.rs | 1 + .../src/executor/source/source_executor.rs | 30 ++++++++++++++++++- src/stream/src/task/stream_manager.rs | 1 + 10 files changed, 57 insertions(+), 1 deletion(-) diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 51d8da9d14d9c..655030e3d510d 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -175,6 +175,7 @@ impl SourceExecutor { rate_limit: None, }, ConnectorProperties::default(), + None, )); let stream = self .source diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index f521032dfe993..50d52d519b668 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -841,6 +841,11 @@ async fn into_chunk_stream_inner( // 1. block source executor // 2. send schema change to Meta // 3. wait for Meta to finish schema change + let (oneshot_tx, oneshot_rx) = tokio::sync::oneshot::channel(); + if let Some(ref tx) = parser.source_ctx().schema_change_tx { + tx.send((schema_change, oneshot_tx)).await.unwrap(); + oneshot_rx.await.unwrap(); + } } } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 4c44f9610bd17..598d59c1f9436 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -32,6 +32,7 @@ use risingwave_pb::catalog::{PbSource, PbStreamSourceInfo}; use risingwave_pb::plan_common::ExternalTableDesc; use risingwave_pb::source::ConnectorSplit; use serde::de::DeserializeOwned; +use tokio::sync::mpsc; use super::cdc::DebeziumCdcMeta; use super::datagen::DatagenMeta; @@ -42,6 +43,7 @@ use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; use crate::error::ConnectorResult as Result; +use crate::parser::schema_change::SchemaChangeEnvelope; use crate::parser::ParserConfig; use crate::source::filesystem::FsPageItem; use crate::source::monitor::EnumeratorMetrics; @@ -178,6 +180,9 @@ pub struct SourceContext { pub metrics: Arc, pub source_ctrl_opts: SourceCtrlOpts, pub connector_props: ConnectorProperties, + // source parser put schema change event into this channel + pub schema_change_tx: + Option)>>, } impl SourceContext { @@ -189,6 +194,9 @@ impl SourceContext { metrics: Arc, source_ctrl_opts: SourceCtrlOpts, connector_props: ConnectorProperties, + schema_change_channel: Option< + mpsc::Sender<(SchemaChangeEnvelope, tokio::sync::oneshot::Sender<()>)>, + >, ) -> Self { Self { actor_id, @@ -198,6 +206,7 @@ impl SourceContext { metrics, source_ctrl_opts, connector_props, + schema_change_tx: schema_change_channel, } } @@ -215,6 +224,7 @@ impl SourceContext { rate_limit: None, }, ConnectorProperties::default(), + None, ) } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ed8842e70825f..46124cca1f84b 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -42,6 +42,7 @@ pub mod test_source; pub use manager::{SourceColumnDesc, SourceColumnType}; use risingwave_common::array::{Array, ArrayRef}; +use risingwave_pb::ddl_service::SchemaChangeEnvelope; use thiserror_ext::AsReport; pub use crate::source::filesystem::opendal_source::{ diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index 1c73a3aeddad6..a94e07949f0f6 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -28,6 +28,7 @@ use risingwave_expr::expr_context::{expr_context_scope, FRAGMENT_ID}; use risingwave_expr::ExprError; use risingwave_pb::plan_common::ExprContext; use risingwave_pb::stream_plan::PbStreamActor; +use risingwave_rpc_client::MetaClient; use thiserror_ext::AsReport; use tokio_stream::StreamExt; use tracing::Instrument; @@ -55,6 +56,9 @@ pub struct ActorContext { pub initial_dispatch_num: usize, // mv_table_id to subscription id pub related_subscriptions: HashMap>, + + // Meta client. currently used for auto schema change + pub meta_client: Option, } pub type ActorContextRef = Arc; @@ -72,6 +76,7 @@ impl ActorContext { // Set 1 for test to enable sanity check on table initial_dispatch_num: 1, related_subscriptions: HashMap::new(), + meta_client: None, }) } @@ -81,6 +86,7 @@ impl ActorContext { streaming_metrics: Arc, initial_dispatch_num: usize, related_subscriptions: HashMap>, + meta_client: Option, ) -> ActorContextRef { Arc::new(Self { id: stream_actor.actor_id, @@ -92,6 +98,7 @@ impl ActorContext { streaming_metrics, initial_dispatch_num, related_subscriptions, + meta_client, }) } diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 788a9a45662cd..13580ca49e001 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -177,6 +177,7 @@ impl FsFetchExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ) } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 32d5d533d904a..6754570c4930b 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -108,6 +108,7 @@ impl FsSourceExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 4a34eabe97e16..8351e48023a05 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -216,6 +216,7 @@ impl SourceBackfillExecutorInner { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + None, ); let stream = source_desc .source diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 8e5c3f9726c28..9ad5359b01aa5 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -24,6 +24,7 @@ use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::{Epoch, EpochPair}; +use risingwave_connector::parser::schema_change::SchemaChangeEnvelope; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::reader::reader::SourceReader; use risingwave_connector::source::{ @@ -32,8 +33,8 @@ use risingwave_connector::source::{ }; use risingwave_hummock_sdk::HummockReadEpoch; use thiserror_ext::AsReport; -use tokio::sync::mpsc; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; +use tokio::sync::{mpsc, oneshot}; use tokio::time::Instant; use super::executor_core::StreamSourceCore; @@ -122,6 +123,32 @@ impl SourceExecutor { .iter() .map(|column_desc| column_desc.column_id) .collect_vec(); + + let (schema_change_tx, rx) = + tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(32); + let meta_client = self.actor_ctx.meta_client.clone(); + let _ = tokio::task::spawn(async move { + let mut schema_change_rx = rx; + while let Some((schema_change, parser_tx)) = schema_change_rx.recv().await { + // handle schema change + if let Some(ref meta_client) = meta_client { + match meta_client + .auto_schema_change(schema_change.to_protobuf()) + .await + { + Ok(_) => { + tracing::info!("schema change success"); + parser_tx.send(()).unwrap(); + } + Err(e) => { + tracing::error!(error = ?e.as_report(), "schema change error"); + parser_tx.send(()).unwrap(); + } + } + } + } + }); + let source_ctx = SourceContext::new( self.actor_ctx.id, self.stream_source_core.as_ref().unwrap().source_id, @@ -137,6 +164,7 @@ impl SourceExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), + Some(schema_change_tx), ); let stream = source_desc .source diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index e7105347c9356..5ed64249bc350 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -579,6 +579,7 @@ impl StreamActorManager { ) }) .collect(), + self.env.meta_client().clone(), ); let vnode_bitmap = actor.vnode_bitmap.as_ref().map(|b| b.into()); let expr_context = actor.expr_context.clone().unwrap(); From 129537451241a677bcbd3053513d283197010f8f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 27 Jul 2024 22:41:29 +0800 Subject: [PATCH 26/75] wip: debuging rpc --- .../source/core/DbzChangeEventConsumer.java | 3 +- proto/connector_service.proto | 1 + src/connector/src/parser/plain_parser.rs | 3 +- .../src/source/cdc/source/message.rs | 2 ++ src/frontend/src/rpc/mod.rs | 2 ++ src/meta/model_v2/migration/src/lib.rs | 2 ++ .../m20240726_063833_auto_schema_change.rs | 35 +++++++++++++++++++ src/meta/service/src/ddl_service.rs | 14 +++++++- src/meta/src/barrier/command.rs | 2 ++ src/storage/src/monitor/local_metrics.rs | 2 +- .../src/executor/source/source_executor.rs | 2 ++ 11 files changed, 64 insertions(+), 4 deletions(-) create mode 100644 src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index d3343b8ae2a8b..7537468bc2af0 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -167,7 +167,8 @@ var record = event.value(); switch (eventType) { case HEARTBEAT: { - var message = msgBuilder.build(); + var message = + msgBuilder.setMsgType(CdcMessage.CdcMessageType.HEARTBEAT).build(); LOG.debug("heartbeat => {}", message.getOffset()); respBuilder.addEvents(message); break; diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 13cf2b2f68cbc..c6824a53626b2 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -152,6 +152,7 @@ message CdcMessage { DATA = 1; TRANSACTION_META = 2; SCHEMA_CHANGE = 3; + HEARTBEAT = 4; } // The value of the Debezium message diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index c1a7036bd342f..84a80fdad5b65 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -101,7 +101,7 @@ impl PlainParser { && let Some(data) = payload { match cdc_meta.msg_type { - CdcMessageType::Data => { + CdcMessageType::Data | CdcMessageType::Heartbeat => { return self.parse_rows(key, Some(data), writer).await; } CdcMessageType::TransactionMeta => { @@ -120,6 +120,7 @@ impl PlainParser { }; } CdcMessageType::SchemaChange => { + tracing::info!("got schema change message"); let accessor = self .schema_change_builder .as_mut() diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index 4c79257adb71d..b610462812a28 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -24,6 +24,7 @@ pub enum CdcMessageType { Data, TransactionMeta, SchemaChange, + Heartbeat, } impl From for CdcMessageType { @@ -32,6 +33,7 @@ impl From for CdcMessageType { cdc_message::CdcMessageType::Data => CdcMessageType::Data, cdc_message::CdcMessageType::TransactionMeta => CdcMessageType::TransactionMeta, cdc_message::CdcMessageType::SchemaChange => CdcMessageType::SchemaChange, + cdc_message::CdcMessageType::Heartbeat => CdcMessageType::Heartbeat, cdc_message::CdcMessageType::Unspecified => CdcMessageType::Unknown, } } diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index 784c46a479b9d..2e4d44f651f99 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -62,6 +62,8 @@ impl FrontendService for FrontendServiceImpl { request: RpcRequest, ) -> Result, Status> { let req = request.into_inner(); + tracing::info!("get_table_replace_plan for table {}", req.table_name); + let table_change = req.table_change.expect("schema change message is required"); let replace_plan = get_new_table_plan(table_change, req.table_name, req.database_id, req.owner).await?; diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 770d83bdabea9..a2b1025bb6a5b 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -17,6 +17,7 @@ mod m20240630_131430_remove_parallel_unit; mod m20240701_060504_hummock_time_travel; mod m20240702_080451_system_param_value; mod m20240702_084927_unnecessary_fk; +mod m20240726_063833_auto_schema_change; pub struct Migrator; @@ -39,6 +40,7 @@ impl MigratorTrait for Migrator { Box::new(m20240702_080451_system_param_value::Migration), Box::new(m20240702_084927_unnecessary_fk::Migration), Box::new(m20240701_060504_hummock_time_travel::Migration), + Box::new(m20240726_063833_auto_schema_change::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs b/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs new file mode 100644 index 0000000000000..a828360d08d14 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs @@ -0,0 +1,35 @@ +use sea_orm_migration::prelude::{Table as MigrationTable, *}; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Table::Table) + .add_column(ColumnDef::new(Table::CdcTableName).string()) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Table::Table) + .drop_column(Table::CdcTableName) + .to_owned(), + ) + .await + } +} + +#[derive(DeriveIden)] +enum Table { + Table, + CdcTableName, +} diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 69b443a8e73d7..248c067407eea 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -939,6 +939,8 @@ impl DdlService for DdlServiceImpl { let worker = workers .first() .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; + + tracing::info!("get client for frontend {:?}", worker); let client = self .env .frontend_client_pool() @@ -955,11 +957,14 @@ impl DdlService for DdlServiceImpl { for table_change in schema_change.table_changes { let cdc_table_name = table_change.cdc_table_name.clone(); + tracing::info!("auto schema change cdc table: {}", cdc_table_name); + // get the table catalog corresponding to the cdc table let tables: Vec
= self .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) + .get_table_catalog_by_cdc_table_name(cdc_table_name.replace("\"", "")) .await?; + tracing::info!("number of table to replace: {}", tables.len()); for table in tables { // send a request to the frontend to get the ReplaceTablePlan @@ -973,6 +978,12 @@ impl DdlService for DdlServiceImpl { .await .map_err(|err| MetaError::from(err))?; + if let Some(plan) = resp.replace_plan.as_ref() { + plan.table + .as_ref() + .inspect(|t| tracing::info!("Table to replace: {}", t.name)); + } + if let Some(plan) = resp.replace_plan { // start the schema change procedure self.ddl_controller @@ -980,6 +991,7 @@ impl DdlService for DdlServiceImpl { plan, ))) .await?; + tracing::info!("replace table {} success", table.id); } } } diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 2c65f467b4a98..760177bb73eab 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -848,6 +848,8 @@ impl CommandContext { pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { let futures = self.info.node_map.values().map(|worker_node| async { + let worker = worker_node.clone(); + tracing::info!("get client for compute {:?}", worker); let client = self .barrier_manager_context .env diff --git a/src/storage/src/monitor/local_metrics.rs b/src/storage/src/monitor/local_metrics.rs index 5fd7fe6c0ef56..e4fd3fa1571b5 100644 --- a/src/storage/src/monitor/local_metrics.rs +++ b/src/storage/src/monitor/local_metrics.rs @@ -210,7 +210,7 @@ impl Drop for StoreLocalStatistic { && !self.added.load(Ordering::Relaxed) && self.need_report() { - tracing::error!("local stats lost!\n{:#?}", self); + // tracing::error!("local stats lost!\n{:#?}", self); } } } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 9ad5359b01aa5..14d23c26d277b 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -130,6 +130,8 @@ impl SourceExecutor { let _ = tokio::task::spawn(async move { let mut schema_change_rx = rx; while let Some((schema_change, parser_tx)) = schema_change_rx.recv().await { + tracing::info!("recv a schema change envelope"); + // handle schema change if let Some(ref meta_client) = meta_client { match meta_client From 82815d18a8b94e00ebfdf582d7b980efc97a5702 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 28 Jul 2024 20:35:10 +0800 Subject: [PATCH 27/75] finish auto replace workflow --- proto/ddl_service.proto | 11 ++++-- .../src/parser/debezium/schema_change.rs | 34 +++++++++++++++++++ src/connector/src/parser/unified/debezium.rs | 7 ++++ .../src/handler/alter_table_column.rs | 2 +- src/frontend/src/session.rs | 7 ++-- src/meta/service/src/ddl_service.rs | 8 ++--- 6 files changed, 59 insertions(+), 10 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 091daa9bb6869..e73418ab5389f 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -445,9 +445,16 @@ message CommentOnResponse { uint64 version = 2; } + message TableSchemaChange { - string cdc_table_name = 1; - repeated plan_common.ColumnCatalog columns = 2; + enum TableChangeType { + UNSPECIFIED = 0; + ALTER = 1; + } + + TableChangeType change_type = 1; + string cdc_table_name = 2; + repeated plan_common.ColumnCatalog columns = 3; } message SchemaChangeEnvelope { diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 07efd099462b3..13f68e88dfbb4 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::catalog::ColumnCatalog; +use risingwave_pb::ddl_service::table_schema_change::TableChangeType as PbTableChangeType; use risingwave_pb::ddl_service::{ SchemaChangeEnvelope as PbSchemaChangeEnvelope, TableSchemaChange as PbTableSchemaChange, }; @@ -22,10 +23,42 @@ pub struct SchemaChangeEnvelope { pub table_changes: Vec, } +#[derive(Debug, Clone, Copy)] +pub(crate) enum TableChangeType { + Unspecified, + Alter, +} + +impl TableChangeType { + pub fn from_proto(value: PbTableChangeType) -> Self { + match value { + PbTableChangeType::Alter => TableChangeType::Alter, + PbTableChangeType::Unspecified => TableChangeType::Unspecified, + } + } + + pub fn to_proto(self) -> PbTableChangeType { + match self { + TableChangeType::Alter => PbTableChangeType::Alter, + TableChangeType::Unspecified => PbTableChangeType::Unspecified, + } + } +} + +impl From<&str> for TableChangeType { + fn from(value: &str) -> Self { + match value { + "ALTER" => TableChangeType::Alter, + _ => TableChangeType::Unspecified, + } + } +} + #[derive(Debug)] pub struct TableSchemaChange { pub(crate) cdc_table_name: String, pub(crate) columns: Vec, + pub(crate) change_type: TableChangeType, } impl SchemaChangeEnvelope { @@ -40,6 +73,7 @@ impl SchemaChangeEnvelope { .map(|column| column.to_protobuf()) .collect(); PbTableSchemaChange { + change_type: table_change.change_type.to_proto() as _, cdc_table_name: table_change.cdc_table_name.clone(), columns, } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 9ce8ae993fa24..85ce91931c89d 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -159,6 +159,12 @@ pub fn parse_schema_change( .as_string() .unwrap(); + let ty = jsonb + .access_object_field("type") + .unwrap() + .as_string() + .unwrap(); + println!("id: {}", id); let mut column_descs: Vec = vec![]; @@ -206,6 +212,7 @@ pub fn parse_schema_change( is_hidden: false, }) .collect_vec(), + change_type: ty.as_str().into(), }); } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index bccf9e9797521..ee65c32661a88 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -98,7 +98,6 @@ pub async fn get_new_table_definition_for_cdc_table( } // since the DDL is committed on upstream, so we can safely replace the original columns with new columns - // replace original columns with new columns let mut new_column_defs = vec![]; for col in new_columns.into_iter() { let ty = to_ast_data_type(col.data_type())?; @@ -118,6 +117,7 @@ fn to_ast_data_type(ty: &DataType) -> Result { DataType::Float32 => Ok(AstDataType::Real), DataType::Float64 => Ok(AstDataType::Double), DataType::Date => Ok(AstDataType::Date), + DataType::Varchar => Ok(AstDataType::Varchar), DataType::Time => Ok(AstDataType::Time(false)), DataType::Timestamp => Ok(AstDataType::Timestamp(false)), DataType::Timestamptz => Ok(AstDataType::Timestamp(true)), diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 794471fc7d74b..5c9776e298d3e 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -261,11 +261,12 @@ impl FrontendEnv { .unwrap(); info!("advertise addr is {}", frontend_address); + let frontend_rpc_addr = opts.health_check_listener_addr.parse().unwrap(); // Register in meta by calling `AddWorkerNode` RPC. let (meta_client, system_params_reader) = MetaClient::register_new( opts.meta_addr, WorkerType::Frontend, - &frontend_address, + &frontend_rpc_addr, Default::default(), &config.meta, ) @@ -355,7 +356,7 @@ impl FrontendEnv { let health_srv = HealthServiceImpl::new(); let frontend_srv = FrontendServiceImpl::new(); - let host = opts.health_check_listener_addr.clone(); + let frontend_rpc_addr = opts.health_check_listener_addr.parse().unwrap(); let telemetry_manager = TelemetryManager::new( Arc::new(meta_client.clone()), @@ -376,7 +377,7 @@ impl FrontendEnv { tonic::transport::Server::builder() .add_service(HealthServer::new(health_srv)) .add_service(FrontendServiceServer::new(frontend_srv)) - .serve(host.parse().unwrap()) + .serve(frontend_rpc_addr) .await .unwrap(); }); diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 248c067407eea..a59890357ae93 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -940,7 +940,7 @@ impl DdlService for DdlServiceImpl { .first() .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; - tracing::info!("get client for frontend {:?}", worker); + tracing::info!(">> get client for frontend {:?}", worker); let client = self .env .frontend_client_pool() @@ -957,14 +957,14 @@ impl DdlService for DdlServiceImpl { for table_change in schema_change.table_changes { let cdc_table_name = table_change.cdc_table_name.clone(); - tracing::info!("auto schema change cdc table: {}", cdc_table_name); + tracing::info!(">> auto schema change cdc table: {}", cdc_table_name); // get the table catalog corresponding to the cdc table let tables: Vec
= self .metadata_manager .get_table_catalog_by_cdc_table_name(cdc_table_name.replace("\"", "")) .await?; - tracing::info!("number of table to replace: {}", tables.len()); + tracing::info!(">> number of table to replace: {}", tables.len()); for table in tables { // send a request to the frontend to get the ReplaceTablePlan @@ -991,7 +991,7 @@ impl DdlService for DdlServiceImpl { plan, ))) .await?; - tracing::info!("replace table {} success", table.id); + tracing::info!(">> replace table {} success", table.id); } } } From 7be0555ecc02fb1f1f4630667c52665e2448a6ef Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 12:38:30 +0800 Subject: [PATCH 28/75] register frontend rpc addr to meta --- src/frontend/src/session.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 5c9776e298d3e..df1d7963d89f1 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -344,7 +344,8 @@ impl FrontendEnv { let observer_join_handle = observer_manager.start().await; join_handles.push(observer_join_handle); - meta_client.activate(&frontend_address).await?; + // meta_client.activate(&frontend_address).await?; + meta_client.activate(&frontend_rpc_addr).await?; let frontend_metrics = Arc::new(GLOBAL_FRONTEND_METRICS.clone()); let source_metrics = Arc::new(GLOBAL_SOURCE_METRICS.clone()); From 78088c9017080840f3617c42d30a094f9c4ef678 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 15:26:58 +0800 Subject: [PATCH 29/75] add e2e test --- .../cdc_inline/auto_schema_change_mysql.slt | 73 +++++++++++++++++++ src/connector/src/parser/mysql.rs | 2 +- 2 files changed, 74 insertions(+), 1 deletion(-) create mode 100644 e2e_test/source/cdc_inline/auto_schema_change_mysql.slt diff --git a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt new file mode 100644 index 0000000000000..ad2f86f496755 --- /dev/null +++ b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt @@ -0,0 +1,73 @@ +control substitution on + +system ok +mysql -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE mytest;" + +system ok +mysql -e " + USE mytest; + DROP TABLE IF EXISTS customers; + CREATE TABLE customers( + id BIGINT PRIMARY KEY, + modified DATETIME, + custinfo JSON + ); + ALTER TABLE customers ADD INDEX zipsa( (CAST(custinfo->'zipcode' AS UNSIGNED ARRAY)) ); + " + +statement ok +create source mysql_source with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'root', + password = '${MYSQL_PWD:}', + database.name = 'mytest', + server.id = '5701' +); + +statement ok +create table rw_customers (id bigint, modified timestamp, custinfo jsonb, primary key (id)) from mysql_source table 'mytest.customers'; + +# Name, Type, Is Hidden, Description +query TTTT +describe rw_customers; +---- +id bigint false NULL +modified timestamp without time zone false NULL +custinfo jsonb false NULL +primary key id NULL NULL +distribution key id NULL NULL +table description rw_customers NULL NULL + + +system ok +mysql -e " + USE mytest; + ALTER TABLE customers ADD COLUMN v1 VARCHAR(255); +" + +system ok +mysql -e " + USE mytest; + ALTER TABLE customers ADD COLUMN v2 double(5,2); +" + +sleep 3s + +# Name, Type, Is Hidden, Description +query TTTT +describe rw_customers; +---- +id bigint false NULL +modified timestamp without time zone false NULL +custinfo jsonb false NULL +v1 character varying false NULL +v2 double precision false NULL +primary key id NULL NULL +distribution key id NULL NULL +table description rw_customers NULL NULL + + +statement ok +drop source mysql_source cascade; diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index f27c7edebb5e8..453ab413f88e0 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -153,7 +153,7 @@ pub fn mysql_typename_to_rw_type(type_name: &str) -> anyhow::Result { "date" => Ok(DataType::Date), "time" => Ok(DataType::Time), "timestamp" => Ok(DataType::Timestamptz), - "datetime" => Ok(DataType::Timestamptz), + "datetime" => Ok(DataType::Timestamp), "json" => Ok(DataType::Jsonb), "binary" | "varbinary" | "blob" | "mediumblob" | "longblob" => Ok(DataType::Bytea), _ => Err(anyhow::anyhow!("unsupported type: {}", type_name)), From a2b2a2b8ed63a0d1db00811b42eb5752f9516b47 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 16:02:40 +0800 Subject: [PATCH 30/75] minor --- e2e_test/source/cdc_inline/auto_schema_change_mysql.slt | 5 ----- 1 file changed, 5 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt index ad2f86f496755..4d6f94eb4340a 100644 --- a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt @@ -45,11 +45,6 @@ system ok mysql -e " USE mytest; ALTER TABLE customers ADD COLUMN v1 VARCHAR(255); -" - -system ok -mysql -e " - USE mytest; ALTER TABLE customers ADD COLUMN v2 double(5,2); " From bd3cbafbdf89e0e6237b4db02ab804528466215b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 16:54:47 +0800 Subject: [PATCH 31/75] minor --- src/meta/service/src/ddl_service.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index a59890357ae93..5c3b6774bbf20 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -962,7 +962,7 @@ impl DdlService for DdlServiceImpl { // get the table catalog corresponding to the cdc table let tables: Vec
= self .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name.replace("\"", "")) + .get_table_catalog_by_cdc_table_name(cdc_table_name) .await?; tracing::info!(">> number of table to replace: {}", tables.len()); From 73878f4b9e87e6c050906fd997cfc79b06cbc213 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 17:03:31 +0800 Subject: [PATCH 32/75] refine --- .../src/parser/debezium/schema_change.rs | 2 +- src/connector/src/parser/mysql.rs | 2 +- src/connector/src/parser/unified/debezium.rs | 36 +++++++------------ 3 files changed, 15 insertions(+), 25 deletions(-) diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 0881cd26cc503..2075745dd110c 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -7,6 +7,6 @@ pub struct SchemaChangeEnvelope { #[derive(Debug)] pub struct TableSchemaChange { - pub(crate) up_table_full_name: String, + pub(crate) cdc_table_name: String, pub(crate) columns: Vec, } diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index f27c7edebb5e8..453ab413f88e0 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -153,7 +153,7 @@ pub fn mysql_typename_to_rw_type(type_name: &str) -> anyhow::Result { "date" => Ok(DataType::Date), "time" => Ok(DataType::Time), "timestamp" => Ok(DataType::Timestamptz), - "datetime" => Ok(DataType::Timestamptz), + "datetime" => Ok(DataType::Timestamp), "json" => Ok(DataType::Jsonb), "binary" | "varbinary" | "blob" | "mediumblob" | "longblob" => Ok(DataType::Bytea), _ => Err(anyhow::anyhow!("unsupported type: {}", type_name)), diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index af74a80fc13ab..2c036ff724edd 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -137,6 +137,14 @@ pub fn parse_transaction_meta( }) } +macro_rules! jsonb_access_field { + ($col:expr, $field:expr, $as_type:tt) => { + $crate::paste! { + $col.access_object_field($field).unwrap().[]().unwrap() + } + }; +} + pub fn parse_schema_change( accessor: &impl Access, connector_props: &ConnectorProperties, @@ -153,34 +161,16 @@ pub fn parse_schema_change( _ => unreachable!(""), }; - let id = jsonb - .access_object_field("id") - .unwrap() - .as_string() - .unwrap(); - - println!("id: {}", id); + let id = jsonb_access_field!(jsonb, "id", string); + let ty = jsonb_access_field!(jsonb, "type", string); let mut column_descs: Vec = vec![]; if let Some(table) = jsonb.access_object_field("table") && let Some(columns) = table.access_object_field("columns") { for col in columns.array_elements().unwrap() { - let name = col - .access_object_field("name") - .unwrap() - .as_string() - .unwrap(); - let type_name = col - .access_object_field("typeName") - .unwrap() - .as_string() - .unwrap(); - let position = col - .access_object_field("position") - .unwrap() - .as_number() - .unwrap(); + let name = jsonb_access_field!(col, "name", string); + let type_name = jsonb_access_field!(col, "typeName", string); let data_type = match *connector_props { ConnectorProperties::PostgresCdc(_) => { @@ -198,7 +188,7 @@ pub fn parse_schema_change( } } schema_changes.push(TableSchemaChange { - up_table_full_name: id, + cdc_table_name: id.replace('"', ""), // remove the double quotes columns: column_descs .into_iter() .map(|column_desc| ColumnCatalog { From 09b7979ef301d55f52426e23f7a35ea0e0411e3b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 29 Jul 2024 17:09:27 +0800 Subject: [PATCH 33/75] refine parsing --- src/connector/src/parser/unified/debezium.rs | 42 ++++++-------------- 1 file changed, 13 insertions(+), 29 deletions(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 85ce91931c89d..833401cc82def 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -137,6 +137,14 @@ pub fn parse_transaction_meta( }) } +macro_rules! jsonb_access_field { + ($col:expr, $field:expr, $as_type:tt) => { + $crate::paste! { + $col.access_object_field($field).unwrap().[]().unwrap() + } + }; +} + pub fn parse_schema_change( accessor: &impl Access, connector_props: &ConnectorProperties, @@ -153,40 +161,16 @@ pub fn parse_schema_change( _ => unreachable!(""), }; - let id = jsonb - .access_object_field("id") - .unwrap() - .as_string() - .unwrap(); - - let ty = jsonb - .access_object_field("type") - .unwrap() - .as_string() - .unwrap(); - - println!("id: {}", id); + let id = jsonb_access_field!(jsonb, "id", string); + let ty = jsonb_access_field!(jsonb, "type", string); let mut column_descs: Vec = vec![]; if let Some(table) = jsonb.access_object_field("table") && let Some(columns) = table.access_object_field("columns") { for col in columns.array_elements().unwrap() { - let name = col - .access_object_field("name") - .unwrap() - .as_string() - .unwrap(); - let type_name = col - .access_object_field("typeName") - .unwrap() - .as_string() - .unwrap(); - let position = col - .access_object_field("position") - .unwrap() - .as_number() - .unwrap(); + let name = jsonb_access_field!(col, "name", string); + let type_name = jsonb_access_field!(col, "typeName", string); let data_type = match *connector_props { ConnectorProperties::PostgresCdc(_) => { @@ -204,7 +188,7 @@ pub fn parse_schema_change( } } schema_changes.push(TableSchemaChange { - cdc_table_name: id, + cdc_table_name: id.replace('"', ""), // remove the double quotes columns: column_descs .into_iter() .map(|column_desc| ColumnCatalog { From 950a09df78fe6b12aadefa0d1976ea959727dcc7 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 30 Jul 2024 18:00:29 +0800 Subject: [PATCH 34/75] add --- proto/ddl_service.proto | 16 ++++ .../src/parser/debezium/schema_change.rs | 74 +++++++++++++++++++ src/connector/src/parser/mod.rs | 4 +- src/connector/src/parser/plain_parser.rs | 27 +++++-- src/connector/src/parser/unified/debezium.rs | 5 +- 5 files changed, 114 insertions(+), 12 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 1b4f4e423949e..81f90a03de88d 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -6,6 +6,7 @@ import "catalog.proto"; import "common.proto"; import "meta.proto"; import "stream_plan.proto"; +import "plan_common.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; @@ -444,6 +445,21 @@ message CommentOnResponse { uint64 version = 2; } +message TableSchemaChange { + enum TableChangeType { + UNSPECIFIED = 0; + ALTER = 1; + } + + TableChangeType change_type = 1; + string cdc_table_name = 2; + repeated plan_common.ColumnCatalog columns = 3; +} + +message SchemaChangeEnvelope { + repeated TableSchemaChange table_changes = 1; +} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 2075745dd110c..72fb15273e28c 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -1,12 +1,86 @@ +// 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 risingwave_common::catalog::ColumnCatalog; +use risingwave_pb::ddl_service::table_schema_change::TableChangeType as PbTableChangeType; +use risingwave_pb::ddl_service::{ + SchemaChangeEnvelope as PbSchemaChangeEnvelope, TableSchemaChange as PbTableSchemaChange, +}; #[derive(Debug)] pub struct SchemaChangeEnvelope { pub table_changes: Vec, } +#[derive(Debug, Clone, Copy, PartialEq)] +pub(crate) enum TableChangeType { + Unspecified, + Alter, +} + +impl TableChangeType { + #[allow(dead_code)] + pub fn from_proto(value: PbTableChangeType) -> Self { + match value { + PbTableChangeType::Alter => TableChangeType::Alter, + PbTableChangeType::Unspecified => TableChangeType::Unspecified, + } + } + + pub fn to_proto(self) -> PbTableChangeType { + match self { + TableChangeType::Alter => PbTableChangeType::Alter, + TableChangeType::Unspecified => PbTableChangeType::Unspecified, + } + } +} + +impl From<&str> for TableChangeType { + fn from(value: &str) -> Self { + match value { + "ALTER" => TableChangeType::Alter, + _ => TableChangeType::Unspecified, + } + } +} + #[derive(Debug)] pub struct TableSchemaChange { pub(crate) cdc_table_name: String, pub(crate) columns: Vec, + pub(crate) change_type: TableChangeType, +} + +impl SchemaChangeEnvelope { + pub fn to_protobuf(&self) -> PbSchemaChangeEnvelope { + let table_changes = self + .table_changes + .iter() + .map(|table_change| { + let columns = table_change + .columns + .iter() + .map(|column| column.to_protobuf()) + .collect(); + PbTableSchemaChange { + change_type: table_change.change_type.to_proto() as _, + cdc_table_name: table_change.cdc_table_name.clone(), + columns, + } + }) + .collect(); + + PbSchemaChangeEnvelope { table_changes } + } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 195c285169bfc..887c8b220d604 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -27,9 +27,7 @@ pub use parquet_parser::ParquetParser; pub use protobuf::*; use risingwave_common::array::{ArrayBuilderImpl, Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::{ - ColumnCatalog, KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME, -}; +use risingwave_common::catalog::{KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME}; use risingwave_common::log::LogSuppresser; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::secret::LocalSecretManager; diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index c1a7036bd342f..2d8345b1313c6 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -26,9 +26,7 @@ use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::{parse_schema_change, parse_transaction_meta}; use crate::parser::unified::AccessImpl; use crate::parser::upsert_parser::get_key_column_name; -use crate::parser::{ - BytesProperties, JsonAccessBuilder, JsonProperties, ParseResult, ParserFormat, -}; +use crate::parser::{BytesProperties, ParseResult, ParserFormat}; use crate::source::cdc::CdcMessageType; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMeta}; @@ -212,6 +210,7 @@ mod tests { use risingwave_pb::connector_service::cdc_message; use super::*; + use crate::parser::schema_change::TableChangeType; use crate::parser::{MessageMeta, SourceStreamChunkBuilder, TransactionControl}; use crate::source::cdc::DebeziumCdcMeta; use crate::source::{ConnectorProperties, DataType, SourceMessage, SplitId}; @@ -328,7 +327,9 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - cdc_message::CdcMessageType::TransactionMeta, + transactional + .then(|| cdc_message::CdcMessageType::TransactionMeta) + .unwrap_or(cdc_message::CdcMessageType::Data), )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -356,7 +357,9 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - cdc_message::CdcMessageType::TransactionMeta, + transactional + .then(|| cdc_message::CdcMessageType::TransactionMeta) + .unwrap_or(cdc_message::CdcMessageType::Data), )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -495,6 +498,18 @@ mod tests { ) .await; - res.unwrap(); + let res = res.unwrap(); + match res { + ParseResult::SchemaChange(schema_change) => { + assert_eq!(schema_change.table_changes.len(), 1); + let table_change = &schema_change.table_changes[0]; + assert_eq!(table_change.cdc_table_name, "mydb.test"); + assert_eq!(table_change.change_type, TableChangeType::Alter); + assert_eq!(table_change.columns.len(), 3); + let column_names = table_change.columns.iter().map(|c| c.name()).collect_vec(); + assert_eq!(column_names, vec!["id", "v1", "v2"]); + } + _ => panic!("unexpected parse result: {:?}", res), + } } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 2c036ff724edd..d930b3ec1a301 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -13,11 +13,9 @@ // limitations under the License. use itertools::Itertools; -use jsonbb::ValueRef; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; use risingwave_common::types::{ - DataType, Datum, DatumCow, Scalar, ScalarImpl, ScalarRefImpl, StructType, Timestamptz, - ToDatumRef, + DataType, Datum, DatumCow, Scalar, ScalarImpl, ScalarRefImpl, Timestamptz, ToDatumRef, }; use risingwave_connector_codec::decoder::AccessExt; use risingwave_pb::plan_common::additional_column::ColumnType; @@ -196,6 +194,7 @@ pub fn parse_schema_change( is_hidden: false, }) .collect_vec(), + change_type: ty.as_str().into(), }); } From 27a7017f869b51502cf4318bb1c5d8fe05d63ead Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 30 Jul 2024 18:39:35 +0800 Subject: [PATCH 35/75] clean --- .../connector/source/core/DbzChangeEventConsumer.java | 3 ++- .../src/main/resources/mysql.properties | 2 +- proto/connector_service.proto | 9 +++++---- src/connector/src/parser/plain_parser.rs | 4 ++-- src/connector/src/source/cdc/source/message.rs | 6 ++++-- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index d3343b8ae2a8b..7537468bc2af0 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -167,7 +167,8 @@ var record = event.value(); switch (eventType) { case HEARTBEAT: { - var message = msgBuilder.build(); + var message = + msgBuilder.setMsgType(CdcMessage.CdcMessageType.HEARTBEAT).build(); LOG.debug("heartbeat => {}", message.getOffset()); respBuilder.addEvents(message); break; diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties index a1dc27879fc6f..0c62a51986b1c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties @@ -13,7 +13,7 @@ table.include.list=${database.name}.${table.name:-*} schema.history.internal.store.only.captured.tables.ddl=true schema.history.internal.store.only.captured.databases.ddl=true # default to disable schema change events -include.schema.changes=${debezium.include.schema.changes:-true} +include.schema.changes=${debezium.include.schema.changes:-false} database.server.id=${server.id} # default to use unencrypted connection database.ssl.mode=${ssl.mode:-disabled} diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 13cf2b2f68cbc..84b1b883225d9 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -149,9 +149,10 @@ message SinkCoordinatorStreamResponse { message CdcMessage { enum CdcMessageType { UNSPECIFIED = 0; - DATA = 1; - TRANSACTION_META = 2; - SCHEMA_CHANGE = 3; + HEARTBEAT = 1; + DATA = 2; + TRANSACTION_META = 3; + SCHEMA_CHANGE = 4; } // The value of the Debezium message @@ -161,7 +162,7 @@ message CdcMessage { string full_table_name = 4; int64 source_ts_ms = 5; - // Deprecated: use `type` instead + // Deprecated: use `msg_type` instead reserved "is_transaction_meta"; reserved 6; diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 2d8345b1313c6..711d5af557967 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -99,7 +99,7 @@ impl PlainParser { && let Some(data) = payload { match cdc_meta.msg_type { - CdcMessageType::Data => { + CdcMessageType::Data | CdcMessageType::Heartbeat => { return self.parse_rows(key, Some(data), writer).await; } CdcMessageType::TransactionMeta => { @@ -130,7 +130,7 @@ impl PlainParser { Err(err) => Err(err)?, }; } - CdcMessageType::Unknown => { + CdcMessageType::Unspecified => { unreachable!() } } diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index 4c79257adb71d..831e242fe78de 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -20,7 +20,8 @@ use crate::source::SourceMeta; #[derive(Clone, Debug)] pub enum CdcMessageType { - Unknown, + Unspecified, + Heartbeat, Data, TransactionMeta, SchemaChange, @@ -30,9 +31,10 @@ impl From for CdcMessageType { fn from(msg_type: cdc_message::CdcMessageType) -> Self { match msg_type { cdc_message::CdcMessageType::Data => CdcMessageType::Data, + cdc_message::CdcMessageType::Heartbeat => CdcMessageType::Heartbeat, cdc_message::CdcMessageType::TransactionMeta => CdcMessageType::TransactionMeta, cdc_message::CdcMessageType::SchemaChange => CdcMessageType::SchemaChange, - cdc_message::CdcMessageType::Unspecified => CdcMessageType::Unknown, + cdc_message::CdcMessageType::Unspecified => CdcMessageType::Unspecified, } } } From 158ff10da3817402a2d253d95e14921ae5f1ca3e Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 30 Jul 2024 20:14:51 +0800 Subject: [PATCH 36/75] format --- proto/ddl_service.proto | 2 +- src/connector/src/parser/plain_parser.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 81f90a03de88d..cfa4bd9b19e17 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -5,8 +5,8 @@ package ddl_service; import "catalog.proto"; import "common.proto"; import "meta.proto"; -import "stream_plan.proto"; import "plan_common.proto"; +import "stream_plan.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 711d5af557967..0bb74ef408c72 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -328,7 +328,7 @@ mod tests { "orders".to_string(), 0, transactional - .then(|| cdc_message::CdcMessageType::TransactionMeta) + .then_some(cdc_message::CdcMessageType::TransactionMeta) .unwrap_or(cdc_message::CdcMessageType::Data), )), split_id: SplitId::from("1001"), @@ -358,7 +358,7 @@ mod tests { "orders".to_string(), 0, transactional - .then(|| cdc_message::CdcMessageType::TransactionMeta) + .then_some(cdc_message::CdcMessageType::TransactionMeta) .unwrap_or(cdc_message::CdcMessageType::Data), )), split_id: SplitId::from("1001"), From a7c00d6266b8c640019d9deb7e9288615a9d716f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 30 Jul 2024 20:35:44 +0800 Subject: [PATCH 37/75] clippy --- src/connector/src/parser/plain_parser.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 0bb74ef408c72..a347d4d383a32 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -327,9 +327,11 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional - .then_some(cdc_message::CdcMessageType::TransactionMeta) - .unwrap_or(cdc_message::CdcMessageType::Data), + if transactional { + cdc_message::CdcMessageType::TransactionMeta + } else { + cdc_message::CdcMessageType::Data + }, )), split_id: SplitId::from("1001"), offset: "0".into(), @@ -357,9 +359,11 @@ mod tests { meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta::new( "orders".to_string(), 0, - transactional - .then_some(cdc_message::CdcMessageType::TransactionMeta) - .unwrap_or(cdc_message::CdcMessageType::Data), + if transactional { + cdc_message::CdcMessageType::TransactionMeta + } else { + cdc_message::CdcMessageType::Data + }, )), split_id: SplitId::from("1001"), offset: "0".into(), From 702d1468a46e7d59acf9ff0aa4387a9064bbeaef Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 31 Jul 2024 10:55:35 +0800 Subject: [PATCH 38/75] minor --- src/storage/src/monitor/local_metrics.rs | 2 +- src/utils/pgwire/src/pg_server.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/storage/src/monitor/local_metrics.rs b/src/storage/src/monitor/local_metrics.rs index e4fd3fa1571b5..5fd7fe6c0ef56 100644 --- a/src/storage/src/monitor/local_metrics.rs +++ b/src/storage/src/monitor/local_metrics.rs @@ -210,7 +210,7 @@ impl Drop for StoreLocalStatistic { && !self.added.load(Ordering::Relaxed) && self.need_report() { - // tracing::error!("local stats lost!\n{:#?}", self); + tracing::error!("local stats lost!\n{:#?}", self); } } } diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index b7fea536f9053..87d28e7f566b6 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -283,7 +283,6 @@ pub async fn pg_serve( #[cfg(madsim)] let worker_runtime = tokio::runtime::Builder::new_multi_thread().build().unwrap(); - // let session_mgr = Arc::new(session_mgr); let session_mgr_clone = session_mgr.clone(); let f = async move { loop { From 12778aaf5dbce0c90fe691c180f9b6e8a9622d6d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 31 Jul 2024 15:17:40 +0800 Subject: [PATCH 39/75] clean again --- .../src/parser/debezium/schema_change.rs | 7 +++++++ src/connector/src/parser/mod.rs | 18 +++++++++++------ src/connector/src/parser/plain_parser.rs | 1 - src/frontend/src/rpc/mod.rs | 4 ++-- src/frontend/src/session.rs | 3 ++- src/frontend/src/test_utils.rs | 2 +- src/meta/service/src/ddl_service.rs | 19 ++++++++---------- .../src/executor/source/source_executor.rs | 20 +++++++++---------- 8 files changed, 42 insertions(+), 32 deletions(-) diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index dc1effbfa82a1..c572734aaf897 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -75,4 +75,11 @@ impl SchemaChangeEnvelope { PbSchemaChangeEnvelope { table_changes } } + + pub fn table_names(&self) -> Vec { + self.table_changes + .iter() + .map(|table_change| table_change.cdc_table_name.clone()) + .collect() + } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 9acd26b4fe08d..be376d164fd4c 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -835,14 +835,20 @@ async fn into_chunk_stream_inner( }, Ok(ParseResult::SchemaChange(schema_change)) => { - // TODO: - // 1. block source executor - // 2. send schema change to Meta - // 3. wait for Meta to finish schema change let (oneshot_tx, oneshot_rx) = tokio::sync::oneshot::channel(); + // we bubble up the schema change event to the source executor via channel, + // and wait for the source executor to finish the schema change process before + // parsing the following messages. if let Some(ref tx) = parser.source_ctx().schema_change_tx { - tx.send((schema_change, oneshot_tx)).await.unwrap(); - oneshot_rx.await.unwrap(); + tx.send((schema_change, oneshot_tx)) + .await + .expect("send schema change to executor"); + match oneshot_rx.await { + Ok(()) => {} + Err(e) => { + tracing::error!(error = %e.as_report(), "failed to wait for schema change"); + } + } } } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 717179e7a8331..a347d4d383a32 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -118,7 +118,6 @@ impl PlainParser { }; } CdcMessageType::SchemaChange => { - tracing::info!("got schema change message"); let accessor = self .schema_change_builder .as_mut() diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index 0568ffd2aaa8c..6f79eed517867 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -74,6 +74,7 @@ impl FrontendService for FrontendServiceImpl { } } +/// Get the new table plan for the given table schema change async fn get_new_table_plan( table_change: TableSchemaChange, table_name: String, @@ -87,7 +88,6 @@ async fn get_new_table_plan( // get a session object for the corresponding user and database let session = session_mgr.get_session(database_id, owner)?; - // call the handle alter method let new_columns = table_change.columns.into_iter().map(|c| c.into()).collect(); let table_name = ObjectName::from(vec![table_name.as_str().into()]); let (new_table_definition, original_catalog) = @@ -105,7 +105,7 @@ async fn get_new_table_plan( table: Some(table), fragment_graph: Some(graph), table_col_index_mapping: Some(col_index_mapping.to_protobuf()), - source: None, + source: None, // none for cdc table job_type: job_type as _, }) } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index df1d7963d89f1..ad909561ace66 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -263,6 +263,8 @@ impl FrontendEnv { let frontend_rpc_addr = opts.health_check_listener_addr.parse().unwrap(); // Register in meta by calling `AddWorkerNode` RPC. + // Use the rpc server address as the frontend address, since Meta needs to get frontend rpc + // client based on this address. let (meta_client, system_params_reader) = MetaClient::register_new( opts.meta_addr, WorkerType::Frontend, @@ -344,7 +346,6 @@ impl FrontendEnv { let observer_join_handle = observer_manager.start().await; join_handles.push(observer_join_handle); - // meta_client.activate(&frontend_address).await?; meta_client.activate(&frontend_rpc_addr).await?; let frontend_metrics = Arc::new(GLOBAL_FRONTEND_METRICS.clone()); diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index d3087179129e4..242058c49e166 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -90,7 +90,7 @@ impl SessionManager for LocalFrontend { _database_id: u32, _user_name: u32, ) -> std::result::Result, BoxedError> { - todo!() + unreachable!() } fn connect( diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 3b0c03e47c1cd..43ed3b62cc468 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -930,6 +930,7 @@ impl DdlService for DdlServiceImpl { ) -> Result, Status> { let req = request.into_inner(); + // randomly select a frontend worker to get the replace table plan let mut workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) @@ -939,7 +940,6 @@ impl DdlService for DdlServiceImpl { .first() .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; - tracing::info!(">> get client for frontend {:?}", worker); let client = self .env .frontend_client_pool() @@ -955,16 +955,16 @@ impl DdlService for DdlServiceImpl { for table_change in schema_change.table_changes { let cdc_table_name = table_change.cdc_table_name.clone(); - - tracing::info!(">> auto schema change cdc table: {}", cdc_table_name); - // get the table catalog corresponding to the cdc table let tables: Vec
= self .metadata_manager .get_table_catalog_by_cdc_table_name(cdc_table_name) .await?; - tracing::info!(">> number of table to replace: {}", tables.len()); + tracing::info!( + "Table jobs to replace: {:?}", + tables.iter().map(|t| t.id).collect::>() + ); for table in tables { // send a request to the frontend to get the ReplaceTablePlan let resp = client @@ -977,20 +977,17 @@ impl DdlService for DdlServiceImpl { .await .map_err(MetaError::from)?; - if let Some(plan) = resp.replace_plan.as_ref() { + if let Some(plan) = resp.replace_plan { plan.table .as_ref() - .inspect(|t| tracing::info!("Table to replace: {}", t.name)); - } - - if let Some(plan) = resp.replace_plan { + .inspect(|t| tracing::info!("Table job to replace: {}", t.id)); // start the schema change procedure self.ddl_controller .run_command(DdlCommand::ReplaceTable(Self::extract_replace_table_info( plan, ))) .await?; - tracing::info!(">> replace table {} success", table.id); + tracing::info!("Table replaced {} success", table.id); } } } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 14d23c26d277b..75c468506013a 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -124,27 +124,27 @@ impl SourceExecutor { .map(|column_desc| column_desc.column_id) .collect_vec(); - let (schema_change_tx, rx) = - tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(32); + let (schema_change_tx, mut schema_change_rx) = + tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(16); let meta_client = self.actor_ctx.meta_client.clone(); + // spawn a task to handle schema change event from source parser let _ = tokio::task::spawn(async move { - let mut schema_change_rx = rx; - while let Some((schema_change, parser_tx)) = schema_change_rx.recv().await { - tracing::info!("recv a schema change envelope"); - - // handle schema change + while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { + let table_names = schema_change.table_names(); + tracing::info!("recv a schema change event for tables: {:?}", table_names); if let Some(ref meta_client) = meta_client { + // TODO: retry on rpc error match meta_client .auto_schema_change(schema_change.to_protobuf()) .await { Ok(_) => { - tracing::info!("schema change success"); - parser_tx.send(()).unwrap(); + tracing::info!("schema change success for tables: {:?}", table_names); + finish_tx.send(()).unwrap(); } Err(e) => { tracing::error!(error = ?e.as_report(), "schema change error"); - parser_tx.send(()).unwrap(); + finish_tx.send(()).unwrap(); } } } From 64527459e9fc653852e86dadc224cc42a058589c Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 31 Jul 2024 17:59:13 +0800 Subject: [PATCH 40/75] fix comment --- .../connector/source/core/DbzChangeEventConsumer.java | 6 ------ proto/ddl_service.proto | 2 ++ src/connector/Cargo.toml | 1 - src/connector/src/parser/debezium/schema_change.rs | 8 ++++++++ src/connector/src/parser/mod.rs | 4 +++- src/connector/src/parser/unified/debezium.rs | 6 ++++++ 6 files changed, 19 insertions(+), 8 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index 7537468bc2af0..f1005447c6616 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -192,12 +192,6 @@ var record = event.value(); case SCHEMA_CHANGE: { - var ddl = ((Struct) record.value()).getString("ddl"); - if (ddl.contains("CREATE") || ddl.contains("DROP")) { - LOG.info("skip create/drop table event"); - continue; - } - var sourceStruct = ((Struct) record.value()).getStruct("source"); if (sourceStruct == null) { throw new CdcConnectorException( diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index cfa4bd9b19e17..f78c08e2a9b52 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -449,6 +449,8 @@ message TableSchemaChange { enum TableChangeType { UNSPECIFIED = 0; ALTER = 1; + CREATE = 2; + DROP = 3; } TableChangeType change_type = 1; diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 528dc3884bfca..a5db0c4fcb938 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -136,7 +136,6 @@ sea-schema = { version = "0.14", default-features = false, features = [ "discovery", "sqlx-postgres", "sqlx-mysql", - "sqlx-sqlite", ] } serde = { version = "1", features = ["derive", "rc"] } serde_derive = "1" diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 72fb15273e28c..4c61b52caaba9 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -27,6 +27,8 @@ pub struct SchemaChangeEnvelope { pub(crate) enum TableChangeType { Unspecified, Alter, + Create, + Drop, } impl TableChangeType { @@ -34,6 +36,8 @@ impl TableChangeType { pub fn from_proto(value: PbTableChangeType) -> Self { match value { PbTableChangeType::Alter => TableChangeType::Alter, + PbTableChangeType::Create => TableChangeType::Create, + PbTableChangeType::Drop => TableChangeType::Drop, PbTableChangeType::Unspecified => TableChangeType::Unspecified, } } @@ -41,6 +45,8 @@ impl TableChangeType { pub fn to_proto(self) -> PbTableChangeType { match self { TableChangeType::Alter => PbTableChangeType::Alter, + TableChangeType::Create => PbTableChangeType::Create, + TableChangeType::Drop => PbTableChangeType::Drop, TableChangeType::Unspecified => PbTableChangeType::Unspecified, } } @@ -50,6 +56,8 @@ impl From<&str> for TableChangeType { fn from(value: &str) -> Self { match value { "ALTER" => TableChangeType::Alter, + "CREATE" => TableChangeType::Create, + "DROP" => TableChangeType::Drop, _ => TableChangeType::Unspecified, } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 887c8b220d604..055eab777be5b 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -834,7 +834,9 @@ async fn into_chunk_stream_inner( } }, - Ok(ParseResult::SchemaChange(_)) => todo!(), + Ok(ParseResult::SchemaChange(_)) => { + // TODO + } } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index d930b3ec1a301..22d58f597e30a 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -23,6 +23,7 @@ use risingwave_pb::plan_common::additional_column::ColumnType; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; use crate::parser::debezium::schema_change::{SchemaChangeEnvelope, TableSchemaChange}; use crate::parser::mysql::mysql_typename_to_rw_type; +use crate::parser::schema_change::TableChangeType; use crate::parser::TransactionControl; use crate::source::{ConnectorProperties, SourceColumnDesc}; @@ -161,6 +162,11 @@ pub fn parse_schema_change( let id = jsonb_access_field!(jsonb, "id", string); let ty = jsonb_access_field!(jsonb, "type", string); + let ddl_type: TableChangeType = ty.as_str().into(); + if matches!(ddl_type, TableChangeType::Create | TableChangeType::Drop) { + tracing::debug!("skip table schema change for create/drop command"); + continue; + } let mut column_descs: Vec = vec![]; if let Some(table) = jsonb.access_object_field("table") From c50856a4799646a02d1bcfff3a8e063b6a8c3db2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 31 Jul 2024 18:01:54 +0800 Subject: [PATCH 41/75] minor --- src/common/src/types/jsonb.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index a16d3ca53a255..fa80069080ff4 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -301,6 +301,7 @@ impl<'a> JsonbRef<'a> { .ok_or_else(|| format!("cannot cast jsonb {} to type boolean", self.type_name())) } + /// If the JSON is a string, returns the associated string. pub fn as_string(&self) -> Result { self.0 .as_str() From a8f52e4585631cadb88699de907c17a814e0f3da Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 1 Aug 2024 17:01:54 +0800 Subject: [PATCH 42/75] fix comments --- proto/connector_service.proto | 6 +- src/connector/codec/src/decoder/mod.rs | 6 +- src/connector/src/parser/mysql.rs | 7 ++- src/connector/src/parser/plain_parser.rs | 77 ++++++++++++++++++++---- 4 files changed, 73 insertions(+), 23 deletions(-) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 84b1b883225d9..cf549a8e2e493 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -161,14 +161,10 @@ message CdcMessage { string offset = 3; string full_table_name = 4; int64 source_ts_ms = 5; - - // Deprecated: use `msg_type` instead - reserved "is_transaction_meta"; - reserved 6; + CdcMessageType msg_type = 6; // The key of the Debezium message, which only used by `mongodb-cdc` connector. string key = 7; - CdcMessageType msg_type = 8; } enum SourceType { diff --git a/src/connector/codec/src/decoder/mod.rs b/src/connector/codec/src/decoder/mod.rs index 2e62aabb22b85..814e06a166c6c 100644 --- a/src/connector/codec/src/decoder/mod.rs +++ b/src/connector/codec/src/decoder/mod.rs @@ -44,9 +44,9 @@ pub enum AccessError { #[error(transparent)] NotImplemented(#[from] NotImplemented), - - #[error(transparent)] - Other(#[from] anyhow::Error), + // NOTE: We intentionally don't embed `anyhow::Error` in `AccessError` since it happens + // in record-level and it might be too heavy to capture the backtrace + // when creating a new `anyhow::Error`. } pub type AccessResult = std::result::Result; diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index 453ab413f88e0..02238eb2f7b4e 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -22,6 +22,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{ DataType, Date, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; +use risingwave_connector_codec::decoder::{AccessError, AccessResult}; use rust_decimal::Decimal as RustDecimal; use thiserror_ext::AsReport; @@ -141,7 +142,7 @@ pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> Owne OwnedRow::new(datums) } -pub fn mysql_typename_to_rw_type(type_name: &str) -> anyhow::Result { +pub fn mysql_typename_to_rw_type(type_name: &str) -> AccessResult { match type_name.to_lowercase().as_str() { "tinyint" | "smallint" => Ok(DataType::Int16), "int" => Ok(DataType::Int32), @@ -156,7 +157,9 @@ pub fn mysql_typename_to_rw_type(type_name: &str) -> anyhow::Result { "datetime" => Ok(DataType::Timestamp), "json" => Ok(DataType::Jsonb), "binary" | "varbinary" | "blob" | "mediumblob" | "longblob" => Ok(DataType::Bytea), - _ => Err(anyhow::anyhow!("unsupported type: {}", type_name)), + _ => Err(AccessError::UnsupportedType { + ty: type_name.to_string(), + }), } } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index a347d4d383a32..663fcb30e6ac9 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -202,6 +202,7 @@ mod tests { use std::ops::Deref; use std::sync::Arc; + use expect_test::expect; use futures::executor::block_on; use futures::StreamExt; use futures_async_stream::try_stream; @@ -210,7 +211,6 @@ mod tests { use risingwave_pb::connector_service::cdc_message; use super::*; - use crate::parser::schema_change::TableChangeType; use crate::parser::{MessageMeta, SourceStreamChunkBuilder, TransactionControl}; use crate::source::cdc::DebeziumCdcMeta; use crate::source::{ConnectorProperties, DataType, SourceMessage, SplitId}; @@ -503,17 +503,68 @@ mod tests { .await; let res = res.unwrap(); - match res { - ParseResult::SchemaChange(schema_change) => { - assert_eq!(schema_change.table_changes.len(), 1); - let table_change = &schema_change.table_changes[0]; - assert_eq!(table_change.cdc_table_name, "mydb.test"); - assert_eq!(table_change.change_type, TableChangeType::Alter); - assert_eq!(table_change.columns.len(), 3); - let column_names = table_change.columns.iter().map(|c| c.name()).collect_vec(); - assert_eq!(column_names, vec!["id", "v1", "v2"]); - } - _ => panic!("unexpected parse result: {:?}", res), - } + expect![[r#" + SchemaChange( + SchemaChangeEnvelope { + table_changes: [ + TableSchemaChange { + cdc_table_name: "mydb.test", + columns: [ + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Int32, + column_id: #2147483646, + name: "id", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Timestamptz, + column_id: #2147483646, + name: "v1", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: Varchar, + column_id: #2147483646, + name: "v2", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column: AdditionalColumn { + column_type: None, + }, + version: Pr13707, + }, + is_hidden: false, + }, + ], + change_type: Alter, + }, + ], + }, + ) + "#]] + .assert_debug_eq(&res); } } From b85a604f38a1c0db1d071e0dbaa541afa8b12728 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 1 Aug 2024 19:17:44 +0800 Subject: [PATCH 43/75] remove fullTableName --- .../connector/source/core/DbzChangeEventConsumer.java | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java index f1005447c6616..375b4d4a3ad62 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzChangeEventConsumer.java @@ -200,21 +200,15 @@ var record = event.value(); // upstream event time long sourceTsMs = sourceStruct.getInt64("ts_ms"); - // concat full table name, right now we only support MySQL schema change - // event - var fullTableName = - String.format( - "%s.%s", - sourceStruct.getString("db"), - sourceStruct.getString("table")); byte[] payload = payloadConverter.fromConnectData( record.topic(), record.valueSchema(), record.value()); + // We intentionally don't set the fullTableName for schema change event, + // since it doesn't need to be routed to a specific cdc table var message = msgBuilder .setMsgType(CdcMessage.CdcMessageType.SCHEMA_CHANGE) - .setFullTableName(fullTableName) .setPayload(new String(payload, StandardCharsets.UTF_8)) .setSourceTsMs(sourceTsMs) .build(); From 22e67e878badc46a381a1ed6a31b464e1dc05acd Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 1 Aug 2024 21:17:26 +0800 Subject: [PATCH 44/75] minor --- src/meta/src/barrier/command.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 760177bb73eab..013a8df72906e 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -849,7 +849,6 @@ impl CommandContext { pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { let futures = self.info.node_map.values().map(|worker_node| async { let worker = worker_node.clone(); - tracing::info!("get client for compute {:?}", worker); let client = self .barrier_manager_context .env From 70c92992fd6fb65c86884fa0ff35ec46bbaab87b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 2 Aug 2024 11:43:13 +0800 Subject: [PATCH 45/75] fix type cast --- src/connector/src/parser/mysql.rs | 22 -------- src/connector/src/parser/unified/debezium.rs | 16 +++++- .../src/source/cdc/external/mysql.rs | 53 ++++++++++++++++++- 3 files changed, 65 insertions(+), 26 deletions(-) diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index 02238eb2f7b4e..a28dddc9aa65a 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -22,7 +22,6 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{ DataType, Date, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; -use risingwave_connector_codec::decoder::{AccessError, AccessResult}; use rust_decimal::Decimal as RustDecimal; use thiserror_ext::AsReport; @@ -142,27 +141,6 @@ pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> Owne OwnedRow::new(datums) } -pub fn mysql_typename_to_rw_type(type_name: &str) -> AccessResult { - match type_name.to_lowercase().as_str() { - "tinyint" | "smallint" => Ok(DataType::Int16), - "int" => Ok(DataType::Int32), - "bigint" => Ok(DataType::Int64), - "float" => Ok(DataType::Float32), - "double" => Ok(DataType::Float64), - "decimal" => Ok(DataType::Decimal), - "char" | "varchar" | "text" | "mediumtext" | "longtext" => Ok(DataType::Varchar), - "date" => Ok(DataType::Date), - "time" => Ok(DataType::Time), - "timestamp" => Ok(DataType::Timestamptz), - "datetime" => Ok(DataType::Timestamp), - "json" => Ok(DataType::Jsonb), - "binary" | "varbinary" | "blob" | "mediumblob" | "longblob" => Ok(DataType::Bytea), - _ => Err(AccessError::UnsupportedType { - ty: type_name.to_string(), - }), - } -} - #[cfg(test)] mod tests { diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 22d58f597e30a..e4ec3f9870b43 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -19,12 +19,13 @@ use risingwave_common::types::{ }; use risingwave_connector_codec::decoder::AccessExt; use risingwave_pb::plan_common::additional_column::ColumnType; +use thiserror_ext::AsReport; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; use crate::parser::debezium::schema_change::{SchemaChangeEnvelope, TableSchemaChange}; -use crate::parser::mysql::mysql_typename_to_rw_type; use crate::parser::schema_change::TableChangeType; use crate::parser::TransactionControl; +use crate::source::cdc::external::mysql::{mysql_type_to_rw_type, type_name_to_mysql_type}; use crate::source::{ConnectorProperties, SourceColumnDesc}; // Example of Debezium JSON value: @@ -181,7 +182,18 @@ pub fn parse_schema_change( unimplemented!() } ConnectorProperties::MysqlCdc(_) => { - mysql_typename_to_rw_type(type_name.as_str())? + let ty = type_name_to_mysql_type(type_name.as_str()); + match ty { + Some(ty) => mysql_type_to_rw_type(&ty).map_err(|err| { + tracing::warn!(error=%err.as_report(), "unsupported mysql type in schema change message"); + AccessError::UnsupportedType { + ty: type_name.clone(), + } + })?, + None => { + Err(AccessError::UnsupportedType { ty: type_name })? + } + } } _ => { unreachable!() diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index e5f53720dd6ee..6947ba7a46d6b 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -106,7 +106,7 @@ impl MySqlExternalTable { let mut column_descs = vec![]; let mut pk_names = vec![]; for col in columns { - let data_type = type_to_rw_type(&col.col_type)?; + let data_type = mysql_type_to_rw_type(&col.col_type)?; // column name in mysql is case-insensitive, convert to lowercase let col_name = col.name.to_lowercase(); column_descs.push(ColumnDesc::named( @@ -138,7 +138,56 @@ impl MySqlExternalTable { } } -fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { +pub fn type_name_to_mysql_type(ty_name: &str) -> Option { + macro_rules! column_type { + ($($name:literal => $variant:ident),* $(,)?) => { + match ty_name.to_lowercase().as_str() { + $( + $name => Some(ColumnType::$variant(Default::default())), + )* + _ => None, + } + }; + } + + column_type! { + "bit" => Bit, + "tinyint" => TinyInt, + "smallint" => SmallInt, + "mediumint" => MediumInt, + "int" => Int, + "bigint" => BigInt, + "decimal" => Decimal, + "float" => Float, + "double" => Double, + "time" => Time, + "datetime" => DateTime, + "timestamp" => Timestamp, + "char" => Char, + "nchar" => NChar, + "varchar" => Varchar, + "nvarchar" => NVarchar, + "binary" => Binary, + "varbinary" => Varbinary, + "text" => Text, + "tinytext" => TinyText, + "mediumtext" => MediumText, + "longtext" => LongText, + "blob" => Blob, + "enum" => Enum, + "set" => Set, + "geometry" => Geometry, + "point" => Point, + "linestring" => LineString, + "polygon" => Polygon, + "multipoint" => MultiPoint, + "multilinestring" => MultiLineString, + "multipolygon" => MultiPolygon, + "geometrycollection" => GeometryCollection, + } +} + +pub fn mysql_type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { let dtype = match col_type { ColumnType::Serial => DataType::Int32, ColumnType::Bit(attr) => { From af8bca93affa09f86bd0688af9c04673c61cba26 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 2 Aug 2024 15:55:18 +0800 Subject: [PATCH 46/75] clean --- src/frontend/src/catalog/table_catalog.rs | 2 ++ src/frontend/src/scheduler/distributed/query.rs | 1 + src/meta/src/barrier/command.rs | 1 - src/storage/src/filter_key_extractor.rs | 1 + src/utils/pgwire/src/pg_server.rs | 6 +++--- 5 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 5d5e7e9060872..49ca0f42adbdb 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -673,6 +673,7 @@ mod tests { created_at_cluster_version: None, initialized_at_cluster_version: None, version_column_index: None, + cdc_table_name: None, } .into(); @@ -735,6 +736,7 @@ mod tests { initialized_at_cluster_version: None, dependent_relations: vec![], version_column_index: None, + cdc_table_name: None, } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index ce13960ab221f..d4e3e2925ff23 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -596,6 +596,7 @@ pub(crate) mod tests { incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, + cdc_table_name: None, }; let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 013a8df72906e..2c65f467b4a98 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -848,7 +848,6 @@ impl CommandContext { pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { let futures = self.info.node_map.values().map(|worker_node| async { - let worker = worker_node.clone(); let client = self .barrier_manager_context .env diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index 36598a9ec9e47..880b794a4847f 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -553,6 +553,7 @@ mod tests { incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, + cdc_table_name: None, } } diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 87d28e7f566b6..32aefa4602ca0 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -384,8 +384,8 @@ mod tests { fn get_session( &self, - database_id: u32, - user_name: u32, + _database_id: u32, + _user_name: u32, ) -> Result, BoxedError> { todo!() } @@ -529,7 +529,7 @@ mod tests { tokio::spawn(async move { pg_serve( &bind_addr, - session_mgr, + Arc::new(session_mgr), None, None, CancellationToken::new(), // dummy From 5b1d8d6de1d37eac9ae7e508f4de35724970cd11 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 2 Aug 2024 16:18:18 +0800 Subject: [PATCH 47/75] clippy --- proto/ddl_service.proto | 2 -- proto/frontend_service.proto | 1 - src/frontend/src/handler/alter_table_column.rs | 4 ++-- src/stream/src/executor/source/source_executor.rs | 2 +- 4 files changed, 3 insertions(+), 6 deletions(-) diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index d9a36d8ae52fc..ab51b7113e8f6 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -445,7 +445,6 @@ message CommentOnResponse { uint64 version = 2; } - message TableSchemaChange { enum TableChangeType { UNSPECIFIED = 0; @@ -469,7 +468,6 @@ message AutoSchemaChangeRequest { message AutoSchemaChangeResponse {} - service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 8fb49e1bedb3a..8914091e9c5c7 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -7,7 +7,6 @@ import "ddl_service.proto"; option java_package = "com.risingwave.proto"; option optimize_for = SPEED; - message GetTableReplacePlanRequest { uint32 database_id = 1; uint32 owner = 2; diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index ee65c32661a88..477759d907a87 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -97,9 +97,9 @@ pub async fn get_new_table_definition_for_cdc_table( ))? } - // since the DDL is committed on upstream, so we can safely replace the original columns with new columns + // replace the original columns with new version columns let mut new_column_defs = vec![]; - for col in new_columns.into_iter() { + for col in new_columns { let ty = to_ast_data_type(col.data_type())?; new_column_defs.push(ColumnDef::new(col.name().into(), ty, None, vec![])); } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 75c468506013a..ac5f8538122d7 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -128,7 +128,7 @@ impl SourceExecutor { tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(16); let meta_client = self.actor_ctx.meta_client.clone(); // spawn a task to handle schema change event from source parser - let _ = tokio::task::spawn(async move { + let _join_handle = tokio::task::spawn(async move { while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { let table_names = schema_change.table_names(); tracing::info!("recv a schema change event for tables: {:?}", table_names); From 6d18dce5572a4aef3285dbcc4ed086fd013d710b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 6 Aug 2024 15:10:14 +0800 Subject: [PATCH 48/75] fix comments --- .../cdc_inline/auto_schema_change_mysql.slt | 3 +- .../src/main/resources/mysql.properties | 2 +- proto/catalog.proto | 3 ++ proto/frontend_service.proto | 1 - .../src/handler/alter_table_column.rs | 32 ++++++++++++------- src/frontend/src/lib.rs | 3 +- src/frontend/src/rpc/mod.rs | 2 +- src/frontend/src/session.rs | 12 +++---- src/frontend/src/test_utils.rs | 2 +- src/meta/service/src/ddl_service.rs | 5 ++- src/stream/src/executor/actor.rs | 2 +- .../src/executor/source/source_executor.rs | 32 ++++++++++--------- src/utils/pgwire/src/pg_server.rs | 13 +++++--- 13 files changed, 66 insertions(+), 46 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt index 4d6f94eb4340a..31bb9d1b0421b 100644 --- a/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_change_mysql.slt @@ -23,7 +23,8 @@ create source mysql_source with ( username = 'root', password = '${MYSQL_PWD:}', database.name = 'mytest', - server.id = '5701' + server.id = '5701', + auto.schema.change = 'true' ); statement ok diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties index a1dc27879fc6f..b5b7b71882c08 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties @@ -13,7 +13,7 @@ table.include.list=${database.name}.${table.name:-*} schema.history.internal.store.only.captured.tables.ddl=true schema.history.internal.store.only.captured.databases.ddl=true # default to disable schema change events -include.schema.changes=${debezium.include.schema.changes:-true} +include.schema.changes=${auto.schema.change:-false} database.server.id=${server.id} # default to use unencrypted connection database.ssl.mode=${ssl.mode:-disabled} diff --git a/proto/catalog.proto b/proto/catalog.proto index 9e6eda63939a7..ee82489191f8b 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -409,6 +409,9 @@ message Table { // conflict" operations. optional uint32 version_column_index = 38; + // This field stores the full name of the upstream table for a CDC table, + // which is used in auto schema change to query Tables that are mapped to the + // same upstream table. optional string cdc_table_name = 39; // Per-table catalog version, used by schema change. `None` for internal diff --git a/proto/frontend_service.proto b/proto/frontend_service.proto index 8914091e9c5c7..7cdac815fde66 100644 --- a/proto/frontend_service.proto +++ b/proto/frontend_service.proto @@ -18,7 +18,6 @@ message GetTableReplacePlanResponse { ddl_service.ReplaceTablePlan replace_plan = 1; } -// Frontend will start a background worker to service FrontendService { rpc GetTableReplacePlan(GetTableReplacePlanRequest) returns (GetTableReplacePlanResponse); } diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 419276673445d..2e30f58e1366d 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -94,18 +94,24 @@ pub async fn get_new_table_definition_for_cdc_table( source_schema.is_none(), "source schema should be None for CDC table" ); - if original_columns.is_empty() { - Err(ErrorCode::NotSupported( - "alter a table with empty column definitions".to_string(), - "Please recreate the table with column definitions.".to_string(), - ))? - } - // replace the original columns with new version columns + let orig_column_map: HashMap = HashMap::from_iter( + original_columns + .iter() + .map(|col| (col.name.real_value(), col.clone())), + ); + + // update the original columns with new version columns let mut new_column_defs = vec![]; for col in new_columns { - let ty = to_ast_data_type(col.data_type())?; - new_column_defs.push(ColumnDef::new(col.name().into(), ty, None, vec![])); + // if the column exists in the original definitoins, use the original column definition. + // since we don't support altering the column type right now + if let Some(original_col) = orig_column_map.get(&col.name().to_string()) { + new_column_defs.push(original_col.clone()); + } else { + let ty = to_ast_data_type(col.data_type())?; + new_column_defs.push(ColumnDef::new(col.name().into(), ty, None, vec![])); + } } *original_columns = new_column_defs; @@ -120,6 +126,8 @@ fn to_ast_data_type(ty: &DataType) -> Result { DataType::Int64 => Ok(AstDataType::BigInt), DataType::Float32 => Ok(AstDataType::Real), DataType::Float64 => Ok(AstDataType::Double), + // TODO: handle precision and scale for decimal + DataType::Decimal => Ok(AstDataType::Decimal(None, None)), DataType::Date => Ok(AstDataType::Date), DataType::Varchar => Ok(AstDataType::Varchar), DataType::Time => Ok(AstDataType::Time(false)), @@ -141,7 +149,9 @@ fn to_ast_data_type(ty: &DataType) -> Result { .try_collect()?; Ok(AstDataType::Struct(fields)) } - _ => Err(anyhow!("unsupported data type: {:?}", ty).context("to_ast_data_type"))?, + DataType::Serial | DataType::Int256 => { + Err(anyhow!("unsupported data type: {:?}", ty).context("to_ast_data_type"))? + } } } diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index c43074cbac7d8..ba99348ff41c0 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -121,10 +121,11 @@ pub struct FrontendOpts { #[clap( long, + alias = "health-check-listener-addr", env = "RW_HEALTH_CHECK_LISTENER_ADDR", default_value = "127.0.0.1:6786" )] - pub health_check_listener_addr: String, + pub frontend_rpc_listener_addr: String, /// The path of `risingwave.toml` configuration file. /// diff --git a/src/frontend/src/rpc/mod.rs b/src/frontend/src/rpc/mod.rs index 8c3d4d701c27c..257695cc99e48 100644 --- a/src/frontend/src/rpc/mod.rs +++ b/src/frontend/src/rpc/mod.rs @@ -88,7 +88,7 @@ async fn get_new_table_plan( .expect("session manager has been initialized"); // get a session object for the corresponding user and database - let session = session_mgr.get_session(database_id, owner)?; + let session = session_mgr.create_dummy_session(database_id, owner)?; let new_columns = table_change.columns.into_iter().map(|c| c.into()).collect(); let table_name = ObjectName::from(vec![table_name.as_str().into()]); diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 54af8777c4f87..4eb6ac680903d 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -261,7 +261,7 @@ impl FrontendEnv { .unwrap(); info!("advertise addr is {}", frontend_address); - let frontend_rpc_addr = opts.health_check_listener_addr.parse().unwrap(); + let frontend_rpc_addr = opts.frontend_rpc_listener_addr.parse().unwrap(); // Register in meta by calling `AddWorkerNode` RPC. // Use the rpc server address as the frontend address, since Meta needs to get frontend rpc // client based on this address. @@ -358,7 +358,7 @@ impl FrontendEnv { let health_srv = HealthServiceImpl::new(); let frontend_srv = FrontendServiceImpl::new(); - let frontend_rpc_addr = opts.health_check_listener_addr.parse().unwrap(); + let frontend_rpc_addr = opts.frontend_rpc_listener_addr.parse().unwrap(); let telemetry_manager = TelemetryManager::new( Arc::new(meta_client.clone()), @@ -385,7 +385,7 @@ impl FrontendEnv { }); info!( "Frontend RPC Listener is set up on {}", - opts.health_check_listener_addr.clone() + opts.frontend_rpc_listener_addr.clone() ); let creating_streaming_job_tracker = @@ -1138,19 +1138,19 @@ pub struct SessionManagerImpl { impl SessionManager for SessionManagerImpl { type Session = SessionImpl; - fn get_session( + fn create_dummy_session( &self, database_id: u32, user_id: u32, ) -> std::result::Result, BoxedError> { - let dumb_addr = Address::Tcp(SocketAddr::new( + let dummy_addr = Address::Tcp(SocketAddr::new( IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691, // port of meta )); let user_reader = self.env.user_info_reader(); let reader = user_reader.read_guard(); if let Some(user_name) = reader.get_user_name_by_id(user_id) { - self.connect_inner(database_id, user_name.as_str(), Arc::new(dumb_addr)) + self.connect_inner(database_id, user_name.as_str(), Arc::new(dummy_addr)) } else { Err(Box::new(Error::new( ErrorKind::InvalidInput, diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 242058c49e166..086d4ff7de251 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -85,7 +85,7 @@ pub struct LocalFrontend { impl SessionManager for LocalFrontend { type Session = SessionImpl; - fn get_session( + fn create_dummy_session( &self, _database_id: u32, _user_name: u32, diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 940f57c15a7ee..7d6e299352362 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -931,13 +931,12 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); // randomly select a frontend worker to get the replace table plan - let mut workers = self + let workers = self .metadata_manager .list_worker_node(Some(WorkerType::Frontend), Some(State::Running)) .await?; - workers.shuffle(&mut thread_rng()); let worker = workers - .first() + .choose(&mut thread_rng()) .ok_or_else(|| MetaError::from(anyhow!("no frontend worker available")))?; let client = self diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index a94e07949f0f6..22eb9a1ff37f2 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -57,7 +57,7 @@ pub struct ActorContext { // mv_table_id to subscription id pub related_subscriptions: HashMap>, - // Meta client. currently used for auto schema change + // Meta client. currently used for auto schema change. `None` for test only pub meta_client: Option, } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index ac5f8538122d7..8e2e49aec7636 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -126,26 +126,28 @@ impl SourceExecutor { let (schema_change_tx, mut schema_change_rx) = tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(16); - let meta_client = self.actor_ctx.meta_client.clone(); + let meta_client = self + .actor_ctx + .meta_client + .clone() + .expect("A meta client is required for auto schema change."); // spawn a task to handle schema change event from source parser let _join_handle = tokio::task::spawn(async move { while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { let table_names = schema_change.table_names(); tracing::info!("recv a schema change event for tables: {:?}", table_names); - if let Some(ref meta_client) = meta_client { - // TODO: retry on rpc error - match meta_client - .auto_schema_change(schema_change.to_protobuf()) - .await - { - Ok(_) => { - tracing::info!("schema change success for tables: {:?}", table_names); - finish_tx.send(()).unwrap(); - } - Err(e) => { - tracing::error!(error = ?e.as_report(), "schema change error"); - finish_tx.send(()).unwrap(); - } + // TODO: retry on rpc error + match meta_client + .auto_schema_change(schema_change.to_protobuf()) + .await + { + Ok(_) => { + tracing::info!("schema change success for tables: {:?}", table_names); + finish_tx.send(()).unwrap(); + } + Err(e) => { + tracing::error!(error = ?e.as_report(), "schema change error"); + finish_tx.send(()).unwrap(); } } } diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index 32aefa4602ca0..b5763548d3458 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -47,8 +47,13 @@ pub type SessionId = (ProcessId, SecretKey); pub trait SessionManager: Send + Sync + 'static { type Session: Session; - fn get_session(&self, database_id: u32, user_id: u32) - -> Result, BoxedError>; + /// In the process of auto schema change, we need a dummy session to access + /// catalog information in frontend and build a replace plan for the table. + fn create_dummy_session( + &self, + database_id: u32, + user_id: u32, + ) -> Result, BoxedError>; fn connect( &self, @@ -382,12 +387,12 @@ mod tests { impl SessionManager for MockSessionManager { type Session = MockSession; - fn get_session( + fn create_dummy_session( &self, _database_id: u32, _user_name: u32, ) -> Result, BoxedError> { - todo!() + unimplemented!() } fn connect( From d1711c91a2ee51ada9967ced92ec0ebf2e30180a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 7 Aug 2024 11:18:26 +0800 Subject: [PATCH 49/75] fix check --- 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 2e30f58e1366d..c91b08344892f 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -106,7 +106,7 @@ pub async fn get_new_table_definition_for_cdc_table( for col in new_columns { // if the column exists in the original definitoins, use the original column definition. // since we don't support altering the column type right now - if let Some(original_col) = orig_column_map.get(&col.name().to_string()) { + if let Some(original_col) = orig_column_map.get(col.name()) { new_column_defs.push(original_col.clone()); } else { let ty = to_ast_data_type(col.data_type())?; From 3038bf24aebe0a58293ca69807ed1d7d05f4e09d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 7 Aug 2024 11:57:03 +0800 Subject: [PATCH 50/75] add license check --- src/connector/src/source/cdc/mod.rs | 1 + src/frontend/src/handler/create_source.rs | 17 ++++++++++++++--- src/license/src/feature.rs | 1 + 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index c86450b59471c..e261b35cb4467 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -45,6 +45,7 @@ pub const CDC_BACKFILL_SNAPSHOT_BATCH_SIZE_KEY: &str = "snapshot.batch_size"; // We enable transaction for shared cdc source by default pub const CDC_TRANSACTIONAL_KEY: &str = "transactional"; pub const CDC_WAIT_FOR_STREAMING_START_TIMEOUT: &str = "cdc.source.wait.streaming.start.timeout"; +pub const CDC_AUTO_SCHEMA_CHANGE_KEY: &str = "auto.schema.change"; pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME; pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index de838b7bebf9f..ff5354e3dae18 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -27,6 +27,7 @@ use risingwave_common::catalog::{ debug_assert_column_ids_distinct, ColumnCatalog, ColumnDesc, ColumnId, Schema, TableId, INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME, }; +use risingwave_common::license::Feature; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; use risingwave_connector::parser::additional_columns::{ @@ -43,9 +44,9 @@ use risingwave_connector::schema::schema_registry::{ use risingwave_connector::schema::AWS_GLUE_SCHEMA_ARN_KEY; use risingwave_connector::sink::iceberg::IcebergConfig; use risingwave_connector::source::cdc::{ - CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CDC_TRANSACTIONAL_KEY, - CDC_WAIT_FOR_STREAMING_START_TIMEOUT, CITUS_CDC_CONNECTOR, MONGODB_CDC_CONNECTOR, - MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, + CDC_AUTO_SCHEMA_CHANGE_KEY, CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, + CDC_TRANSACTIONAL_KEY, CDC_WAIT_FOR_STREAMING_START_TIMEOUT, CITUS_CDC_CONNECTOR, + MONGODB_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, }; use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; @@ -1357,6 +1358,16 @@ pub fn bind_connector_props( validate_compatibility(source_schema, &mut with_properties)?; let create_cdc_source_job = with_properties.is_shareable_cdc_connector(); if is_create_source && create_cdc_source_job { + if let Some(value) = with_properties.get(CDC_AUTO_SCHEMA_CHANGE_KEY) + && value + .parse::() + .map_err(|_| anyhow!("invalid value of '{}' option", CDC_AUTO_SCHEMA_CHANGE_KEY))? + { + Feature::CdcAutoSchemaChange + .check_available() + .map_err(|e| anyhow::anyhow!(e))?; + } + // set connector to backfill mode with_properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into()); // enable cdc sharing mode, which will capture all tables in the given `database.name` diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 186e15c998ae4..d05f0243c3ad7 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -47,6 +47,7 @@ macro_rules! for_all_features { { TimeTravel, Paid, "Query historical data within the retention period."}, { GlueSchemaRegistry, Paid, "Use Schema Registry from AWS Glue rather than Confluent." }, { SecretManagement, Paid, "Secret management." }, + { CdcAutoSchemaChange, Paid, "Auto replicate upstream DDL to CDC Table." }, } }; } From 6ac586bb107a43a54f1ac7bb8f3b705b768507d9 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 7 Aug 2024 14:15:25 +0800 Subject: [PATCH 51/75] fix ut --- .../src/executor/source/source_executor.rs | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 8e2e49aec7636..fe09d040973e2 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -126,28 +126,26 @@ impl SourceExecutor { let (schema_change_tx, mut schema_change_rx) = tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(16); - let meta_client = self - .actor_ctx - .meta_client - .clone() - .expect("A meta client is required for auto schema change."); + let meta_client = self.actor_ctx.meta_client.clone(); // spawn a task to handle schema change event from source parser let _join_handle = tokio::task::spawn(async move { while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { let table_names = schema_change.table_names(); tracing::info!("recv a schema change event for tables: {:?}", table_names); // TODO: retry on rpc error - match meta_client - .auto_schema_change(schema_change.to_protobuf()) - .await - { - Ok(_) => { - tracing::info!("schema change success for tables: {:?}", table_names); - finish_tx.send(()).unwrap(); - } - Err(e) => { - tracing::error!(error = ?e.as_report(), "schema change error"); - finish_tx.send(()).unwrap(); + if let Some(ref meta_client) = meta_client { + match meta_client + .auto_schema_change(schema_change.to_protobuf()) + .await + { + Ok(_) => { + tracing::info!("schema change success for tables: {:?}", table_names); + finish_tx.send(()).unwrap(); + } + Err(e) => { + tracing::error!(error = ?e.as_report(), "schema change error"); + finish_tx.send(()).unwrap(); + } } } } From 0dee4393083f70370f6301cdb45859cf55ba07b7 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 7 Aug 2024 15:13:31 +0800 Subject: [PATCH 52/75] minor --- src/cmd_all/src/standalone.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index ceb890f4cb3af..60aaa9aa5b4ee 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -504,7 +504,7 @@ mod test { ], ), prometheus_listener_addr: "127.0.0.1:1234", - health_check_listener_addr: "127.0.0.1:6786", + frontend_rpc_listener_addr: "127.0.0.1:6786", config_path: "src/config/test.toml", metrics_level: None, enable_barrier_read: None, From f17d1c23fd4aebabdf53df58ee49174a122c42ac Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 8 Aug 2024 22:42:24 +0800 Subject: [PATCH 53/75] fix sim test --- src/frontend/src/lib.rs | 5 ++--- src/frontend/src/session.rs | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 5722124e0e0e9..7ea79cf6f0df8 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -188,9 +188,8 @@ pub fn start( // slow compile in release mode. Box::pin(async move { let listen_addr = opts.listen_addr.clone(); - let session_mgr = SESSION_MANAGER - .get_or_init(|| async { Arc::new(SessionManagerImpl::new(opts).await.unwrap()) }) - .await; + let session_mgr = Arc::new(SessionManagerImpl::new(opts).await.unwrap()); + SESSION_MANAGER.get_or_init(|| session_mgr.clone()); let redact_sql_option_keywords = Arc::new( session_mgr .env() diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 4eb6ac680903d..0fbf4404fcdb6 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1125,8 +1125,8 @@ impl SessionImpl { } } -pub static SESSION_MANAGER: tokio::sync::OnceCell> = - tokio::sync::OnceCell::const_new(); +pub static SESSION_MANAGER: std::sync::OnceLock> = + std::sync::OnceLock::new(); pub struct SessionManagerImpl { env: FrontendEnv, From 40ccdd490b6d9d7a9481e429461bc8d4f6a6f5bd Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 9 Aug 2024 00:19:21 +0800 Subject: [PATCH 54/75] fix sim test --- src/frontend/src/session.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 0fbf4404fcdb6..28653c13835e3 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -261,7 +261,12 @@ impl FrontendEnv { .unwrap(); info!("advertise addr is {}", frontend_address); - let frontend_rpc_addr = opts.frontend_rpc_listener_addr.parse().unwrap(); + let addr: HostAddr = opts.frontend_rpc_listener_addr.parse().unwrap(); + // Use the host of advertise address for the frontend rpc address. + let frontend_rpc_addr = HostAddr { + host: frontend_address.host.clone(), + port: addr.port, + }; // Register in meta by calling `AddWorkerNode` RPC. // Use the rpc server address as the frontend address, since Meta needs to get frontend rpc // client based on this address. From ed91f7e87e8946a7b391da56b52480a419378761 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 12 Aug 2024 12:13:08 +0800 Subject: [PATCH 55/75] refactor frontend rpc addr impl --- proto/common.proto | 3 +++ proto/meta.proto | 3 +++ src/compute/src/server.rs | 1 + src/ctl/src/cmd_impl/meta/migration.rs | 1 + .../rw_catalog/rw_worker_nodes.rs | 22 ++++++++++++++--- src/frontend/src/session.rs | 20 +++++++++------- .../m20240726_063833_auto_schema_change.rs | 24 +++++++++++++++++++ src/meta/model_v2/src/worker_property.rs | 1 + src/meta/src/controller/cluster.rs | 4 +++- src/meta/src/hummock/test_utils.rs | 1 + src/meta/src/manager/cluster.rs | 3 ++- src/rpc_client/src/lib.rs | 16 +++++++++++-- 12 files changed, 83 insertions(+), 16 deletions(-) diff --git a/proto/common.proto b/proto/common.proto index d6c596ec4c497..1bc65b8c03ff2 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -54,6 +54,9 @@ message WorkerNode { bool is_streaming = 1; bool is_serving = 2; bool is_unschedulable = 3; + // Secondary host address for the worker node. + // This is used for frontend node to register its rpc address + string secondary_host = 4; } message Resource { string rw_version = 1; diff --git a/proto/meta.proto b/proto/meta.proto index 0371b5540a6da..a7bc4e68874b6 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -311,6 +311,9 @@ message AddWorkerNodeRequest { bool is_streaming = 2; bool is_serving = 3; bool is_unschedulable = 4; + // Secondary host address for the worker node. + // This is used for frontend node to register its rpc address + string secondary_host = 5; } common.WorkerType worker_type = 1; common.HostAddress host = 2; diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3270897c29d31..18771052bfe85 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -127,6 +127,7 @@ pub async fn compute_node_serve( is_streaming: opts.role.for_streaming(), is_serving: opts.role.for_serving(), is_unschedulable: false, + secondary_host: "".to_string(), }, &config.meta, ) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index ed82f57af2af1..7482cd86f5fa2 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -162,6 +162,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an is_serving: Set(pb_property.is_serving), is_unschedulable: Set(pb_property.is_unschedulable), parallelism: Set(worker.worker_node.parallelism() as _), + secondary_host: Set(pb_property.secondary_host.clone()), }; WorkerProperty::insert(property) .exec(&meta_store_sql.conn) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index 6c21f524e684e..1391bfa9148a6 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -15,6 +15,7 @@ use itertools::Itertools; use risingwave_common::types::{Fields, Timestamptz}; use risingwave_frontend_macro::system_catalog; +use risingwave_pb::common::WorkerType; use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; @@ -32,6 +33,7 @@ struct RwWorkerNode { is_streaming: Option, is_serving: Option, is_unschedulable: Option, + secondary_host: Option, rw_version: Option, system_total_memory_bytes: Option, system_total_cpu_cores: Option, @@ -49,6 +51,7 @@ async fn read_rw_worker_nodes_info(reader: &SysCatalogReaderImpl) -> Result Result for PbWorkerNode { is_streaming: p.is_streaming, is_serving: p.is_serving, is_unschedulable: p.is_unschedulable, + secondary_host: p.secondary_host.clone(), }), transactional_id: info.0.transaction_id.map(|id| id as _), resource: info.2.resource, @@ -670,7 +671,7 @@ impl ClusterControllerInner { }; let insert_res = Worker::insert(worker).exec(&txn).await?; let worker_id = insert_res.last_insert_id as WorkerId; - if r#type == PbWorkerType::ComputeNode { + if r#type == PbWorkerType::ComputeNode || r#type == PbWorkerType::Frontend { let property = worker_property::ActiveModel { worker_id: Set(worker_id), parallelism: Set(add_property @@ -680,6 +681,7 @@ impl ClusterControllerInner { is_streaming: Set(add_property.is_streaming), is_serving: Set(add_property.is_serving), is_unschedulable: Set(add_property.is_unschedulable), + secondary_host: Set(add_property.secondary_host), }; WorkerProperty::insert(property).exec(&txn).await?; } diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 9a764f871cd63..c12a8de165143 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -352,6 +352,7 @@ pub async fn setup_compute_env_with_metric( is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 93e50dec3706b..5748602440a18 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -491,11 +491,12 @@ impl ClusterManager { worker_type: WorkerType, worker_property: AddNodeProperty, ) -> Option { - if worker_type == WorkerType::ComputeNode { + if worker_type == WorkerType::ComputeNode || worker_type == WorkerType::Frontend { Some(Property { is_streaming: worker_property.is_streaming, is_serving: worker_property.is_serving, is_unschedulable: worker_property.is_unschedulable, + secondary_host: worker_property.secondary_host, }) } else { None diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 037ae44972fc8..cc422e3f94635 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -32,6 +32,7 @@ use std::any::type_name; use std::fmt::{Debug, Formatter}; use std::future::Future; use std::iter::repeat; +use std::str::FromStr; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -42,14 +43,16 @@ use futures::{Stream, StreamExt}; use moka::future::Cache; use rand::prelude::SliceRandom; use risingwave_common::util::addr::HostAddr; -use risingwave_pb::common::WorkerNode; +use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_pb::meta::heartbeat_request::extra_info; use tokio::sync::mpsc::{ channel, unbounded_channel, Receiver, Sender, UnboundedReceiver, UnboundedSender, }; pub mod error; + use error::Result; + mod compactor_client; mod compute_client; mod connector_client; @@ -129,7 +132,16 @@ where /// Gets the RPC client for the given node. If the connection is not established, a /// new client will be created and returned. pub async fn get(&self, node: &WorkerNode) -> Result { - let addr: HostAddr = node.get_host().unwrap().into(); + let addr = if node.get_type().unwrap() == WorkerType::Frontend { + let prop = node + .property + .as_ref() + .expect("frontend node property is missing"); + HostAddr::from_str(prop.secondary_host.as_str())? + } else { + node.get_host().unwrap().into() + }; + self.get_by_addr(addr).await } From 259c05575ed7a3bd4884c6c3cf9ab6f43d42712a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 12 Aug 2024 23:32:55 +0800 Subject: [PATCH 56/75] fix --- src/batch/src/worker_manager/worker_node_manager.rs | 2 ++ src/common/src/vnode_mapping/vnode_placement.rs | 1 + src/frontend/src/scheduler/distributed/query.rs | 3 +++ src/meta/src/controller/cluster.rs | 2 ++ src/meta/src/hummock/manager/tests.rs | 2 ++ src/meta/src/manager/cluster.rs | 4 ++++ src/meta/src/stream/stream_manager.rs | 1 + 7 files changed, 15 insertions(+) diff --git a/src/batch/src/worker_manager/worker_node_manager.rs b/src/batch/src/worker_manager/worker_node_manager.rs index c4f4829c36110..ee9eed558ad19 100644 --- a/src/batch/src/worker_manager/worker_node_manager.rs +++ b/src/batch/src/worker_manager/worker_node_manager.rs @@ -430,6 +430,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: true, + secondary_host: "".to_string(), }), transactional_id: Some(1), ..Default::default() @@ -444,6 +445,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: false, + secondary_host: "".to_string(), }), transactional_id: Some(2), ..Default::default() diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index aea08a1b74352..10cb93b9fc290 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -213,6 +213,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: false, + secondary_host: "".to_string(), }; let count_same_vnode_mapping = |wm1: &WorkerSlotMapping, wm2: &WorkerSlotMapping| { diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index d4e3e2925ff23..97c3bad24ca3c 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -681,6 +681,7 @@ pub(crate) mod tests { is_unschedulable: false, is_serving: true, is_streaming: true, + secondary_host: "".to_string(), }), transactional_id: Some(0), ..Default::default() @@ -698,6 +699,7 @@ pub(crate) mod tests { is_unschedulable: false, is_serving: true, is_streaming: true, + secondary_host: "".to_string(), }), transactional_id: Some(1), ..Default::default() @@ -715,6 +717,7 @@ pub(crate) mod tests { is_unschedulable: false, is_serving: true, is_streaming: true, + secondary_host: "".to_string(), }), transactional_id: Some(2), ..Default::default() diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index c8dab2c5caf04..8df302380a64a 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -928,6 +928,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }; let hosts = mock_worker_hosts_for_test(worker_count); let mut worker_ids = vec![]; @@ -1018,6 +1019,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }; let worker_id = cluster_ctl .add_worker( diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index a0780641a3f30..e963d4d86bbe0 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -402,6 +402,7 @@ async fn test_release_context_resource() { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) @@ -484,6 +485,7 @@ async fn test_hummock_manager_basic() { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 5748602440a18..bfaf2d7150382 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -808,6 +808,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) @@ -849,6 +850,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) @@ -871,6 +873,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) @@ -920,6 +923,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 050238e2dbc6d..8f124c6f12f56 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -1013,6 +1013,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) From df652358b3318324a3480ce494173eaab66290d1 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 15:55:00 +0800 Subject: [PATCH 57/75] add feature guard flag --- src/common/src/config.rs | 8 +++ src/stream/src/executor/actor.rs | 6 ++ .../src/executor/source/source_executor.rs | 60 ++++++++++++------- src/stream/src/task/stream_manager.rs | 2 + 4 files changed, 54 insertions(+), 22 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index eeb393edb4ab9..9ee8a7e52ec32 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1015,6 +1015,10 @@ pub struct StreamingDeveloperConfig { /// If not specified, the value of `server.connection_pool_size` will be used. #[serde(default = "default::developer::stream_exchange_connection_pool_size")] pub exchange_connection_pool_size: Option, + + /// A flag to allow disabling the auto schema change handling + #[serde(default = "default::developer::stream_enable_auto_schema_change")] + pub enable_auto_schema_change: bool, } /// The subsections `[batch.developer]`. @@ -1885,6 +1889,10 @@ pub mod default { pub fn enable_actor_tokio_metrics() -> bool { false } + + pub fn stream_enable_auto_schema_change() -> bool { + true + } } pub use crate::system_param::default as system; diff --git a/src/stream/src/executor/actor.rs b/src/stream/src/executor/actor.rs index 22eb9a1ff37f2..852514cb16e1c 100644 --- a/src/stream/src/executor/actor.rs +++ b/src/stream/src/executor/actor.rs @@ -21,6 +21,7 @@ use await_tree::InstrumentAwait; use futures::future::join_all; use hytra::TrAdder; use risingwave_common::catalog::TableId; +use risingwave_common::config::StreamingConfig; use risingwave_common::log::LogSuppresser; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::util::epoch::EpochPair; @@ -59,6 +60,8 @@ pub struct ActorContext { // Meta client. currently used for auto schema change. `None` for test only pub meta_client: Option, + + pub streaming_config: Arc, } pub type ActorContextRef = Arc; @@ -77,6 +80,7 @@ impl ActorContext { initial_dispatch_num: 1, related_subscriptions: HashMap::new(), meta_client: None, + streaming_config: Arc::new(StreamingConfig::default()), }) } @@ -87,6 +91,7 @@ impl ActorContext { initial_dispatch_num: usize, related_subscriptions: HashMap>, meta_client: Option, + streaming_config: Arc, ) -> ActorContextRef { Arc::new(Self { id: stream_actor.actor_id, @@ -99,6 +104,7 @@ impl ActorContext { initial_dispatch_num, related_subscriptions, meta_client, + streaming_config, }) } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index fb88cc9730787..6c951a8a28110 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -126,30 +126,39 @@ impl SourceExecutor { let (schema_change_tx, mut schema_change_rx) = tokio::sync::mpsc::channel::<(SchemaChangeEnvelope, oneshot::Sender<()>)>(16); - let meta_client = self.actor_ctx.meta_client.clone(); - // spawn a task to handle schema change event from source parser - let _join_handle = tokio::task::spawn(async move { - while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { - let table_names = schema_change.table_names(); - tracing::info!("recv a schema change event for tables: {:?}", table_names); - // TODO: retry on rpc error - if let Some(ref meta_client) = meta_client { - match meta_client - .auto_schema_change(schema_change.to_protobuf()) - .await - { - Ok(_) => { - tracing::info!("schema change success for tables: {:?}", table_names); - finish_tx.send(()).unwrap(); - } - Err(e) => { - tracing::error!(error = ?e.as_report(), "schema change error"); - finish_tx.send(()).unwrap(); + let schema_change_tx = if self.is_auto_schema_change_enable() { + let meta_client = self.actor_ctx.meta_client.clone(); + // spawn a task to handle schema change event from source parser + let _join_handle = tokio::task::spawn(async move { + while let Some((schema_change, finish_tx)) = schema_change_rx.recv().await { + let table_names = schema_change.table_names(); + tracing::info!("recv a schema change event for tables: {:?}", table_names); + // TODO: retry on rpc error + if let Some(ref meta_client) = meta_client { + match meta_client + .auto_schema_change(schema_change.to_protobuf()) + .await + { + Ok(_) => { + tracing::info!( + "schema change success for tables: {:?}", + table_names + ); + finish_tx.send(()).unwrap(); + } + Err(e) => { + tracing::error!(error = ?e.as_report(), "schema change error"); + finish_tx.send(()).unwrap(); + } } } } - } - }); + }); + Some(schema_change_tx) + } else { + info!("auto schema change is disabled in config"); + None + }; let source_ctx = SourceContext::new( self.actor_ctx.id, @@ -166,7 +175,7 @@ impl SourceExecutor { rate_limit: self.rate_limit_rps, }, source_desc.source.config.clone(), - Some(schema_change_tx), + schema_change_tx, ); let stream = source_desc .source @@ -177,6 +186,13 @@ impl SourceExecutor { Ok(apply_rate_limit(stream?, self.rate_limit_rps).boxed()) } + fn is_auto_schema_change_enable(&self) -> bool { + self.actor_ctx + .streaming_config + .developer + .enable_auto_schema_change + } + /// `source_id | source_name | actor_id | fragment_id` #[inline] fn get_metric_labels(&self) -> [String; 4] { diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index e6f9e17f6afda..61c4e887779f5 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -571,6 +571,7 @@ impl StreamActorManager { } = actor; let actor = actor.unwrap(); let actor_id = actor.actor_id; + let streaming_config = Arc::new(shared_context.config.clone()); let actor_context = ActorContext::create( &actor, self.env.total_mem_usage(), @@ -586,6 +587,7 @@ impl StreamActorManager { }) .collect(), self.env.meta_client().clone(), + streaming_config, ); let vnode_bitmap = actor.vnode_bitmap.as_ref().map(|b| b.into()); let expr_context = actor.expr_context.clone().unwrap(); From 766313ca21766365594e119331ee75ca47c9ae8f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 17:51:10 +0800 Subject: [PATCH 58/75] retry with grpc errors --- src/meta/service/src/ddl_service.rs | 5 +- src/rpc_client/src/frontend_client.rs | 79 +++++++++++++++++++++------ src/rpc_client/src/lib.rs | 19 +------ 3 files changed, 65 insertions(+), 38 deletions(-) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 7d6e299352362..81b90d0f2005a 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -966,6 +966,7 @@ impl DdlService for DdlServiceImpl { ); for table in tables { // send a request to the frontend to get the ReplaceTablePlan + // will retry with exponential backoff if the request fails let resp = client .get_table_replace_plan(GetTableReplacePlanRequest { database_id: table.database_id, @@ -973,8 +974,8 @@ impl DdlService for DdlServiceImpl { table_name: table.name, table_change: Some(table_change.clone()), }) - .await - .map_err(MetaError::from)?; + .await? + .into_inner(); if let Some(plan) = resp.replace_plan { plan.table diff --git a/src/rpc_client/src/frontend_client.rs b/src/rpc_client/src/frontend_client.rs index 01d8ac1bc4ce1..3fd02d99107e2 100644 --- a/src/rpc_client/src/frontend_client.rs +++ b/src/rpc_client/src/frontend_client.rs @@ -21,21 +21,20 @@ use risingwave_common::monitor::{EndpointExt, TcpConfig}; use risingwave_common::util::addr::HostAddr; use risingwave_pb::frontend_service::frontend_service_client::FrontendServiceClient; use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse}; +use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tonic::transport::Endpoint; +use tonic::Response; use crate::error::Result; use crate::tracing::{Channel, TracingInjectedChannelExt}; -use crate::{frontend_rpc_client_method_impl, RpcClient, RpcClientPool}; +use crate::{RpcClient, RpcClientPool}; -#[derive(Clone)] -pub struct FrontendClient(FrontendServiceClient); +const DEFAULT_RETRY_INTERVAL: u64 = 50; +const DEFAULT_RETRY_MAX_DELAY: Duration = Duration::from_secs(5); +const DEFAULT_RETRY_MAX_ATTEMPTS: usize = 10; -#[async_trait] -impl RpcClient for FrontendClient { - async fn new_client(host_addr: HostAddr) -> Result { - Self::new(host_addr).await - } -} +#[derive(Clone)] +struct FrontendClient(FrontendServiceClient); impl FrontendClient { async fn new(host_addr: HostAddr) -> Result { @@ -59,17 +58,61 @@ impl FrontendClient { } // similar to the stream_client used in the Meta node -pub type FrontendClientPool = RpcClientPool; +pub type FrontendClientPool = RpcClientPool; pub type FrontendClientPoolRef = Arc; -macro_rules! for_all_frontend_rpc { - ($macro:ident) => { - $macro! { - { 0, get_table_replace_plan, GetTableReplacePlanRequest, GetTableReplacePlanResponse } - } - }; +#[async_trait] +impl RpcClient for FrontendRetryClient { + async fn new_client(host_addr: HostAddr) -> Result { + Self::new(host_addr).await + } } -impl FrontendClient { - for_all_frontend_rpc! { frontend_rpc_client_method_impl } +#[derive(Clone)] +pub struct FrontendRetryClient { + client: FrontendClient, +} + +impl FrontendRetryClient { + async fn new(host_addr: HostAddr) -> Result { + let client = FrontendClient::new(host_addr).await?; + Ok(Self { client }) + } + + #[inline(always)] + fn get_retry_strategy() -> impl Iterator { + ExponentialBackoff::from_millis(DEFAULT_RETRY_INTERVAL) + .max_delay(DEFAULT_RETRY_MAX_DELAY) + .take(DEFAULT_RETRY_MAX_ATTEMPTS) + .map(jitter) + } + + fn should_retry(status: &tonic::Status) -> bool { + if status.code() == tonic::Code::Unavailable + || status.code() == tonic::Code::Unknown + || status.code() == tonic::Code::Unauthenticated + || status.code() == tonic::Code::Aborted + { + return true; + } + false + } + + pub async fn get_table_replace_plan( + &self, + request: GetTableReplacePlanRequest, + ) -> std::result::Result, tonic::Status> { + tokio_retry::RetryIf::spawn( + Self::get_retry_strategy(), + || async { + self.client + .to_owned() + .0 + .get_table_replace_plan(request.clone()) + .await + }, + Self::should_retry, + ) + .await + } } diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index cc422e3f94635..02336d5ee1b2e 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -66,7 +66,7 @@ mod tracing; pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; -pub use frontend_client::{FrontendClient, FrontendClientPool, FrontendClientPoolRef}; +pub use frontend_client::{FrontendClientPool, FrontendClientPoolRef}; pub use hummock_meta_client::{CompactionEventItem, HummockMetaClient}; pub use meta_client::{MetaClient, SinkCoordinationRpcClient}; use rw_futures_util::await_future_with_monitor_error_stream; @@ -210,23 +210,6 @@ macro_rules! meta_rpc_client_method_impl { } } -#[macro_export] -macro_rules! frontend_rpc_client_method_impl { - ($( { $client:tt, $fn_name:ident, $req:ty, $resp:ty }),*) => { - $( - pub async fn $fn_name(&self, request: $req) -> $crate::Result<$resp> { - Ok(self - .$client - .to_owned() - .$fn_name(request) - .await - .map_err($crate::error::RpcError::from_frontend_status)? - .into_inner()) - } - )* - } -} - pub const DEFAULT_BUFFER_SIZE: usize = 16; pub struct BidiStreamSender { From 756d84093eb9c657bfe46b796688a7949bb84d1d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 19:06:26 +0800 Subject: [PATCH 59/75] Revert "chore: bump `tonic` to v0.12 (#17889)" This reverts commit e96c39d2760bdc84c17059f58faec90be1fed1a7. --- Cargo.lock | 360 ++++++------------ Cargo.toml | 18 +- ci/docker-compose.yml | 1 + e2e_test/sink/kafka/protobuf.slt | 16 +- src/batch/Cargo.toml | 2 +- src/batch/src/executor/hash_agg.rs | 2 +- .../executor/join/distributed_lookup_join.rs | 7 +- src/batch/src/executor/join/hash_join.rs | 2 +- .../src/executor/join/local_lookup_join.rs | 4 +- src/batch/src/executor/mod.rs | 4 +- src/batch/src/executor/order_by.rs | 2 +- src/batch/src/executor/row_seq_scan.rs | 7 +- src/batch/src/spill/spill_op.rs | 2 +- src/batch/src/task/broadcast_channel.rs | 2 +- src/batch/src/task/task_execution.rs | 2 +- src/bench/Cargo.toml | 2 +- src/common/Cargo.toml | 2 +- src/common/common_service/Cargo.toml | 2 +- src/common/common_service/src/tracing.rs | 8 +- src/common/metrics/Cargo.toml | 14 +- src/common/metrics/src/monitor/connection.rs | 130 +------ .../src/vnode_mapping/vnode_placement.rs | 4 +- src/compute/Cargo.toml | 5 +- .../src/rpc/service/monitor_service.rs | 13 +- src/connector/Cargo.toml | 14 +- src/error/src/tonic.rs | 2 +- src/expr/impl/Cargo.toml | 4 +- src/frontend/Cargo.toml | 2 +- src/frontend/src/catalog/source_catalog.rs | 9 +- .../src/catalog/system_catalog/mod.rs | 2 +- src/frontend/src/observer/observer_manager.rs | 3 +- .../src/scheduler/distributed/stage.rs | 6 +- src/frontend/src/scheduler/snapshot.rs | 6 +- src/meta/Cargo.toml | 4 +- src/meta/node/src/server.rs | 26 +- src/meta/service/Cargo.toml | 2 +- src/meta/service/src/ddl_service.rs | 2 +- src/meta/src/barrier/mod.rs | 2 +- src/meta/src/controller/cluster.rs | 6 +- src/meta/src/hummock/manager/commit_epoch.rs | 2 +- src/meta/src/hummock/model/pinned_snapshot.rs | 2 +- src/meta/src/hummock/model/pinned_version.rs | 2 +- src/meta/src/manager/catalog/user.rs | 32 +- src/meta/src/rpc/intercept.rs | 8 +- src/object_store/Cargo.toml | 2 +- src/prost/build.rs | 2 +- src/prost/src/lib.rs | 1 - src/risedevtool/Cargo.toml | 2 +- src/rpc_client/Cargo.toml | 6 +- src/rpc_client/src/meta_client.rs | 2 +- src/rpc_client/src/tracing.rs | 50 ++- src/storage/backup/src/lib.rs | 2 +- .../compaction_group/hummock_version_ext.rs | 2 +- src/storage/hummock_sdk/src/time_travel.rs | 2 +- src/storage/hummock_sdk/src/version.rs | 16 +- src/stream/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- src/tests/state_cleaning_test/Cargo.toml | 2 +- src/utils/runtime/Cargo.toml | 2 +- 59 files changed, 329 insertions(+), 513 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7183c990bfce9..f1a6e60f2880d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1751,7 +1751,7 @@ dependencies = [ "http 1.1.0", "http-body 1.0.0", "http-body-util", - "hyper 1.4.1", + "hyper 1.1.0", "hyper-util", "itoa", "matchit", @@ -2791,22 +2791,22 @@ dependencies = [ [[package]] name = "console-api" -version = "0.8.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86ed14aa9c9f927213c6e4f3ef75faaad3406134efe84ba2cb7983431d5f0931" +checksum = "a257c22cd7e487dd4a13d413beabc512c5052f0bc048db0da6a84c3d8a6142fd" dependencies = [ "futures-core", - "prost 0.13.1", - "prost-types 0.13.1", - "tonic 0.12.1", + "prost 0.12.1", + "prost-types 0.12.1", + "tonic 0.11.0", "tracing-core", ] [[package]] name = "console-subscriber" -version = "0.4.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2e3a111a37f3333946ebf9da370ba5c5577b18eb342ec683eb488dd21980302" +checksum = "31c4cc54bae66f7d9188996404abdf7fdfa23034ef8e43478c8810828abad758" dependencies = [ "console-api", "crossbeam-channel", @@ -2814,15 +2814,14 @@ dependencies = [ "futures-task", "hdrhistogram", "humantime", - "hyper-util", - "prost 0.13.1", - "prost-types 0.13.1", + "prost 0.12.1", + "prost-types 0.12.1", "serde", "serde_json", "thread_local", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", - "tonic 0.12.1", + "tokio-stream", + "tonic 0.11.0", "tracing", "tracing-core", "tracing-subscriber", @@ -4437,15 +4436,15 @@ dependencies = [ [[package]] name = "etcd-client" -version = "0.14.0" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39bde3ce50a626efeb1caa9ab1083972d178bebb55ca627639c8ded507dfcbde" +checksum = "4ae697f3928e8c89ae6f4dcf788059f49fd01a76dc53e63628f5a33881f5715e" dependencies = [ - "http 1.1.0", - "prost 0.13.1", + "http 0.2.9", + "prost 0.12.1", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", - "tonic 0.12.1", + "tokio-stream", + "tonic 0.10.2", "tonic-build", "tower", "tower-service", @@ -5186,7 +5185,7 @@ dependencies = [ "thiserror", "time", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", "url", "yup-oauth2", ] @@ -5302,9 +5301,9 @@ dependencies = [ [[package]] name = "google-cloud-auth" -version = "0.16.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1112c453c2e155b3e683204ffff52bcc6d6495d04b68d9e90cd24161270c5058" +checksum = "e09ed5b2998bc8d0d3df09c859028210d4961b8fe779cfda8dc8ca4e83d5def2" dependencies = [ "async-trait", "base64 0.21.7", @@ -5324,9 +5323,9 @@ dependencies = [ [[package]] name = "google-cloud-bigquery" -version = "0.12.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "305cb7214d11b719e9f00f982c1ee1304c674f7a8dfc44a43b8bad3c909750c2" +checksum = "1e321c127945bb44a5cf5129c37530e2494b97afefe7f334a983ac754e40914e" dependencies = [ "anyhow", "arrow 50.0.0", @@ -5351,29 +5350,29 @@ dependencies = [ [[package]] name = "google-cloud-gax" -version = "0.19.0" +version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c3eaaad103912825594d674a4b1e556ccbb05a13a6cac17dcfd871997fb760a" +checksum = "8cb60314136e37de9e2a05ddb427b9c5a39c3d188de2e2f026c6af74425eef44" dependencies = [ "google-cloud-token", - "http 1.1.0", + "http 0.2.9", "thiserror", "tokio", "tokio-retry", - "tonic 0.12.1", + "tonic 0.10.2", "tower", "tracing", ] [[package]] name = "google-cloud-googleapis" -version = "0.15.0" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ae8ab26ef7c7c3f7dfb9cc3982293d031d8e78c85d00ddfb704b5c35aeff7c8" +checksum = "32cd184c52aa2619ac1b16ad8b5a752e91d25be88a8cf08eaec19777dfacbe54" dependencies = [ - "prost 0.13.1", - "prost-types 0.13.1", - "tonic 0.12.1", + "prost 0.12.1", + "prost-types 0.12.1", + "tonic 0.10.2", ] [[package]] @@ -5389,9 +5388,9 @@ dependencies = [ [[package]] name = "google-cloud-pubsub" -version = "0.28.1" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55ef73601dcec5ea144e59969e921d35d66000211603fee8023b7947af09248f" +checksum = "0a35e4a008db5cf01a5c03d3c67bd90b3cad77427ca949f3c8eddd90c4a3c932" dependencies = [ "async-channel 1.9.0", "async-stream", @@ -5399,7 +5398,7 @@ dependencies = [ "google-cloud-gax", "google-cloud-googleapis", "google-cloud-token", - "prost-types 0.13.1", + "prost-types 0.12.1", "thiserror", "tokio", "tokio-util", @@ -5408,9 +5407,9 @@ dependencies = [ [[package]] name = "google-cloud-token" -version = "0.1.2" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f49c12ba8b21d128a2ce8585955246977fbce4415f680ebf9199b6f9d6d725f" +checksum = "0fcd62eb34e3de2f085bcc33a09c3e17c4f65650f36d53eb328b00d63bcb536a" dependencies = [ "async-trait", ] @@ -5596,9 +5595,9 @@ checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" [[package]] name = "hermit-abi" -version = "0.3.9" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" +checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" [[package]] name = "hex" @@ -5761,9 +5760,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.4.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50dfd22e0e76d0f662d429a5f80fcaf3855009297eab6a0a9f8543834744ba05" +checksum = "fb5aa53871fc917b1a9ed87b683a5d86db645e23acb32c2e0785a353e522fb75" dependencies = [ "bytes", "futures-channel", @@ -5775,7 +5774,6 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "smallvec", "tokio", "want", ] @@ -5805,7 +5803,7 @@ checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" dependencies = [ "futures-util", "http 1.1.0", - "hyper 1.4.1", + "hyper 1.1.0", "hyper-util", "rustls 0.22.4", "rustls-pki-types", @@ -5826,19 +5824,6 @@ dependencies = [ "tokio-io-timeout", ] -[[package]] -name = "hyper-timeout" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3203a961e5c83b6f5498933e78b6b263e208c197b63e9c6c53cc82ffd3f63793" -dependencies = [ - "hyper 1.4.1", - "hyper-util", - "pin-project-lite", - "tokio", - "tower-service", -] - [[package]] name = "hyper-tls" version = "0.5.0" @@ -5860,7 +5845,7 @@ checksum = "70206fc6890eaca9fde8a0bf71caa2ddfc9fe045ac9e5c70df101a7dbde866e0" dependencies = [ "bytes", "http-body-util", - "hyper 1.4.1", + "hyper 1.1.0", "hyper-util", "native-tls", "tokio", @@ -5870,16 +5855,16 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.6" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ab92f4f49ee4fb4f997c784b7a2e0fa70050211e0b6a287f898c3c9785ca956" +checksum = "ca38ef113da30126bbff9cd1705f9273e15d45498615d138b0c20279ac7a76aa" dependencies = [ "bytes", "futures-channel", "futures-util", "http 1.1.0", "http-body 1.0.0", - "hyper 1.4.1", + "hyper 1.1.0", "pin-project-lite", "socket2 0.5.6", "tokio", @@ -6829,13 +6814,13 @@ dependencies = [ [[package]] name = "madsim-etcd-client" -version = "0.6.0+0.14.0" +version = "0.4.0+0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8edcf23498cb590e415ce2ba6c7f186c7aa3340e7aa716ddddb34faf0a9ffdfb" +checksum = "02b4b5de48bb7f3f7eae0bca62b3ed0b7d714b1b273d7347329b92c3a2eef113" dependencies = [ "etcd-client", "futures-util", - "http 1.1.0", + "http 0.2.9", "madsim", "serde", "serde_with 3.8.0", @@ -6843,7 +6828,7 @@ dependencies = [ "thiserror", "tokio", "toml 0.8.12", - "tonic 0.12.1", + "tonic 0.10.2", "tracing", ] @@ -6896,29 +6881,29 @@ dependencies = [ [[package]] name = "madsim-tonic" -version = "0.5.1+0.12.0" +version = "0.4.1+0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61c668c82f0c2aca7ffed3235047f2539e6e41278c7c47a822999f3b7a067887" +checksum = "813977c7870103e113a0332d97731f961bc48aaa8860edd318ef7d7754214436" dependencies = [ "async-stream", "chrono", "futures-util", "madsim", "tokio", - "tonic 0.12.1", + "tonic 0.10.2", "tower", "tracing", ] [[package]] name = "madsim-tonic-build" -version = "0.5.0+0.12.0" +version = "0.4.2+0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f271a476bbaa9d2139e1e1a5beb869c6119e805a0b67ad2b2857e4a8785b111a" +checksum = "4a2ad2776ba20221ccbe4e136e2fa0f7ab90eebd608373177f3e74a198a288ec" dependencies = [ "prettyplease 0.2.15", "proc-macro2", - "prost-build 0.13.1", + "prost-build 0.12.1", "quote", "syn 2.0.66", "tonic-build", @@ -7947,7 +7932,7 @@ dependencies = [ "rand", "thiserror", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", ] [[package]] @@ -8000,20 +7985,20 @@ dependencies = [ [[package]] name = "otlp-embedded" version = "0.0.1" -source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=e6cd165b9bc85783b42c106e99186b86b73e3507#e6cd165b9bc85783b42c106e99186b86b73e3507" +source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=492c244e0be91feb659c0cd48a624bbd96045a33#492c244e0be91feb659c0cd48a624bbd96045a33" dependencies = [ "axum 0.7.4", "datasize", "hex", - "itertools 0.13.0", - "prost 0.13.1", + "itertools 0.12.1", + "madsim-tonic", + "madsim-tonic-build", + "prost 0.12.1", "rust-embed", "schnellru", "serde", "serde_json", "tokio", - "tonic 0.12.1", - "tonic-build", "tracing", ] @@ -8950,16 +8935,6 @@ dependencies = [ "prost-derive 0.12.1", ] -[[package]] -name = "prost" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13db3d3fde688c61e2446b4d843bc27a7e8af269a69440c0308021dc92333cc" -dependencies = [ - "bytes", - "prost-derive 0.13.1", -] - [[package]] name = "prost-build" version = "0.11.9" @@ -9004,27 +8979,6 @@ dependencies = [ "which", ] -[[package]] -name = "prost-build" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bb182580f71dd070f88d01ce3de9f4da5021db7115d2e1c3605a754153b77c1" -dependencies = [ - "bytes", - "heck 0.5.0", - "itertools 0.13.0", - "log", - "multimap 0.10.0", - "once_cell", - "petgraph", - "prettyplease 0.2.15", - "prost 0.13.1", - "prost-types 0.13.1", - "regex", - "syn 2.0.66", - "tempfile", -] - [[package]] name = "prost-derive" version = "0.11.9" @@ -9051,19 +9005,6 @@ dependencies = [ "syn 2.0.66", ] -[[package]] -name = "prost-derive" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18bec9b0adc4eba778b33684b7ba3e7137789434769ee3ce3930463ef904cfca" -dependencies = [ - "anyhow", - "itertools 0.13.0", - "proc-macro2", - "quote", - "syn 2.0.66", -] - [[package]] name = "prost-helpers" version = "0.1.0" @@ -9075,13 +9016,13 @@ dependencies = [ [[package]] name = "prost-reflect" -version = "0.14.0" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55a6a9143ae25c25fa7b6a48d6cc08b10785372060009c25140a4e7c340e95af" +checksum = "9ae9372e3227f3685376a0836e5c248611eafc95a0be900d44bc6cdf225b700f" dependencies = [ "once_cell", - "prost 0.13.1", - "prost-types 0.13.1", + "prost 0.12.1", + "prost-types 0.12.1", ] [[package]] @@ -9102,15 +9043,6 @@ dependencies = [ "prost 0.12.1", ] -[[package]] -name = "prost-types" -version = "0.13.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cee5168b05f49d4b0ca581206eb14a7b22fafd963efe729ac48eb03266e25cc2" -dependencies = [ - "prost 0.13.1", -] - [[package]] name = "protobuf" version = "2.28.0" @@ -9228,7 +9160,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.12.1", + "parking_lot 0.11.2", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -9691,7 +9623,7 @@ dependencies = [ "http 1.1.0", "http-body 1.0.0", "http-body-util", - "hyper 1.4.1", + "hyper 1.1.0", "hyper-rustls 0.26.0", "hyper-tls 0.6.0", "hyper-util", @@ -9881,7 +9813,7 @@ dependencies = [ "bytes", "itertools 0.12.1", "parking_lot 0.12.1", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_meta_model_v2", @@ -9923,7 +9855,7 @@ dependencies = [ "parquet 52.0.0", "paste", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_common", "risingwave_common_estimate_size", @@ -9943,7 +9875,7 @@ dependencies = [ "thiserror-ext", "tikv-jemallocator", "tokio-metrics", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tokio-util", "tracing", "twox-hash", @@ -9986,7 +9918,7 @@ dependencies = [ "serde", "serde_yaml", "thiserror-ext", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "toml 0.8.12", "tracing", "tracing-subscriber", @@ -10092,7 +10024,7 @@ dependencies = [ "governor", "hashbrown 0.14.3", "hex", - "http 1.1.0", + "http 0.2.9", "http-body 0.4.5", "humantime", "hytra", @@ -10118,7 +10050,7 @@ dependencies = [ "pretty_assertions", "procfs 0.16.0", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "regex", "reqwest 0.12.4", @@ -10200,18 +10132,14 @@ dependencies = [ name = "risingwave_common_metrics" version = "1.11.0-alpha" dependencies = [ - "auto_impl", "bytes", "clap", "darwin-libproc", "easy-ext", "futures", "http 0.2.9", - "http 1.1.0", - "http-body 1.0.0", + "http-body 0.4.5", "hyper 0.14.27", - "hyper 1.4.1", - "hyper-util", "hytra", "itertools 0.12.1", "libc", @@ -10252,7 +10180,7 @@ dependencies = [ "anyhow", "bincode 1.3.3", "parking_lot 0.12.1", - "prost 0.13.1", + "prost 0.12.1", "risingwave_pb", "serde", "thiserror", @@ -10267,7 +10195,7 @@ dependencies = [ "async-trait", "axum 0.7.4", "futures", - "http 1.1.0", + "hyper 0.14.27", "madsim-tokio", "madsim-tonic", "prometheus", @@ -10321,7 +10249,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "parking_lot 0.12.1", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_common_heap_profiling", "risingwave_common_service", @@ -10347,15 +10275,14 @@ dependencies = [ "foyer", "futures", "futures-async-stream", - "http 1.1.0", - "hyper 1.4.1", + "hyper 0.14.27", "itertools 0.12.1", "madsim-tokio", "madsim-tonic", "maplit", "pprof", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_batch", "risingwave_common", @@ -10374,7 +10301,7 @@ dependencies = [ "tempfile", "thiserror-ext", "tikv-jemalloc-ctl", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tower", "tracing", "uuid", @@ -10461,10 +10388,10 @@ dependencies = [ "postgres-openssl", "pretty_assertions", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "prost-build 0.12.1", "prost-reflect", - "prost-types 0.13.1", + "prost-types 0.12.1", "protobuf-native", "protobuf-src", "pulsar", @@ -10504,7 +10431,7 @@ dependencies = [ "time", "tokio-postgres", "tokio-retry", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tokio-util", "tracing", "tracing-subscriber", @@ -10562,7 +10489,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", - "prost 0.13.1", + "prost 0.12.1", "regex", "risingwave_common", "risingwave_connector", @@ -10802,7 +10729,7 @@ dependencies = [ "pretty-xmlish", "pretty_assertions", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_batch", "risingwave_common", @@ -10829,7 +10756,7 @@ dependencies = [ "tempfile", "thiserror", "thiserror-ext", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tracing", "uuid", "workspace-hack", @@ -10854,7 +10781,7 @@ dependencies = [ "hex", "itertools 0.12.1", "parse-display", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_common_estimate_size", "risingwave_pb", @@ -10912,7 +10839,7 @@ dependencies = [ "madsim-tokio", "mockall", "parking_lot 0.12.1", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_pb", @@ -10931,7 +10858,7 @@ dependencies = [ "futures", "jni", "madsim-tokio", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_expr", "risingwave_hummock_sdk", @@ -10961,7 +10888,7 @@ dependencies = [ "jni", "madsim-tokio", "paste", - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_expr", "risingwave_hummock_sdk", @@ -11028,7 +10955,7 @@ dependencies = [ "function_name", "futures", "hex", - "http 1.1.0", + "hyper 0.14.27", "itertools 0.12.1", "jsonbb", "madsim-etcd-client", @@ -11043,7 +10970,7 @@ dependencies = [ "parking_lot 0.12.1", "prometheus", "prometheus-http-query", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_backup", "risingwave_common", @@ -11070,7 +10997,7 @@ dependencies = [ "thiserror", "thiserror-ext", "tokio-retry", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tower", "tower-http", "tracing", @@ -11116,7 +11043,7 @@ dependencies = [ name = "risingwave_meta_model_v2" version = "1.11.0-alpha" dependencies = [ - "prost 0.13.1", + "prost 0.12.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_pb", @@ -11170,7 +11097,7 @@ dependencies = [ "itertools 0.12.1", "madsim-tokio", "madsim-tonic", - "prost 0.13.1", + "prost 0.12.1", "rand", "regex", "risingwave_common", @@ -11183,7 +11110,7 @@ dependencies = [ "serde_json", "sync-point", "thiserror-ext", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tracing", "workspace-hack", ] @@ -11235,8 +11162,8 @@ dependencies = [ "madsim-tonic-build", "pbjson", "pbjson-build", - "prost 0.13.1", - "prost-build 0.13.1", + "prost 0.12.1", + "prost-build 0.12.1", "prost-helpers", "risingwave_error", "serde", @@ -11291,8 +11218,8 @@ dependencies = [ "easy-ext", "either", "futures", - "http 1.1.0", - "hyper 1.4.1", + "http 0.2.9", + "hyper 0.14.27", "itertools 0.12.1", "lru 0.7.6", "madsim-tokio", @@ -11309,7 +11236,7 @@ dependencies = [ "thiserror", "thiserror-ext", "tokio-retry", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tower", "tracing", "url", @@ -11395,7 +11322,7 @@ dependencies = [ "tempfile", "tikv-jemallocator", "tokio-postgres", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tracing", "tracing-subscriber", ] @@ -11463,7 +11390,7 @@ dependencies = [ "serde", "serde_with 3.8.0", "tokio-postgres", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "toml 0.8.12", "tracing", "workspace-hack", @@ -11510,7 +11437,7 @@ dependencies = [ "parking_lot 0.12.1", "procfs 0.16.0", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_backup", "risingwave_common", @@ -11579,7 +11506,7 @@ dependencies = [ "pin-project", "prehash", "prometheus", - "prost 0.13.1", + "prost 0.12.1", "rand", "risingwave_common", "risingwave_common_estimate_size", @@ -11604,7 +11531,7 @@ dependencies = [ "thiserror-ext", "tokio-metrics", "tokio-retry", - "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", + "tokio-stream", "tracing", "tracing-test", "workspace-hack", @@ -12150,9 +12077,9 @@ dependencies = [ [[package]] name = "schnellru" -version = "0.2.3" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9a8ef13a93c54d20580de1e5c413e624e53121d42fc7e2c11d10ef7f8b02367" +checksum = "772575a524feeb803e5b0fcbc6dd9f367e579488197c94c6e4023aad2305774d" dependencies = [ "ahash 0.8.11", "cfg-if", @@ -13195,7 +13122,7 @@ dependencies = [ "thiserror", "time", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", "tracing", "url", "uuid", @@ -13949,7 +13876,7 @@ dependencies = [ "futures-util", "pin-project-lite", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", ] [[package]] @@ -14043,19 +13970,8 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "git+https://github.com/madsim-rs/tokio.git?rev=0dd1055#0dd105567b323c863c29f794d2221ed588956d8d" +version = "0.1.14" +source = "git+https://github.com/madsim-rs/tokio.git?rev=fe39bb8e#fe39bb8e8ab0ed96ee1b4477ab5508c20ce017fb" dependencies = [ "futures-core", "madsim-tokio", @@ -14158,11 +14074,12 @@ dependencies = [ "axum 0.6.20", "base64 0.21.7", "bytes", + "flate2", "h2 0.3.26", "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-timeout 0.4.1", + "hyper-timeout", "percent-encoding", "pin-project", "prost 0.12.1", @@ -14170,11 +14087,12 @@ dependencies = [ "rustls-pemfile 1.0.4", "tokio", "tokio-rustls 0.24.1", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", "tower", "tower-layer", "tower-service", "tracing", + "webpki-roots 0.25.2", ] [[package]] @@ -14192,61 +14110,27 @@ dependencies = [ "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-timeout 0.4.1", + "hyper-timeout", "percent-encoding", "pin-project", "prost 0.12.1", "tokio", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tokio-stream", "tower", "tower-layer", "tower-service", "tracing", ] -[[package]] -name = "tonic" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38659f4a91aba8598d27821589f5db7dddd94601e7a01b1e485a50e5484c7401" -dependencies = [ - "async-stream", - "async-trait", - "axum 0.7.4", - "base64 0.22.0", - "bytes", - "flate2", - "h2 0.4.4", - "http 1.1.0", - "http-body 1.0.0", - "http-body-util", - "hyper 1.4.1", - "hyper-timeout 0.5.1", - "hyper-util", - "percent-encoding", - "pin-project", - "prost 0.13.1", - "rustls-pemfile 2.1.1", - "socket2 0.5.6", - "tokio", - "tokio-rustls 0.26.0", - "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", - "tower", - "tower-layer", - "tower-service", - "tracing", - "webpki-roots 0.26.1", -] - [[package]] name = "tonic-build" -version = "0.12.1" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "568392c5a2bd0020723e3f387891176aabafe36fd9fcd074ad309dfa0c8eb964" +checksum = "9d021fc044c18582b9a2408cd0dd05b1596e3ecdb5c4df822bb0183545683889" dependencies = [ "prettyplease 0.2.15", "proc-macro2", - "prost-build 0.13.1", + "prost-build 0.12.1", "quote", "syn 2.0.66", ] diff --git a/Cargo.toml b/Cargo.toml index 5bfab4feb27fb..9b07142dcf021 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -121,7 +121,7 @@ aws-smithy-types = { version = "1", default-features = false, features = [ aws-endpoint = "0.60" aws-types = "1" axum = "=0.7.4" # TODO: 0.7.5+ does not work with current toolchain -etcd-client = { package = "madsim-etcd-client", version = "0.6" } +etcd-client = { package = "madsim-etcd-client", version = "0.4" } futures-async-stream = "0.2.9" hytra = "0.1" rdkafka = { package = "madsim-rdkafka", version = "0.4.1", features = [ @@ -129,11 +129,11 @@ rdkafka = { package = "madsim-rdkafka", version = "0.4.1", features = [ ] } hashbrown = { version = "0.14", features = ["ahash", "inline-more", "nightly"] } criterion = { version = "0.5", features = ["async_futures"] } -tonic = { package = "madsim-tonic", version = "0.5.1" } -tonic-build = { package = "madsim-tonic-build", version = "0.5" } -otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "e6cd165b9bc85783b42c106e99186b86b73e3507" } -prost = { version = "0.13" } -prost-build = { version = "0.13" } +tonic = { package = "madsim-tonic", version = "0.4.1" } +tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } +otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "492c244e0be91feb659c0cd48a624bbd96045a33" } +prost = { version = "0.12" } +prost-build = { version = "0.12" } icelake = { git = "https://github.com/risingwavelabs/icelake.git", rev = "1860eb315183a5f3f72b4097c1e40d49407f8373", features = [ "prometheus", ] } @@ -180,7 +180,6 @@ tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git" "profiling", "stats", ], rev = "64a2d9" } -# TODO(http-bump): bump to use tonic 0.12 once minitrace-opentelemetry is updated opentelemetry = "0.23" opentelemetry-otlp = "0.16" opentelemetry_sdk = { version = "0.23", default-features = false } @@ -196,7 +195,6 @@ sea-orm = { version = "0.12.14", features = [ "runtime-tokio-native-tls", ] } sqlx = "0.7" -tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055", features = ["net", "fs"] } tokio-util = "0.7" tracing-opentelemetry = "0.24" rand = { version = "0.8", features = ["small_rng"] } @@ -337,9 +335,7 @@ opt-level = 2 # Patch third-party crates for deterministic simulation. quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "948bdc3" } getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae" } -# Don't patch `tokio-stream`, but only use the madsim version for **direct** dependencies. -# Imagine an unpatched dependency depends on the original `tokio` and the patched `tokio-stream`. -# tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055" } +tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "fe39bb8e" } tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = "95e2fd3" } tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = "05b33b4" } diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 4b1954ff5ae2c..b230e0381d147 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -266,6 +266,7 @@ services: SCHEMA_REGISTRY_HOST_NAME: schemaregistry SCHEMA_REGISTRY_LISTENERS: http://schemaregistry:8082 SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: message_queue:29092 + SCHEMA_REGISTRY_DEBUG: 'true' pulsar-server: container_name: pulsar-server diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 5f032ba32f8dc..0abd242e3c79d 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -201,19 +201,25 @@ format plain encode protobuf ( message = 'recursive.AllTypes'); statement ok -drop table from_kafka cascade; +drop sink sink_upsert; statement ok -drop table from_kafka_csr_trivial cascade; +drop sink sink_csr_nested; statement ok -drop table from_kafka_csr_nested cascade; +drop sink sink_csr_trivial; statement ok -drop table from_kafka_raw cascade; +drop sink sink0; statement ok -drop table into_kafka cascade; +drop table into_kafka; + +statement ok +drop table from_kafka_raw; system ok rpk topic delete test-rw-sink-upsert-protobuf + +statement ok +drop table from_kafka; diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 403eb864229d3..099ae9019afcf 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -63,7 +63,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "fs", ] } tokio-metrics = "0.3.0" -tokio-stream = { workspace = true } +tokio-stream = "0.1" tokio-util = { workspace = true } tonic = { workspace = true } tracing = "0.1" diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index 00073217f7ead..d69d4fbc8b174 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -20,6 +20,7 @@ use bytes::Bytes; use futures_async_stream::try_stream; use hashbrown::hash_map::Entry; use itertools::Itertools; +use prost::Message; use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{Field, Schema}; @@ -34,7 +35,6 @@ use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; use risingwave_pb::data::DataChunk as PbDataChunk; -use risingwave_pb::Message; use crate::error::{BatchError, Result}; use crate::executor::aggregation::build as build_agg; diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 1068ffd7f3349..f5ad5ab5ed984 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -354,7 +354,10 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let pk_prefix = OwnedRow::new(scan_range.eq_conds); if self.lookup_prefix_len == self.table.pk_indices().len() { - let row = self.table.get_row(&pk_prefix, self.epoch.into()).await?; + let row = self + .table + .get_row(&pk_prefix, self.epoch.clone().into()) + .await?; if let Some(row) = row { self.row_list.push(row); @@ -363,7 +366,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let iter = self .table .batch_iter_with_pk_bounds( - self.epoch.into(), + self.epoch.clone().into(), &pk_prefix, .., false, diff --git a/src/batch/src/executor/join/hash_join.rs b/src/batch/src/executor/join/hash_join.rs index 3bfb583d6459d..026f03fb65deb 100644 --- a/src/batch/src/executor/join/hash_join.rs +++ b/src/batch/src/executor/join/hash_join.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use bytes::Bytes; use futures_async_stream::try_stream; use itertools::Itertools; +use prost::Message; use risingwave_common::array::{Array, DataChunk, RowRef}; use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::Schema; @@ -33,7 +34,6 @@ use risingwave_common_estimate_size::EstimateSize; use risingwave_expr::expr::{build_from_prost, BoxedExpression, Expression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; -use risingwave_pb::Message; use super::{ChunkedData, JoinType, RowId}; use crate::error::{BatchError, Result}; diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index a3be00fc39a22..7fcaba71a9c3b 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -134,7 +134,7 @@ impl InnerSideExecutorBuilder { ..Default::default() }), }), - epoch: Some(self.epoch), + epoch: Some(self.epoch.clone()), tracing_context: TracingContext::from_current_span().to_protobuf(), }; @@ -237,7 +237,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder &plan_node, &task_id, self.context.clone(), - self.epoch, + self.epoch.clone(), self.shutdown_rx.clone(), ); diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index 80dc57b4f3620..3a64901c64a04 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -174,7 +174,7 @@ impl<'a, C: Clone> ExecutorBuilder<'a, C> { plan_node, self.task_id, self.context.clone(), - self.epoch, + self.epoch.clone(), self.shutdown_rx.clone(), ) } @@ -188,7 +188,7 @@ impl<'a, C: Clone> ExecutorBuilder<'a, C> { } pub fn epoch(&self) -> BatchQueryEpoch { - self.epoch + self.epoch.clone() } } diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index ad7cc13992346..3f8c8e106c78f 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use futures_async_stream::try_stream; use itertools::Itertools; +use prost::Message; use risingwave_common::array::DataChunk; use risingwave_common::catalog::Schema; use risingwave_common::memory::MemoryContext; @@ -27,7 +28,6 @@ use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common_estimate_size::EstimateSize; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; -use risingwave_pb::Message; use super::{ BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder, diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index b897dbd813787..b8287147c6750 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -237,7 +237,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let ordered = seq_scan_node.ordered; - let epoch = source.epoch; + let epoch = source.epoch.clone(); let limit = seq_scan_node.limit; let as_of = seq_scan_node .as_of @@ -341,7 +341,8 @@ impl RowSeqScanExecutor { for point_get in point_gets { let table = table.clone(); if let Some(row) = - Self::execute_point_get(table, point_get, query_epoch, histogram.clone()).await? + Self::execute_point_get(table, point_get, query_epoch.clone(), histogram.clone()) + .await? { if let Some(chunk) = data_chunk_builder.append_one_row(row) { returned += chunk.cardinality() as u64; @@ -372,7 +373,7 @@ impl RowSeqScanExecutor { table.clone(), range, ordered, - query_epoch, + query_epoch.clone(), chunk_size, limit, histogram.clone(), diff --git a/src/batch/src/spill/spill_op.rs b/src/batch/src/spill/spill_op.rs index b3e842a269ec7..237ee3baf0099 100644 --- a/src/batch/src/spill/spill_op.rs +++ b/src/batch/src/spill/spill_op.rs @@ -22,9 +22,9 @@ use futures_util::AsyncReadExt; use opendal::layers::RetryLayer; use opendal::services::{Fs, Memory}; use opendal::Operator; +use prost::Message; use risingwave_common::array::DataChunk; use risingwave_pb::data::DataChunk as PbDataChunk; -use risingwave_pb::Message; use thiserror_ext::AsReport; use tokio::sync::Mutex; use twox_hash::XxHash64; diff --git a/src/batch/src/task/broadcast_channel.rs b/src/batch/src/task/broadcast_channel.rs index 9781e38e7d7f6..d66eda7d7d620 100644 --- a/src/batch/src/task/broadcast_channel.rs +++ b/src/batch/src/task/broadcast_channel.rs @@ -86,7 +86,7 @@ pub fn new_broadcast_channel( output_channel_size: usize, ) -> (ChanSenderImpl, Vec) { let broadcast_info = match shuffle.distribution { - Some(exchange_info::Distribution::BroadcastInfo(ref v)) => *v, + Some(exchange_info::Distribution::BroadcastInfo(ref v)) => v.clone(), _ => BroadcastInfo::default(), }; diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 7186ced55febd..4536dad1c031f 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -393,7 +393,7 @@ impl BatchTaskExecution { self.plan.root.as_ref().unwrap(), &self.task_id, self.context.clone(), - self.epoch, + self.epoch.clone(), self.shutdown_rx.clone(), ) .build(), diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 43451ebaeb9d1..d451ef46ef838 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -50,7 +50,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "time", "signal", ] } -tokio-stream = { workspace = true } +tokio-stream = "0.1" toml = "0.8" tracing = "0.1" tracing-subscriber = "0.3.17" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 2cc1d81f1a38d..a117dce645ae6 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -55,7 +55,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } governor = { version = "0.6", default-features = false, features = ["std"] } hashbrown = "0.14" hex = "0.4.3" -http = "1" +http = "0.2" humantime = "2.1" hytra = { workspace = true } itertools = { workspace = true } diff --git a/src/common/common_service/Cargo.toml b/src/common/common_service/Cargo.toml index 87206ab7cbc1d..cb43702f3f9e6 100644 --- a/src/common/common_service/Cargo.toml +++ b/src/common/common_service/Cargo.toml @@ -18,7 +18,7 @@ normal = ["workspace-hack"] async-trait = "0.1" axum = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } -http = "1" +hyper = "0.14" # required by tonic prometheus = { version = "0.13" } risingwave_common = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/common/common_service/src/tracing.rs b/src/common/common_service/src/tracing.rs index de6f43bbf33f3..3ee4a64231c29 100644 --- a/src/common/common_service/src/tracing.rs +++ b/src/common/common_service/src/tracing.rs @@ -15,8 +15,8 @@ use std::task::{Context, Poll}; use futures::Future; +use hyper::Body; use risingwave_common::util::tracing::TracingContext; -use tonic::body::BoxBody; use tower::{Layer, Service}; use tracing::Instrument; @@ -49,9 +49,9 @@ pub struct TracingExtract { inner: S, } -impl Service> for TracingExtract +impl Service> for TracingExtract where - S: Service> + Clone + Send + 'static, + S: Service> + Clone + Send + 'static, S::Future: Send + 'static, { type Error = S::Error; @@ -63,7 +63,7 @@ where self.inner.poll_ready(cx) } - fn call(&mut self, req: http::Request) -> Self::Future { + fn call(&mut self, req: hyper::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary diff --git a/src/common/metrics/Cargo.toml b/src/common/metrics/Cargo.toml index 0c32b557cebb2..4f3e8b20936b2 100644 --- a/src/common/metrics/Cargo.toml +++ b/src/common/metrics/Cargo.toml @@ -15,16 +15,12 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -auto_impl = "1" bytes = "1" clap = { workspace = true } easy-ext = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -http = "1" -http-02 = { package = "http", version = "0.2" } -hyper = { version = "1" } -hyper-014 = { package = "hyper", version = "0.14" } -hyper-util = { version = "0.1", features = ["client-legacy"] } +http = "0.2" +hyper = { version = "0.14", features = ["client"] } # used by tonic hytra = { workspace = true } itertools = { workspace = true } parking_lot = { workspace = true } @@ -36,13 +32,13 @@ serde = { version = "1", features = ["derive"] } thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio" } tonic = { workspace = true } -tower-layer = "0.3.2" -tower-service = "0.3.2" tracing = "0.1" tracing-subscriber = "0.3.17" [target.'cfg(not(madsim))'.dependencies] -http-body = "1" +http-body = "0.4.5" +tower-layer = "0.3.2" +tower-service = "0.3.2" [target.'cfg(target_os = "linux")'.dependencies] procfs = { version = "0.16", default-features = false } libc = "0.2" diff --git a/src/common/metrics/src/monitor/connection.rs b/src/common/metrics/src/monitor/connection.rs index aa7c8c8d4baa3..e5774a3f16d7d 100644 --- a/src/common/metrics/src/monitor/connection.rs +++ b/src/common/metrics/src/monitor/connection.rs @@ -24,9 +24,10 @@ use std::time::Duration; use futures::FutureExt; use http::Uri; -use hyper_util::client::legacy::connect::dns::{GaiAddrs, GaiFuture, GaiResolver, Name}; -use hyper_util::client::legacy::connect::{Connected, Connection, HttpConnector}; -use hyper_util::rt::TokioIo; +use hyper::client::connect::dns::{GaiAddrs, GaiFuture, GaiResolver, Name}; +use hyper::client::connect::Connection; +use hyper::client::HttpConnector; +use hyper::service::Service; use itertools::Itertools; use pin_project_lite::pin_project; use prometheus::{ @@ -36,13 +37,11 @@ use prometheus::{ use thiserror_ext::AsReport; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; use tonic::transport::{Channel, Endpoint}; -use tower_service::Service; use tracing::{debug, info, warn}; use crate::monitor::GLOBAL_METRICS_REGISTRY; use crate::{register_guarded_int_counter_vec_with_registry, LabelGuardedIntCounterVec}; -#[auto_impl::auto_impl(&mut)] pub trait MonitorAsyncReadWrite { fn on_read(&mut self, _size: usize) {} fn on_eof(&mut self) {} @@ -75,14 +74,6 @@ impl MonitoredConnection { let this = this.project(); (this.inner, this.monitor) } - - /// Delegate async read/write traits between tokio and hyper. - fn hyper_tokio_delegate( - self: Pin<&mut Self>, - ) -> TokioIo>, &mut M>> { - let (inner, monitor) = MonitoredConnection::project_into(self); - TokioIo::new(MonitoredConnection::new(TokioIo::new(inner), monitor)) - } } impl AsyncRead for MonitoredConnection { @@ -121,16 +112,6 @@ impl AsyncRead for MonitoredConnection hyper::rt::Read for MonitoredConnection { - fn poll_read( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: hyper::rt::ReadBufCursor<'_>, - ) -> Poll> { - hyper::rt::Read::poll_read(std::pin::pin!(self.hyper_tokio_delegate()), cx, buf) - } -} - impl AsyncWrite for MonitoredConnection { fn poll_write( self: Pin<&mut Self>, @@ -205,41 +186,8 @@ impl AsyncWrite for MonitoredConnection } } -impl hyper::rt::Write for MonitoredConnection { - fn poll_write( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - buf: &[u8], - ) -> Poll> { - hyper::rt::Write::poll_write(std::pin::pin!(self.hyper_tokio_delegate()), cx, buf) - } - - fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - hyper::rt::Write::poll_flush(std::pin::pin!(self.hyper_tokio_delegate()), cx) - } - - fn poll_shutdown( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - hyper::rt::Write::poll_shutdown(std::pin::pin!(self.hyper_tokio_delegate()), cx) - } - - fn is_write_vectored(&self) -> bool { - self.inner.is_write_vectored() - } - - fn poll_write_vectored( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - bufs: &[std::io::IoSlice<'_>], - ) -> Poll> { - hyper::rt::Write::poll_write_vectored(std::pin::pin!(self.hyper_tokio_delegate()), cx, bufs) - } -} - impl Connection for MonitoredConnection { - fn connected(&self) -> Connected { + fn connected(&self) -> hyper::client::connect::Connected { self.inner.connected() } } @@ -327,58 +275,6 @@ where } } -// Compatibility implementation for hyper 0.14 ecosystem. -// Should be the same as those with imports from `http::Uri` and `hyper_util::client::legacy`. -// TODO(http-bump): remove this after there is no more dependency on hyper 0.14. -mod compat { - use http_02::Uri; - use hyper_014::client::connect::{Connected, Connection}; - - use super::*; - - impl, M: MonitorNewConnection + Clone + 'static> Service - for MonitoredConnection - where - C::Future: 'static, - { - type Error = C::Error; - type Response = MonitoredConnection; - - type Future = impl Future> + 'static; - - fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { - let ret = self.inner.poll_ready(cx); - if let Poll::Ready(Err(_)) = &ret { - self.monitor.on_err("".to_string()); - } - ret - } - - fn call(&mut self, uri: Uri) -> Self::Future { - let endpoint = format!("{:?}", uri.host()); - let monitor = self.monitor.clone(); - self.inner - .call(uri) - .map(move |result: Result<_, _>| match result { - Ok(resp) => Ok(MonitoredConnection::new( - resp, - monitor.new_connection_monitor(endpoint), - )), - Err(e) => { - monitor.on_err(endpoint); - Err(e) - } - }) - } - } - - impl Connection for MonitoredConnection { - fn connected(&self) -> Connected { - self.inner.connected() - } - } -} - #[derive(Clone)] pub struct ConnectionMetrics { connection_count: IntGaugeVec, @@ -638,16 +534,18 @@ impl tonic::transport::server::Router { signal: impl Future, ) -> impl Future where - L: tower_layer::Layer, - L::Service: Service, Response = http::Response> - + Clone + L: tower_layer::Layer, + L::Service: Service< + http::request::Request, + Response = http::response::Response, + > + Clone + Send + 'static, - <>::Service as Service< - http::Request, + <>::Service as Service< + http::request::Request, >>::Future: Send + 'static, - <>::Service as Service< - http::Request, + <>::Service as Service< + http::request::Request, >>::Error: Into> + Send, ResBody: http_body::Body + Send + 'static, ResBody::Error: Into>, diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 1afdac2baa2f3..10cb93b9fc290 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -232,7 +232,7 @@ mod tests { let worker_1 = WorkerNode { id: 1, parallelism: 1, - property: Some(serving_property), + property: Some(serving_property.clone()), ..Default::default() }; @@ -247,7 +247,7 @@ mod tests { let worker_2 = WorkerNode { id: 2, parallelism: 50, - property: Some(serving_property), + property: Some(serving_property.clone()), ..Default::default() }; diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index ed1758029092b..a3f74792982f2 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -23,8 +23,7 @@ either = "1" foyer = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } -http = "1" -hyper = "1" +hyper = "0.14" # required by tonic itertools = { workspace = true } maplit = "1.0.2" pprof = { version = "0.13", features = ["flamegraph"] } @@ -55,7 +54,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", "fs", ] } -tokio-stream = { workspace = true } +tokio-stream = "0.1" tonic = { workspace = true } tower = { version = "0.4", features = ["util", "load-shed"] } tracing = "0.1" diff --git a/src/compute/src/rpc/service/monitor_service.rs b/src/compute/src/rpc/service/monitor_service.rs index 0acc30e0c2430..a9a41d753ac96 100644 --- a/src/compute/src/rpc/service/monitor_service.rs +++ b/src/compute/src/rpc/service/monitor_service.rs @@ -389,7 +389,8 @@ pub mod grpc_middleware { use either::Either; use futures::Future; - use tonic::body::BoxBody; + use hyper::Body; + use tonic::transport::NamedService; use tower::{Layer, Service}; /// Manages the await-trees of `gRPC` requests that are currently served by the compute node. @@ -437,9 +438,10 @@ pub mod grpc_middleware { next_id: Arc, } - impl Service> for AwaitTreeMiddleware + impl Service> for AwaitTreeMiddleware where - S: Service> + Clone, + S: Service> + Clone + Send + 'static, + S::Future: Send + 'static, { type Error = S::Error; type Response = S::Response; @@ -450,7 +452,7 @@ pub mod grpc_middleware { self.inner.poll_ready(cx) } - fn call(&mut self, req: http::Request) -> Self::Future { + fn call(&mut self, req: hyper::Request) -> Self::Future { let Some(registry) = self.registry.clone() else { return Either::Left(self.inner.call(req)); }; @@ -477,8 +479,7 @@ pub mod grpc_middleware { } } - #[cfg(not(madsim))] - impl tonic::server::NamedService for AwaitTreeMiddleware { + impl NamedService for AwaitTreeMiddleware { const NAME: &'static str = S::NAME; } } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 38e82ccdf76ea..b46fc15bb605b 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -61,10 +61,10 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } gcp-bigquery-client = "0.18.0" glob = "0.3" -google-cloud-bigquery = { version = "0.12.0", features = ["auth"] } -google-cloud-gax = "0.19.0" -google-cloud-googleapis = { version = "0.15", features = ["pubsub", "bigquery"] } -google-cloud-pubsub = "0.28" +google-cloud-bigquery = { version = "0.9.0", features = ["auth"] } +google-cloud-gax = "0.17.0" +google-cloud-googleapis = { version = "0.13", features = ["pubsub", "bigquery"] } +google-cloud-pubsub = "0.25" http = "0.2" iceberg = { workspace = true } iceberg-catalog-rest = { workspace = true } @@ -100,8 +100,8 @@ pg_bigdecimal = { git = "https://github.com/risingwavelabs/rust-pg_bigdecimal", postgres-openssl = "0.5.0" prometheus = { version = "0.13", features = ["process"] } prost = { workspace = true, features = ["no-recursion-limit"] } -prost-reflect = "0.14" -prost-types = "0.13" +prost-reflect = "0.13" +prost-types = "0.12" protobuf-native = "0.2.2" pulsar = { version = "6.3", default-features = false, features = [ "tokio-runtime", @@ -172,7 +172,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ ] } tokio-postgres = { version = "0.7", features = ["with-uuid-1"] } tokio-retry = "0.3" -tokio-stream = { workspace = true } +tokio-stream = "0.1" tokio-util = { workspace = true, features = ["codec", "io"] } tonic = { workspace = true } tracing = "0.1" diff --git a/src/error/src/tonic.rs b/src/error/src/tonic.rs index f17b6b8ea9d44..4e3476c460fd6 100644 --- a/src/error/src/tonic.rs +++ b/src/error/src/tonic.rs @@ -244,7 +244,7 @@ mod tests { }; let server_status = original.to_status(tonic::Code::Internal, "test"); - let body = server_status.into_http(); + let body = server_status.to_http(); let client_status = tonic::Status::from_header_map(body.headers()).unwrap(); let wrapper = TonicStatusWrapper::new(client_status); diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index e493037c200b7..a9c955f91dcca 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -44,7 +44,7 @@ educe = "0.6" fancy-regex = "0.13" futures-async-stream = { workspace = true } futures-util = "0.3" -ginepro = "0.7" # TODO(http-bump): bump to 0.8 once arrow-udf switches to tonic 0.12 +ginepro = "0.7" hex = "0.4" icelake = { workspace = true } itertools = { workspace = true } @@ -71,7 +71,7 @@ sql-json-path = { version = "0.1", features = ["jsonbb"] } thiserror = "1" thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = ["time"] } -tonic = { version = "0.10", optional = true } # TODO(http-bump): bump once arrow-udf switches to tonic 0.12 +tonic = { version = "0.10", optional = true } tracing = "0.1" zstd = { version = "0.13", default-features = false, optional = true } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 3a95eab660b09..89d29e076a38f 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -93,7 +93,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", "fs", ] } -tokio-stream = { workspace = true } +tokio-stream = "0.1" tonic = { workspace = true } tracing = "0.1" uuid = "1" diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index f453a89204501..8e64a6db4e2b9 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -114,9 +114,12 @@ impl From<&PbSource> for SourceCatalog { let owner = prost.owner; let watermark_descs = prost.get_watermark_descs().clone(); - let associated_table_id = prost.optional_associated_table_id.map(|id| match id { - OptionalAssociatedTableId::AssociatedTableId(id) => id, - }); + let associated_table_id = prost + .optional_associated_table_id + .clone() + .map(|id| match id { + OptionalAssociatedTableId::AssociatedTableId(id) => id, + }); let version = prost.version; let connection_id = prost.connection_id; diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 432266d1871a9..5e32dcb7b2aba 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -240,7 +240,7 @@ fn get_acl_items( ) -> String { let mut res = String::from("{"); let mut empty_flag = true; - let super_privilege = available_prost_privilege(*object, for_dml_table); + let super_privilege = available_prost_privilege(object.clone(), for_dml_table); for user in users { let privileges = if user.is_super { vec![&super_privilege] diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index b49abcb023429..80f6316f9ca90 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -472,7 +472,8 @@ impl FrontendObserverNode { match info { Info::HummockSnapshot(hummock_snapshot) => match resp.operation() { Operation::Update => { - self.hummock_snapshot_manager.update(*hummock_snapshot); + self.hummock_snapshot_manager + .update(hummock_snapshot.clone()); } _ => panic!("receive an unsupported notify {:?}", resp), }, diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index e30dfa0dad377..a6c76c78bf23d 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -196,7 +196,7 @@ impl StageExecution { match cur_state { Pending { msg_sender } => { let runner = StageRunner { - epoch: self.epoch, + epoch: self.epoch.clone(), stage: self.stage.clone(), worker_node_manager: self.worker_node_manager.clone(), tasks: self.tasks.clone(), @@ -649,7 +649,7 @@ impl StageRunner { &plan_node, &task_id, self.ctx.to_batch_task_context(), - self.epoch, + self.epoch.clone(), shutdown_rx.clone(), ); @@ -935,7 +935,7 @@ impl StageRunner { let t_id = task_id.task_id; let stream_status: Fuse> = compute_client - .create_task(task_id, plan_fragment, self.epoch, expr_context) + .create_task(task_id, plan_fragment, self.epoch.clone(), expr_context) .await .inspect_err(|_| self.mask_failed_serving_worker(&worker)) .map_err(|e| anyhow!(e))? diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 73a3ade5799b5..9d13573e03a7b 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -121,7 +121,7 @@ impl PinnedSnapshot { impl Drop for PinnedSnapshot { fn drop(&mut self) { - let _ = self.unpin_sender.send(Operation::Unpin(self.value)); + let _ = self.unpin_sender.send(Operation::Unpin(self.value.clone())); } } @@ -202,7 +202,9 @@ impl HummockSnapshotManager { false } else { // First tell the worker that a new snapshot is going to be pinned. - self.worker_sender.send(Operation::Pin(snapshot)).unwrap(); + self.worker_sender + .send(Operation::Pin(snapshot.clone())) + .unwrap(); // Then set the latest snapshot. *old_snapshot = Arc::new(PinnedSnapshot { value: snapshot, diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 4511e9f61d894..b35ce9e73d96b 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -36,7 +36,7 @@ flate2 = "1" function_name = "0.3.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hex = "0.4" -http = "1" +hyper = "0.14" # required by tonic itertools = { workspace = true } jsonbb = { workspace = true } maplit = "1.0.2" @@ -81,7 +81,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", ] } tokio-retry = "0.3" -tokio-stream = { workspace = true } +tokio-stream = { version = "0.1", features = ["net"] } tonic = { workspace = true } tower = { version = "0.4", features = ["util", "load-shed"] } tracing = "0.1" diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 5d1b3570e1ce3..5c72b39bcd156 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -772,7 +772,7 @@ pub async fn start_service_as_election_leader( risingwave_pb::meta::event_log::Event::MetaNodeStart(event), ]); - let server_builder = tonic::transport::Server::builder() + let server = tonic::transport::Server::builder() .layer(MetricsMiddlewareLayer::new(meta_metrics)) .layer(TracingExtractLayer::new()) .add_service(HeartbeatServiceServer::new(heartbeat_srv)) @@ -794,19 +794,17 @@ pub async fn start_service_as_election_leader( .add_service(ServingServiceServer::new(serving_srv)) .add_service(CloudServiceServer::new(cloud_srv)) .add_service(SinkCoordinationServiceServer::new(sink_coordination_srv)) - .add_service(EventLogServiceServer::new(event_log_srv)); - #[cfg(not(madsim))] // `otlp-embedded` does not use madsim-patched tonic - let server_builder = server_builder.add_service(TraceServiceServer::new(trace_srv)); - - let server = server_builder.monitored_serve_with_shutdown( - address_info.listen_addr, - "grpc-meta-leader-service", - TcpConfig { - tcp_nodelay: true, - keepalive_duration: None, - }, - shutdown.clone().cancelled_owned(), - ); + .add_service(EventLogServiceServer::new(event_log_srv)) + .add_service(TraceServiceServer::new(trace_srv)) + .monitored_serve_with_shutdown( + address_info.listen_addr, + "grpc-meta-leader-service", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + shutdown.clone().cancelled_owned(), + ); started::set(); let _server_handle = tokio::spawn(server); diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 69986f8570234..1e3330a2b53a0 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -40,7 +40,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "time", "signal", ] } -tokio-stream = { workspace = true } +tokio-stream = { version = "0.1", features = ["net"] } tonic = { workspace = true } tracing = "0.1" diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index cd94e7c3af862..81b90d0f2005a 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -912,7 +912,7 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); let table_id = req.get_table_id(); - let parallelism = *req.get_parallelism()?; + let parallelism = req.get_parallelism()?.clone(); let deferred = req.get_deferred(); self.ddl_controller diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 48e8fcbbc05c1..b8b829ab9cddb 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -688,7 +688,7 @@ impl GlobalBarrierManager { r#type: node.r#type, host: node.host.clone(), parallelism: node.parallelism, - property: node.property, + property: node.property.clone(), resource: node.resource.clone(), ..Default::default() }, diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index d80ffb54e66a4..8df302380a64a 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -938,7 +938,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property, + property.clone(), PbResource::default(), ) .await?, @@ -970,7 +970,7 @@ mod tests { ); // re-register existing worker node with larger parallelism and change its serving mode. - let mut new_property = property; + let mut new_property = property.clone(); new_property.worker_node_parallelism = (parallelism_num * 2) as _; new_property.is_serving = false; cluster_ctl @@ -1025,7 +1025,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property, + property.clone(), PbResource::default(), ) .await?; diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index c7c2057fcc96e..0f42c17751377 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -377,7 +377,7 @@ impl HummockManager { committed_epoch: epoch, current_epoch: epoch, }; - let prev_snapshot = self.latest_snapshot.swap(snapshot.into()); + let prev_snapshot = self.latest_snapshot.swap(snapshot.clone().into()); assert!(prev_snapshot.committed_epoch < epoch); assert!(prev_snapshot.current_epoch < epoch); diff --git a/src/meta/src/hummock/model/pinned_snapshot.rs b/src/meta/src/hummock/model/pinned_snapshot.rs index fd009e22b789f..f485d9dab7211 100644 --- a/src/meta/src/hummock/model/pinned_snapshot.rs +++ b/src/meta/src/hummock/model/pinned_snapshot.rs @@ -28,7 +28,7 @@ impl MetadataModel for HummockPinnedSnapshot { } fn to_protobuf(&self) -> Self::PbType { - *self + self.clone() } fn from_protobuf(prost: Self::PbType) -> Self { diff --git a/src/meta/src/hummock/model/pinned_version.rs b/src/meta/src/hummock/model/pinned_version.rs index b2e7d97501b2b..e8f6b2e65e75e 100644 --- a/src/meta/src/hummock/model/pinned_version.rs +++ b/src/meta/src/hummock/model/pinned_version.rs @@ -28,7 +28,7 @@ impl MetadataModel for HummockPinnedVersion { } fn to_protobuf(&self) -> Self::PbType { - *self + self.clone() } fn from_protobuf(prost: Self::PbType) -> Self { diff --git a/src/meta/src/manager/catalog/user.rs b/src/meta/src/manager/catalog/user.rs index 81181b0fc1e17..f6e2f9e03e835 100644 --- a/src/meta/src/manager/catalog/user.rs +++ b/src/meta/src/manager/catalog/user.rs @@ -234,7 +234,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Update, Action::Delete], true, )], @@ -249,7 +249,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Insert], false, )], @@ -258,7 +258,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object)); + assert_eq!(user.grant_privileges[0].object, Some(object.clone())); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 2); assert!(user.grant_privileges[0] .action_with_opts @@ -269,7 +269,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Insert], true, )], @@ -284,7 +284,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Insert], true, )], @@ -293,7 +293,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object)); + assert_eq!(user.grant_privileges[0].object, Some(object.clone())); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 2); assert!(user.grant_privileges[0] .action_with_opts @@ -304,7 +304,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Insert], true, )], @@ -319,7 +319,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Update, Action::Delete], true, )], @@ -328,7 +328,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object)); + assert_eq!(user.grant_privileges[0].object, Some(object.clone())); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 4); assert!(user.grant_privileges[0] .action_with_opts @@ -339,7 +339,7 @@ mod tests { let res = catalog_manager .revoke_privilege( &[test_user_id], - &[make_privilege(object, &[Action::Connect], false)], + &[make_privilege(object.clone(), &[Action::Connect], false)], 0, test_sub_user_id, true, @@ -355,7 +355,11 @@ mod tests { let res = catalog_manager .revoke_privilege( &[test_user_id], - &[make_privilege(other_object, &[Action::Connect], false)], + &[make_privilege( + other_object.clone(), + &[Action::Connect], + false, + )], 0, test_sub_user_id, true, @@ -372,7 +376,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[ Action::Select, Action::Insert, @@ -397,7 +401,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[ Action::Select, Action::Insert, @@ -425,7 +429,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object, + object.clone(), &[Action::Select, Action::Insert, Action::Delete], false, )], diff --git a/src/meta/src/rpc/intercept.rs b/src/meta/src/rpc/intercept.rs index 87151e06b88a1..8b5bb67f30943 100644 --- a/src/meta/src/rpc/intercept.rs +++ b/src/meta/src/rpc/intercept.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use futures::Future; -use tonic::body::BoxBody; +use hyper::Body; use tower::{Layer, Service}; use crate::rpc::metrics::MetaMetrics; @@ -49,9 +49,9 @@ pub struct MetricsMiddleware { metrics: Arc, } -impl Service> for MetricsMiddleware +impl Service> for MetricsMiddleware where - S: Service> + Clone + Send + 'static, + S: Service> + Clone + Send + 'static, S::Future: Send + 'static, { type Error = S::Error; @@ -63,7 +63,7 @@ where self.inner.poll_ready(cx) } - fn call(&mut self, req: http::Request) -> Self::Future { + fn call(&mut self, req: hyper::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index fa433a30abcb4..e821c2fc85090 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -26,7 +26,7 @@ crc32fast = "1" either = "1" fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } -hyper = { version = "0.14", features = ["tcp", "client"] } # TODO(http-bump): required by aws sdk +hyper = { version = "0.14", features = ["tcp", "client"] } # required by aws sdk hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" itertools = { workspace = true } diff --git a/src/prost/build.rs b/src/prost/build.rs index 0682a63a02edb..aa2b8a706b690 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -132,7 +132,7 @@ fn main() -> Result<(), Box> { ) .type_attribute("plan_common.GeneratedColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.DefaultColumnDesc", "#[derive(Eq, Hash)]") - .type_attribute("plan_common.Cardinality", "#[derive(Eq, Hash)]") + .type_attribute("plan_common.Cardinality", "#[derive(Eq, Hash, Copy)]") .type_attribute("plan_common.ExternalTableDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.ColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumn", "#[derive(Eq, Hash)]") diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index 5bb2609c81593..c7b8ef27cd748 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -19,7 +19,6 @@ use std::str::FromStr; -pub use prost::Message; use risingwave_error::tonic::ToTonicStatus; use thiserror::Error; diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index 71c2662024dbb..b8a2ca9db14f9 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -23,7 +23,7 @@ clap = { workspace = true } console = "0.15" fs-err = "2.11.0" glob = "0.3" -google-cloud-pubsub = "0.28" +google-cloud-pubsub = "0.25" indicatif = "0.17" itertools = { workspace = true } rdkafka = { workspace = true } diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index 49729c6d9e8ac..37064df273ed0 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -19,8 +19,8 @@ async-trait = "0.1" easy-ext = "1" either = "1.13.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } -http = "1" -hyper = "1" +http = "0.2" +hyper = "0.14" # required by tonic itertools = { workspace = true } lru = { workspace = true } moka = { version = "0.12", features = ["future"] } @@ -43,7 +43,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", ] } tokio-retry = "0.3" -tokio-stream = { workspace = true } +tokio-stream = "0.1" tonic = { workspace = true } tower = "0.4" tracing = "0.1" diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 906261799b13d..b18acb55aac69 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -245,7 +245,7 @@ impl MetaClient { .add_worker_node(AddWorkerNodeRequest { worker_type: worker_type as i32, host: Some(addr.to_protobuf()), - property: Some(property), + property: Some(property.clone()), resource: Some(risingwave_pb::common::worker_node::Resource { rw_version: RW_VERSION.to_string(), total_memory_bytes: system_memory_available_bytes() as _, diff --git a/src/rpc_client/src/tracing.rs b/src/rpc_client/src/tracing.rs index aab07d43225d4..50c98007bb9fd 100644 --- a/src/rpc_client/src/tracing.rs +++ b/src/rpc_client/src/tracing.rs @@ -16,22 +16,46 @@ use std::task::{Context, Poll}; use futures::Future; use risingwave_common::util::tracing::TracingContext; -use tonic::body::BoxBody; -use tower::Service; +use tower::{Layer, Service}; + +/// A layer that decorates the inner service with [`TracingInject`]. +#[derive(Clone, Default)] +pub struct TracingInjectLayer { + _private: (), +} + +impl TracingInjectLayer { + #[allow(dead_code)] + pub fn new() -> Self { + Self::default() + } +} + +impl Layer for TracingInjectLayer { + type Service = TracingInject; + + fn layer(&self, service: S) -> Self::Service { + TracingInject { inner: service } + } +} /// A service wrapper that injects the [`TracingContext`] obtained from the current tracing span /// into the HTTP headers of the request. /// /// See also `TracingExtract` in the `common_service` crate. #[derive(Clone, Debug)] -pub struct TracingInjectChannel { - inner: tonic::transport::Channel, +pub struct TracingInject { + inner: S, } -#[cfg(not(madsim))] -impl Service> for TracingInjectChannel { - type Error = tonic::transport::Error; - type Response = http::Response; +impl Service> for TracingInject +where + S: Service> + Clone + Send + 'static, + S::Future: Send + 'static, + B: hyper::body::HttpBody, // tonic `Channel` uses `BoxBody` instead of `hyper::Body` +{ + type Error = S::Error; + type Response = S::Response; type Future = impl Future>; @@ -39,7 +63,7 @@ impl Service> for TracingInjectChannel { self.inner.poll_ready(cx) } - fn call(&mut self, mut req: http::Request) -> Self::Future { + fn call(&mut self, mut req: hyper::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary @@ -57,21 +81,21 @@ impl Service> for TracingInjectChannel { /// A wrapper around tonic's `Channel` that injects the [`TracingContext`] obtained from the current /// tracing span when making gRPC requests. #[cfg(not(madsim))] -pub type Channel = TracingInjectChannel; +pub type Channel = TracingInject; #[cfg(madsim)] pub type Channel = tonic::transport::Channel; -/// An extension trait for tonic's `Channel` that wraps it into a [`TracingInjectChannel`]. +/// An extension trait for tonic's `Channel` that wraps it in a [`TracingInject`] service. #[easy_ext::ext(TracingInjectedChannelExt)] impl tonic::transport::Channel { - /// Wraps the channel into a [`TracingInjectChannel`], so that the [`TracingContext`] obtained + /// Wraps the channel in a [`TracingInject`] service, so that the [`TracingContext`] obtained /// from the current tracing span is injected into the HTTP headers of the request. /// /// The server can then extract the [`TracingContext`] from the HTTP headers with the /// `TracingExtract` middleware. pub fn tracing_injected(self) -> Channel { #[cfg(not(madsim))] - return TracingInjectChannel { inner: self }; + return TracingInject { inner: self }; #[cfg(madsim)] return self; } diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 8dfba1b62a181..4c53af38eef67 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -122,7 +122,7 @@ impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { state_table_info: m .state_table_info .iter() - .map(|(t, i)| (t.table_id, *i)) + .map(|(t, i)| (t.table_id, i.clone())) .collect(), rw_version: m.rw_version.clone(), } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 1ee4fe0443783..d194f4355aa2b 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -85,7 +85,7 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary } GroupDelta::GroupDestroy(destroy_delta) => { assert!(group_destroy.is_none()); - group_destroy = Some(*destroy_delta); + group_destroy = Some(destroy_delta.clone()); } GroupDelta::GroupMetaChange(meta_delta) => { group_meta_changes.push(meta_delta.clone()); diff --git a/src/storage/hummock_sdk/src/time_travel.rs b/src/storage/hummock_sdk/src/time_travel.rs index 5894ed3e4a6e9..1fbd26ed3485b 100644 --- a/src/storage/hummock_sdk/src/time_travel.rs +++ b/src/storage/hummock_sdk/src/time_travel.rs @@ -338,7 +338,7 @@ impl IncompleteHummockVersionDelta { state_table_info_delta: self .state_table_info_delta .iter() - .map(|(table_id, delta)| (table_id.table_id, *delta)) + .map(|(table_id, delta)| (table_id.table_id, delta.clone())) .collect(), } } diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index e418250f0b6bf..4ce5b89138f8e 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -77,7 +77,7 @@ impl HummockVersionStateTableInfo { pub fn from_protobuf(state_table_info: &HashMap) -> Self { let state_table_info = state_table_info .iter() - .map(|(table_id, info)| (TableId::new(*table_id), *info)) + .map(|(table_id, info)| (TableId::new(*table_id), info.clone())) .collect(); let compaction_group_member_tables = Self::build_compaction_group_member_tables(&state_table_info); @@ -90,7 +90,7 @@ impl HummockVersionStateTableInfo { pub fn to_protobuf(&self) -> HashMap { self.state_table_info .iter() - .map(|(table_id, info)| (table_id.table_id, *info)) + .map(|(table_id, info)| (table_id.table_id, info.clone())) .collect() } @@ -642,7 +642,7 @@ impl From<&PbHummockVersionDelta> for HummockVersionDelta { state_table_info_delta: pb_version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (TableId::new(*table_id), *delta)) + .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) .collect(), } } @@ -679,7 +679,7 @@ impl From<&HummockVersionDelta> for PbHummockVersionDelta { state_table_info_delta: version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (table_id.table_id, *delta)) + .map(|(table_id, delta)| (table_id.table_id, delta.clone())) .collect(), } } @@ -716,7 +716,7 @@ impl From for PbHummockVersionDelta { state_table_info_delta: version_delta .state_table_info_delta .into_iter() - .map(|(table_id, delta)| (table_id.table_id, delta)) + .map(|(table_id, delta)| (table_id.table_id, delta.clone())) .collect(), } } @@ -761,7 +761,7 @@ impl From for HummockVersionDelta { state_table_info_delta: pb_version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (TableId::new(*table_id), *delta)) + .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) .collect(), } } @@ -938,7 +938,7 @@ impl From<&GroupDelta> for PbGroupDelta { delta_type: Some(PbDeltaType::GroupConstruct(pb_group_construct.clone())), }, GroupDelta::GroupDestroy(pb_group_destroy) => PbGroupDelta { - delta_type: Some(PbDeltaType::GroupDestroy(*pb_group_destroy)), + delta_type: Some(PbDeltaType::GroupDestroy(pb_group_destroy.clone())), }, GroupDelta::GroupMetaChange(pb_group_meta_change) => PbGroupDelta { delta_type: Some(PbDeltaType::GroupMetaChange(pb_group_meta_change.clone())), @@ -960,7 +960,7 @@ impl From<&PbGroupDelta> for GroupDelta { GroupDelta::GroupConstruct(pb_group_construct.clone()) } Some(PbDeltaType::GroupDestroy(pb_group_destroy)) => { - GroupDelta::GroupDestroy(*pb_group_destroy) + GroupDelta::GroupDestroy(pb_group_destroy.clone()) } Some(PbDeltaType::GroupMetaChange(pb_group_meta_change)) => { GroupDelta::GroupMetaChange(pb_group_meta_change.clone()) diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index de25cf8439be1..3c85092a4d677 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -79,7 +79,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ ] } tokio-metrics = "0.3.0" tokio-retry = "0.3" -tokio-stream = { workspace = true } +tokio-stream = "0.1" tonic = { workspace = true } tracing = "0.1" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 143d0f0c01c75..6d881f203f670 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -55,7 +55,7 @@ tempfile = "3" tikv-jemallocator = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" -tokio-stream = { workspace = true } +tokio-stream = "0.1" tracing = "0.1" tracing-subscriber = { version = "0.3", features = ["env-filter"] } diff --git a/src/tests/state_cleaning_test/Cargo.toml b/src/tests/state_cleaning_test/Cargo.toml index 6c12898343951..a105360a68f6b 100644 --- a/src/tests/state_cleaning_test/Cargo.toml +++ b/src/tests/state_cleaning_test/Cargo.toml @@ -24,7 +24,7 @@ serde = { version = "1", features = ["derive"] } serde_with = "3" tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" -tokio-stream = { workspace = true } +tokio-stream = { version = "0.1", features = ["fs"] } toml = "0.8" tracing = "0.1" diff --git a/src/utils/runtime/Cargo.toml b/src/utils/runtime/Cargo.toml index ff2902e7a4b4c..0815a005df5b5 100644 --- a/src/utils/runtime/Cargo.toml +++ b/src/utils/runtime/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] [dependencies] await-tree = { workspace = true } console = "0.15" -console-subscriber = "0.4" +console-subscriber = "0.3.0" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } hostname = "0.4" From 219a77e3669bebaf2554800b3f9b5bd32ae7b96a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 19:12:56 +0800 Subject: [PATCH 60/75] minor --- 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 c91b08344892f..1e11e390edfd8 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -149,7 +149,7 @@ fn to_ast_data_type(ty: &DataType) -> Result { .try_collect()?; Ok(AstDataType::Struct(fields)) } - DataType::Serial | DataType::Int256 => { + DataType::Serial | DataType::Int256 | DataType::Map(_) => { Err(anyhow!("unsupported data type: {:?}", ty).context("to_ast_data_type"))? } } From ddd667a27f833f70601278a8d3c04697ec27a4d2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 19:17:14 +0800 Subject: [PATCH 61/75] Reapply "chore: bump `tonic` to v0.12 (#17889)" This reverts commit 756d84093eb9c657bfe46b796688a7949bb84d1d. --- Cargo.lock | 360 ++++++++++++------ Cargo.toml | 18 +- ci/docker-compose.yml | 1 - e2e_test/sink/kafka/protobuf.slt | 16 +- src/batch/Cargo.toml | 2 +- src/batch/src/executor/hash_agg.rs | 2 +- .../executor/join/distributed_lookup_join.rs | 7 +- src/batch/src/executor/join/hash_join.rs | 2 +- .../src/executor/join/local_lookup_join.rs | 4 +- src/batch/src/executor/mod.rs | 4 +- src/batch/src/executor/order_by.rs | 2 +- src/batch/src/executor/row_seq_scan.rs | 7 +- src/batch/src/spill/spill_op.rs | 2 +- src/batch/src/task/broadcast_channel.rs | 2 +- src/batch/src/task/task_execution.rs | 2 +- src/bench/Cargo.toml | 2 +- src/common/Cargo.toml | 2 +- src/common/common_service/Cargo.toml | 2 +- src/common/common_service/src/tracing.rs | 8 +- src/common/metrics/Cargo.toml | 14 +- src/common/metrics/src/monitor/connection.rs | 130 ++++++- .../src/vnode_mapping/vnode_placement.rs | 4 +- src/compute/Cargo.toml | 5 +- .../src/rpc/service/monitor_service.rs | 13 +- src/connector/Cargo.toml | 14 +- src/error/src/tonic.rs | 2 +- src/expr/impl/Cargo.toml | 4 +- src/frontend/Cargo.toml | 2 +- src/frontend/src/catalog/source_catalog.rs | 9 +- .../src/catalog/system_catalog/mod.rs | 2 +- src/frontend/src/observer/observer_manager.rs | 3 +- .../src/scheduler/distributed/stage.rs | 6 +- src/frontend/src/scheduler/snapshot.rs | 6 +- src/meta/Cargo.toml | 4 +- src/meta/node/src/server.rs | 26 +- src/meta/service/Cargo.toml | 2 +- src/meta/service/src/ddl_service.rs | 2 +- src/meta/src/barrier/mod.rs | 2 +- src/meta/src/controller/cluster.rs | 6 +- src/meta/src/hummock/manager/commit_epoch.rs | 2 +- src/meta/src/hummock/model/pinned_snapshot.rs | 2 +- src/meta/src/hummock/model/pinned_version.rs | 2 +- src/meta/src/manager/catalog/user.rs | 32 +- src/meta/src/rpc/intercept.rs | 8 +- src/object_store/Cargo.toml | 2 +- src/prost/build.rs | 2 +- src/prost/src/lib.rs | 1 + src/risedevtool/Cargo.toml | 2 +- src/rpc_client/Cargo.toml | 6 +- src/rpc_client/src/meta_client.rs | 2 +- src/rpc_client/src/tracing.rs | 50 +-- src/storage/backup/src/lib.rs | 2 +- .../compaction_group/hummock_version_ext.rs | 2 +- src/storage/hummock_sdk/src/time_travel.rs | 2 +- src/storage/hummock_sdk/src/version.rs | 16 +- src/stream/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- src/tests/state_cleaning_test/Cargo.toml | 2 +- src/utils/runtime/Cargo.toml | 2 +- 59 files changed, 513 insertions(+), 329 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f1a6e60f2880d..7183c990bfce9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1751,7 +1751,7 @@ dependencies = [ "http 1.1.0", "http-body 1.0.0", "http-body-util", - "hyper 1.1.0", + "hyper 1.4.1", "hyper-util", "itoa", "matchit", @@ -2791,22 +2791,22 @@ dependencies = [ [[package]] name = "console-api" -version = "0.7.0" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a257c22cd7e487dd4a13d413beabc512c5052f0bc048db0da6a84c3d8a6142fd" +checksum = "86ed14aa9c9f927213c6e4f3ef75faaad3406134efe84ba2cb7983431d5f0931" dependencies = [ "futures-core", - "prost 0.12.1", - "prost-types 0.12.1", - "tonic 0.11.0", + "prost 0.13.1", + "prost-types 0.13.1", + "tonic 0.12.1", "tracing-core", ] [[package]] name = "console-subscriber" -version = "0.3.0" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31c4cc54bae66f7d9188996404abdf7fdfa23034ef8e43478c8810828abad758" +checksum = "e2e3a111a37f3333946ebf9da370ba5c5577b18eb342ec683eb488dd21980302" dependencies = [ "console-api", "crossbeam-channel", @@ -2814,14 +2814,15 @@ dependencies = [ "futures-task", "hdrhistogram", "humantime", - "prost 0.12.1", - "prost-types 0.12.1", + "hyper-util", + "prost 0.13.1", + "prost-types 0.13.1", "serde", "serde_json", "thread_local", "tokio", - "tokio-stream", - "tonic 0.11.0", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tonic 0.12.1", "tracing", "tracing-core", "tracing-subscriber", @@ -4436,15 +4437,15 @@ dependencies = [ [[package]] name = "etcd-client" -version = "0.12.4" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ae697f3928e8c89ae6f4dcf788059f49fd01a76dc53e63628f5a33881f5715e" +checksum = "39bde3ce50a626efeb1caa9ab1083972d178bebb55ca627639c8ded507dfcbde" dependencies = [ - "http 0.2.9", - "prost 0.12.1", + "http 1.1.0", + "prost 0.13.1", "tokio", - "tokio-stream", - "tonic 0.10.2", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tonic 0.12.1", "tonic-build", "tower", "tower-service", @@ -5185,7 +5186,7 @@ dependencies = [ "thiserror", "time", "tokio", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", "url", "yup-oauth2", ] @@ -5301,9 +5302,9 @@ dependencies = [ [[package]] name = "google-cloud-auth" -version = "0.15.0" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e09ed5b2998bc8d0d3df09c859028210d4961b8fe779cfda8dc8ca4e83d5def2" +checksum = "1112c453c2e155b3e683204ffff52bcc6d6495d04b68d9e90cd24161270c5058" dependencies = [ "async-trait", "base64 0.21.7", @@ -5323,9 +5324,9 @@ dependencies = [ [[package]] name = "google-cloud-bigquery" -version = "0.9.0" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e321c127945bb44a5cf5129c37530e2494b97afefe7f334a983ac754e40914e" +checksum = "305cb7214d11b719e9f00f982c1ee1304c674f7a8dfc44a43b8bad3c909750c2" dependencies = [ "anyhow", "arrow 50.0.0", @@ -5350,29 +5351,29 @@ dependencies = [ [[package]] name = "google-cloud-gax" -version = "0.17.0" +version = "0.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cb60314136e37de9e2a05ddb427b9c5a39c3d188de2e2f026c6af74425eef44" +checksum = "9c3eaaad103912825594d674a4b1e556ccbb05a13a6cac17dcfd871997fb760a" dependencies = [ "google-cloud-token", - "http 0.2.9", + "http 1.1.0", "thiserror", "tokio", "tokio-retry", - "tonic 0.10.2", + "tonic 0.12.1", "tower", "tracing", ] [[package]] name = "google-cloud-googleapis" -version = "0.13.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32cd184c52aa2619ac1b16ad8b5a752e91d25be88a8cf08eaec19777dfacbe54" +checksum = "0ae8ab26ef7c7c3f7dfb9cc3982293d031d8e78c85d00ddfb704b5c35aeff7c8" dependencies = [ - "prost 0.12.1", - "prost-types 0.12.1", - "tonic 0.10.2", + "prost 0.13.1", + "prost-types 0.13.1", + "tonic 0.12.1", ] [[package]] @@ -5388,9 +5389,9 @@ dependencies = [ [[package]] name = "google-cloud-pubsub" -version = "0.25.0" +version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a35e4a008db5cf01a5c03d3c67bd90b3cad77427ca949f3c8eddd90c4a3c932" +checksum = "55ef73601dcec5ea144e59969e921d35d66000211603fee8023b7947af09248f" dependencies = [ "async-channel 1.9.0", "async-stream", @@ -5398,7 +5399,7 @@ dependencies = [ "google-cloud-gax", "google-cloud-googleapis", "google-cloud-token", - "prost-types 0.12.1", + "prost-types 0.13.1", "thiserror", "tokio", "tokio-util", @@ -5407,9 +5408,9 @@ dependencies = [ [[package]] name = "google-cloud-token" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fcd62eb34e3de2f085bcc33a09c3e17c4f65650f36d53eb328b00d63bcb536a" +checksum = "8f49c12ba8b21d128a2ce8585955246977fbce4415f680ebf9199b6f9d6d725f" dependencies = [ "async-trait", ] @@ -5595,9 +5596,9 @@ checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" [[package]] name = "hermit-abi" -version = "0.3.2" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" [[package]] name = "hex" @@ -5760,9 +5761,9 @@ dependencies = [ [[package]] name = "hyper" -version = "1.1.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5aa53871fc917b1a9ed87b683a5d86db645e23acb32c2e0785a353e522fb75" +checksum = "50dfd22e0e76d0f662d429a5f80fcaf3855009297eab6a0a9f8543834744ba05" dependencies = [ "bytes", "futures-channel", @@ -5774,6 +5775,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", + "smallvec", "tokio", "want", ] @@ -5803,7 +5805,7 @@ checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" dependencies = [ "futures-util", "http 1.1.0", - "hyper 1.1.0", + "hyper 1.4.1", "hyper-util", "rustls 0.22.4", "rustls-pki-types", @@ -5824,6 +5826,19 @@ dependencies = [ "tokio-io-timeout", ] +[[package]] +name = "hyper-timeout" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3203a961e5c83b6f5498933e78b6b263e208c197b63e9c6c53cc82ffd3f63793" +dependencies = [ + "hyper 1.4.1", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + [[package]] name = "hyper-tls" version = "0.5.0" @@ -5845,7 +5860,7 @@ checksum = "70206fc6890eaca9fde8a0bf71caa2ddfc9fe045ac9e5c70df101a7dbde866e0" dependencies = [ "bytes", "http-body-util", - "hyper 1.1.0", + "hyper 1.4.1", "hyper-util", "native-tls", "tokio", @@ -5855,16 +5870,16 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.3" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca38ef113da30126bbff9cd1705f9273e15d45498615d138b0c20279ac7a76aa" +checksum = "3ab92f4f49ee4fb4f997c784b7a2e0fa70050211e0b6a287f898c3c9785ca956" dependencies = [ "bytes", "futures-channel", "futures-util", "http 1.1.0", "http-body 1.0.0", - "hyper 1.1.0", + "hyper 1.4.1", "pin-project-lite", "socket2 0.5.6", "tokio", @@ -6814,13 +6829,13 @@ dependencies = [ [[package]] name = "madsim-etcd-client" -version = "0.4.0+0.12.1" +version = "0.6.0+0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02b4b5de48bb7f3f7eae0bca62b3ed0b7d714b1b273d7347329b92c3a2eef113" +checksum = "8edcf23498cb590e415ce2ba6c7f186c7aa3340e7aa716ddddb34faf0a9ffdfb" dependencies = [ "etcd-client", "futures-util", - "http 0.2.9", + "http 1.1.0", "madsim", "serde", "serde_with 3.8.0", @@ -6828,7 +6843,7 @@ dependencies = [ "thiserror", "tokio", "toml 0.8.12", - "tonic 0.10.2", + "tonic 0.12.1", "tracing", ] @@ -6881,29 +6896,29 @@ dependencies = [ [[package]] name = "madsim-tonic" -version = "0.4.1+0.10.0" +version = "0.5.1+0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "813977c7870103e113a0332d97731f961bc48aaa8860edd318ef7d7754214436" +checksum = "61c668c82f0c2aca7ffed3235047f2539e6e41278c7c47a822999f3b7a067887" dependencies = [ "async-stream", "chrono", "futures-util", "madsim", "tokio", - "tonic 0.10.2", + "tonic 0.12.1", "tower", "tracing", ] [[package]] name = "madsim-tonic-build" -version = "0.4.2+0.10.0" +version = "0.5.0+0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a2ad2776ba20221ccbe4e136e2fa0f7ab90eebd608373177f3e74a198a288ec" +checksum = "f271a476bbaa9d2139e1e1a5beb869c6119e805a0b67ad2b2857e4a8785b111a" dependencies = [ "prettyplease 0.2.15", "proc-macro2", - "prost-build 0.12.1", + "prost-build 0.13.1", "quote", "syn 2.0.66", "tonic-build", @@ -7932,7 +7947,7 @@ dependencies = [ "rand", "thiserror", "tokio", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", ] [[package]] @@ -7985,20 +8000,20 @@ dependencies = [ [[package]] name = "otlp-embedded" version = "0.0.1" -source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=492c244e0be91feb659c0cd48a624bbd96045a33#492c244e0be91feb659c0cd48a624bbd96045a33" +source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=e6cd165b9bc85783b42c106e99186b86b73e3507#e6cd165b9bc85783b42c106e99186b86b73e3507" dependencies = [ "axum 0.7.4", "datasize", "hex", - "itertools 0.12.1", - "madsim-tonic", - "madsim-tonic-build", - "prost 0.12.1", + "itertools 0.13.0", + "prost 0.13.1", "rust-embed", "schnellru", "serde", "serde_json", "tokio", + "tonic 0.12.1", + "tonic-build", "tracing", ] @@ -8935,6 +8950,16 @@ dependencies = [ "prost-derive 0.12.1", ] +[[package]] +name = "prost" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13db3d3fde688c61e2446b4d843bc27a7e8af269a69440c0308021dc92333cc" +dependencies = [ + "bytes", + "prost-derive 0.13.1", +] + [[package]] name = "prost-build" version = "0.11.9" @@ -8979,6 +9004,27 @@ dependencies = [ "which", ] +[[package]] +name = "prost-build" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5bb182580f71dd070f88d01ce3de9f4da5021db7115d2e1c3605a754153b77c1" +dependencies = [ + "bytes", + "heck 0.5.0", + "itertools 0.13.0", + "log", + "multimap 0.10.0", + "once_cell", + "petgraph", + "prettyplease 0.2.15", + "prost 0.13.1", + "prost-types 0.13.1", + "regex", + "syn 2.0.66", + "tempfile", +] + [[package]] name = "prost-derive" version = "0.11.9" @@ -9005,6 +9051,19 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "prost-derive" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18bec9b0adc4eba778b33684b7ba3e7137789434769ee3ce3930463ef904cfca" +dependencies = [ + "anyhow", + "itertools 0.13.0", + "proc-macro2", + "quote", + "syn 2.0.66", +] + [[package]] name = "prost-helpers" version = "0.1.0" @@ -9016,13 +9075,13 @@ dependencies = [ [[package]] name = "prost-reflect" -version = "0.13.0" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ae9372e3227f3685376a0836e5c248611eafc95a0be900d44bc6cdf225b700f" +checksum = "55a6a9143ae25c25fa7b6a48d6cc08b10785372060009c25140a4e7c340e95af" dependencies = [ "once_cell", - "prost 0.12.1", - "prost-types 0.12.1", + "prost 0.13.1", + "prost-types 0.13.1", ] [[package]] @@ -9043,6 +9102,15 @@ dependencies = [ "prost 0.12.1", ] +[[package]] +name = "prost-types" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cee5168b05f49d4b0ca581206eb14a7b22fafd963efe729ac48eb03266e25cc2" +dependencies = [ + "prost 0.13.1", +] + [[package]] name = "protobuf" version = "2.28.0" @@ -9160,7 +9228,7 @@ dependencies = [ "indoc", "libc", "memoffset", - "parking_lot 0.11.2", + "parking_lot 0.12.1", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -9623,7 +9691,7 @@ dependencies = [ "http 1.1.0", "http-body 1.0.0", "http-body-util", - "hyper 1.1.0", + "hyper 1.4.1", "hyper-rustls 0.26.0", "hyper-tls 0.6.0", "hyper-util", @@ -9813,7 +9881,7 @@ dependencies = [ "bytes", "itertools 0.12.1", "parking_lot 0.12.1", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_meta_model_v2", @@ -9855,7 +9923,7 @@ dependencies = [ "parquet 52.0.0", "paste", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_common", "risingwave_common_estimate_size", @@ -9875,7 +9943,7 @@ dependencies = [ "thiserror-ext", "tikv-jemallocator", "tokio-metrics", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tokio-util", "tracing", "twox-hash", @@ -9918,7 +9986,7 @@ dependencies = [ "serde", "serde_yaml", "thiserror-ext", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "toml 0.8.12", "tracing", "tracing-subscriber", @@ -10024,7 +10092,7 @@ dependencies = [ "governor", "hashbrown 0.14.3", "hex", - "http 0.2.9", + "http 1.1.0", "http-body 0.4.5", "humantime", "hytra", @@ -10050,7 +10118,7 @@ dependencies = [ "pretty_assertions", "procfs 0.16.0", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "regex", "reqwest 0.12.4", @@ -10132,14 +10200,18 @@ dependencies = [ name = "risingwave_common_metrics" version = "1.11.0-alpha" dependencies = [ + "auto_impl", "bytes", "clap", "darwin-libproc", "easy-ext", "futures", "http 0.2.9", - "http-body 0.4.5", + "http 1.1.0", + "http-body 1.0.0", "hyper 0.14.27", + "hyper 1.4.1", + "hyper-util", "hytra", "itertools 0.12.1", "libc", @@ -10180,7 +10252,7 @@ dependencies = [ "anyhow", "bincode 1.3.3", "parking_lot 0.12.1", - "prost 0.12.1", + "prost 0.13.1", "risingwave_pb", "serde", "thiserror", @@ -10195,7 +10267,7 @@ dependencies = [ "async-trait", "axum 0.7.4", "futures", - "hyper 0.14.27", + "http 1.1.0", "madsim-tokio", "madsim-tonic", "prometheus", @@ -10249,7 +10321,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "parking_lot 0.12.1", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_common_heap_profiling", "risingwave_common_service", @@ -10275,14 +10347,15 @@ dependencies = [ "foyer", "futures", "futures-async-stream", - "hyper 0.14.27", + "http 1.1.0", + "hyper 1.4.1", "itertools 0.12.1", "madsim-tokio", "madsim-tonic", "maplit", "pprof", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_batch", "risingwave_common", @@ -10301,7 +10374,7 @@ dependencies = [ "tempfile", "thiserror-ext", "tikv-jemalloc-ctl", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tower", "tracing", "uuid", @@ -10388,10 +10461,10 @@ dependencies = [ "postgres-openssl", "pretty_assertions", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "prost-build 0.12.1", "prost-reflect", - "prost-types 0.12.1", + "prost-types 0.13.1", "protobuf-native", "protobuf-src", "pulsar", @@ -10431,7 +10504,7 @@ dependencies = [ "time", "tokio-postgres", "tokio-retry", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tokio-util", "tracing", "tracing-subscriber", @@ -10489,7 +10562,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", - "prost 0.12.1", + "prost 0.13.1", "regex", "risingwave_common", "risingwave_connector", @@ -10729,7 +10802,7 @@ dependencies = [ "pretty-xmlish", "pretty_assertions", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_batch", "risingwave_common", @@ -10756,7 +10829,7 @@ dependencies = [ "tempfile", "thiserror", "thiserror-ext", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tracing", "uuid", "workspace-hack", @@ -10781,7 +10854,7 @@ dependencies = [ "hex", "itertools 0.12.1", "parse-display", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_common_estimate_size", "risingwave_pb", @@ -10839,7 +10912,7 @@ dependencies = [ "madsim-tokio", "mockall", "parking_lot 0.12.1", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_pb", @@ -10858,7 +10931,7 @@ dependencies = [ "futures", "jni", "madsim-tokio", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_expr", "risingwave_hummock_sdk", @@ -10888,7 +10961,7 @@ dependencies = [ "jni", "madsim-tokio", "paste", - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_expr", "risingwave_hummock_sdk", @@ -10955,7 +11028,7 @@ dependencies = [ "function_name", "futures", "hex", - "hyper 0.14.27", + "http 1.1.0", "itertools 0.12.1", "jsonbb", "madsim-etcd-client", @@ -10970,7 +11043,7 @@ dependencies = [ "parking_lot 0.12.1", "prometheus", "prometheus-http-query", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_backup", "risingwave_common", @@ -10997,7 +11070,7 @@ dependencies = [ "thiserror", "thiserror-ext", "tokio-retry", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tower", "tower-http", "tracing", @@ -11043,7 +11116,7 @@ dependencies = [ name = "risingwave_meta_model_v2" version = "1.11.0-alpha" dependencies = [ - "prost 0.12.1", + "prost 0.13.1", "risingwave_common", "risingwave_hummock_sdk", "risingwave_pb", @@ -11097,7 +11170,7 @@ dependencies = [ "itertools 0.12.1", "madsim-tokio", "madsim-tonic", - "prost 0.12.1", + "prost 0.13.1", "rand", "regex", "risingwave_common", @@ -11110,7 +11183,7 @@ dependencies = [ "serde_json", "sync-point", "thiserror-ext", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tracing", "workspace-hack", ] @@ -11162,8 +11235,8 @@ dependencies = [ "madsim-tonic-build", "pbjson", "pbjson-build", - "prost 0.12.1", - "prost-build 0.12.1", + "prost 0.13.1", + "prost-build 0.13.1", "prost-helpers", "risingwave_error", "serde", @@ -11218,8 +11291,8 @@ dependencies = [ "easy-ext", "either", "futures", - "http 0.2.9", - "hyper 0.14.27", + "http 1.1.0", + "hyper 1.4.1", "itertools 0.12.1", "lru 0.7.6", "madsim-tokio", @@ -11236,7 +11309,7 @@ dependencies = [ "thiserror", "thiserror-ext", "tokio-retry", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tower", "tracing", "url", @@ -11322,7 +11395,7 @@ dependencies = [ "tempfile", "tikv-jemallocator", "tokio-postgres", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tracing", "tracing-subscriber", ] @@ -11390,7 +11463,7 @@ dependencies = [ "serde", "serde_with 3.8.0", "tokio-postgres", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "toml 0.8.12", "tracing", "workspace-hack", @@ -11437,7 +11510,7 @@ dependencies = [ "parking_lot 0.12.1", "procfs 0.16.0", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_backup", "risingwave_common", @@ -11506,7 +11579,7 @@ dependencies = [ "pin-project", "prehash", "prometheus", - "prost 0.12.1", + "prost 0.13.1", "rand", "risingwave_common", "risingwave_common_estimate_size", @@ -11531,7 +11604,7 @@ dependencies = [ "thiserror-ext", "tokio-metrics", "tokio-retry", - "tokio-stream", + "tokio-stream 0.1.15 (git+https://github.com/madsim-rs/tokio.git?rev=0dd1055)", "tracing", "tracing-test", "workspace-hack", @@ -12077,9 +12150,9 @@ dependencies = [ [[package]] name = "schnellru" -version = "0.2.1" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "772575a524feeb803e5b0fcbc6dd9f367e579488197c94c6e4023aad2305774d" +checksum = "c9a8ef13a93c54d20580de1e5c413e624e53121d42fc7e2c11d10ef7f8b02367" dependencies = [ "ahash 0.8.11", "cfg-if", @@ -13122,7 +13195,7 @@ dependencies = [ "thiserror", "time", "tokio", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", "tracing", "url", "uuid", @@ -13876,7 +13949,7 @@ dependencies = [ "futures-util", "pin-project-lite", "tokio", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", ] [[package]] @@ -13970,8 +14043,19 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.14" -source = "git+https://github.com/madsim-rs/tokio.git?rev=fe39bb8e#fe39bb8e8ab0ed96ee1b4477ab5508c20ce017fb" +version = "0.1.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-stream" +version = "0.1.15" +source = "git+https://github.com/madsim-rs/tokio.git?rev=0dd1055#0dd105567b323c863c29f794d2221ed588956d8d" dependencies = [ "futures-core", "madsim-tokio", @@ -14074,12 +14158,11 @@ dependencies = [ "axum 0.6.20", "base64 0.21.7", "bytes", - "flate2", "h2 0.3.26", "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-timeout", + "hyper-timeout 0.4.1", "percent-encoding", "pin-project", "prost 0.12.1", @@ -14087,12 +14170,11 @@ dependencies = [ "rustls-pemfile 1.0.4", "tokio", "tokio-rustls 0.24.1", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", "tower", "tower-layer", "tower-service", "tracing", - "webpki-roots 0.25.2", ] [[package]] @@ -14110,27 +14192,61 @@ dependencies = [ "http 0.2.9", "http-body 0.4.5", "hyper 0.14.27", - "hyper-timeout", + "hyper-timeout 0.4.1", "percent-encoding", "pin-project", "prost 0.12.1", "tokio", - "tokio-stream", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", "tower", "tower-layer", "tower-service", "tracing", ] +[[package]] +name = "tonic" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38659f4a91aba8598d27821589f5db7dddd94601e7a01b1e485a50e5484c7401" +dependencies = [ + "async-stream", + "async-trait", + "axum 0.7.4", + "base64 0.22.0", + "bytes", + "flate2", + "h2 0.4.4", + "http 1.1.0", + "http-body 1.0.0", + "http-body-util", + "hyper 1.4.1", + "hyper-timeout 0.5.1", + "hyper-util", + "percent-encoding", + "pin-project", + "prost 0.13.1", + "rustls-pemfile 2.1.1", + "socket2 0.5.6", + "tokio", + "tokio-rustls 0.26.0", + "tokio-stream 0.1.15 (registry+https://github.com/rust-lang/crates.io-index)", + "tower", + "tower-layer", + "tower-service", + "tracing", + "webpki-roots 0.26.1", +] + [[package]] name = "tonic-build" -version = "0.10.2" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d021fc044c18582b9a2408cd0dd05b1596e3ecdb5c4df822bb0183545683889" +checksum = "568392c5a2bd0020723e3f387891176aabafe36fd9fcd074ad309dfa0c8eb964" dependencies = [ "prettyplease 0.2.15", "proc-macro2", - "prost-build 0.12.1", + "prost-build 0.13.1", "quote", "syn 2.0.66", ] diff --git a/Cargo.toml b/Cargo.toml index 9b07142dcf021..5bfab4feb27fb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -121,7 +121,7 @@ aws-smithy-types = { version = "1", default-features = false, features = [ aws-endpoint = "0.60" aws-types = "1" axum = "=0.7.4" # TODO: 0.7.5+ does not work with current toolchain -etcd-client = { package = "madsim-etcd-client", version = "0.4" } +etcd-client = { package = "madsim-etcd-client", version = "0.6" } futures-async-stream = "0.2.9" hytra = "0.1" rdkafka = { package = "madsim-rdkafka", version = "0.4.1", features = [ @@ -129,11 +129,11 @@ rdkafka = { package = "madsim-rdkafka", version = "0.4.1", features = [ ] } hashbrown = { version = "0.14", features = ["ahash", "inline-more", "nightly"] } criterion = { version = "0.5", features = ["async_futures"] } -tonic = { package = "madsim-tonic", version = "0.4.1" } -tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } -otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "492c244e0be91feb659c0cd48a624bbd96045a33" } -prost = { version = "0.12" } -prost-build = { version = "0.12" } +tonic = { package = "madsim-tonic", version = "0.5.1" } +tonic-build = { package = "madsim-tonic-build", version = "0.5" } +otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "e6cd165b9bc85783b42c106e99186b86b73e3507" } +prost = { version = "0.13" } +prost-build = { version = "0.13" } icelake = { git = "https://github.com/risingwavelabs/icelake.git", rev = "1860eb315183a5f3f72b4097c1e40d49407f8373", features = [ "prometheus", ] } @@ -180,6 +180,7 @@ tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git" "profiling", "stats", ], rev = "64a2d9" } +# TODO(http-bump): bump to use tonic 0.12 once minitrace-opentelemetry is updated opentelemetry = "0.23" opentelemetry-otlp = "0.16" opentelemetry_sdk = { version = "0.23", default-features = false } @@ -195,6 +196,7 @@ sea-orm = { version = "0.12.14", features = [ "runtime-tokio-native-tls", ] } sqlx = "0.7" +tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055", features = ["net", "fs"] } tokio-util = "0.7" tracing-opentelemetry = "0.24" rand = { version = "0.8", features = ["small_rng"] } @@ -335,7 +337,9 @@ opt-level = 2 # Patch third-party crates for deterministic simulation. quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "948bdc3" } getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae" } -tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "fe39bb8e" } +# Don't patch `tokio-stream`, but only use the madsim version for **direct** dependencies. +# Imagine an unpatched dependency depends on the original `tokio` and the patched `tokio-stream`. +# tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "0dd1055" } tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = "95e2fd3" } tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } futures-timer = { git = "https://github.com/madsim-rs/futures-timer.git", rev = "05b33b4" } diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index b230e0381d147..4b1954ff5ae2c 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -266,7 +266,6 @@ services: SCHEMA_REGISTRY_HOST_NAME: schemaregistry SCHEMA_REGISTRY_LISTENERS: http://schemaregistry:8082 SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS: message_queue:29092 - SCHEMA_REGISTRY_DEBUG: 'true' pulsar-server: container_name: pulsar-server diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 0abd242e3c79d..5f032ba32f8dc 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -201,25 +201,19 @@ format plain encode protobuf ( message = 'recursive.AllTypes'); statement ok -drop sink sink_upsert; +drop table from_kafka cascade; statement ok -drop sink sink_csr_nested; +drop table from_kafka_csr_trivial cascade; statement ok -drop sink sink_csr_trivial; +drop table from_kafka_csr_nested cascade; statement ok -drop sink sink0; +drop table from_kafka_raw cascade; statement ok -drop table into_kafka; - -statement ok -drop table from_kafka_raw; +drop table into_kafka cascade; system ok rpk topic delete test-rw-sink-upsert-protobuf - -statement ok -drop table from_kafka; diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 099ae9019afcf..403eb864229d3 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -63,7 +63,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "fs", ] } tokio-metrics = "0.3.0" -tokio-stream = "0.1" +tokio-stream = { workspace = true } tokio-util = { workspace = true } tonic = { workspace = true } tracing = "0.1" diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index d69d4fbc8b174..00073217f7ead 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -20,7 +20,6 @@ use bytes::Bytes; use futures_async_stream::try_stream; use hashbrown::hash_map::Entry; use itertools::Itertools; -use prost::Message; use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{Field, Schema}; @@ -35,6 +34,7 @@ use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; use risingwave_pb::data::DataChunk as PbDataChunk; +use risingwave_pb::Message; use crate::error::{BatchError, Result}; use crate::executor::aggregation::build as build_agg; diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index f5ad5ab5ed984..1068ffd7f3349 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -354,10 +354,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let pk_prefix = OwnedRow::new(scan_range.eq_conds); if self.lookup_prefix_len == self.table.pk_indices().len() { - let row = self - .table - .get_row(&pk_prefix, self.epoch.clone().into()) - .await?; + let row = self.table.get_row(&pk_prefix, self.epoch.into()).await?; if let Some(row) = row { self.row_list.push(row); @@ -366,7 +363,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder { let iter = self .table .batch_iter_with_pk_bounds( - self.epoch.clone().into(), + self.epoch.into(), &pk_prefix, .., false, diff --git a/src/batch/src/executor/join/hash_join.rs b/src/batch/src/executor/join/hash_join.rs index 026f03fb65deb..3bfb583d6459d 100644 --- a/src/batch/src/executor/join/hash_join.rs +++ b/src/batch/src/executor/join/hash_join.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use bytes::Bytes; use futures_async_stream::try_stream; use itertools::Itertools; -use prost::Message; use risingwave_common::array::{Array, DataChunk, RowRef}; use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::Schema; @@ -34,6 +33,7 @@ use risingwave_common_estimate_size::EstimateSize; use risingwave_expr::expr::{build_from_prost, BoxedExpression, Expression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; +use risingwave_pb::Message; use super::{ChunkedData, JoinType, RowId}; use crate::error::{BatchError, Result}; diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index 7fcaba71a9c3b..a3be00fc39a22 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -134,7 +134,7 @@ impl InnerSideExecutorBuilder { ..Default::default() }), }), - epoch: Some(self.epoch.clone()), + epoch: Some(self.epoch), tracing_context: TracingContext::from_current_span().to_protobuf(), }; @@ -237,7 +237,7 @@ impl LookupExecutorBuilder for InnerSideExecutorBuilder &plan_node, &task_id, self.context.clone(), - self.epoch.clone(), + self.epoch, self.shutdown_rx.clone(), ); diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index 3a64901c64a04..80dc57b4f3620 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -174,7 +174,7 @@ impl<'a, C: Clone> ExecutorBuilder<'a, C> { plan_node, self.task_id, self.context.clone(), - self.epoch.clone(), + self.epoch, self.shutdown_rx.clone(), ) } @@ -188,7 +188,7 @@ impl<'a, C: Clone> ExecutorBuilder<'a, C> { } pub fn epoch(&self) -> BatchQueryEpoch { - self.epoch.clone() + self.epoch } } diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index 3f8c8e106c78f..ad7cc13992346 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use bytes::Bytes; use futures_async_stream::try_stream; use itertools::Itertools; -use prost::Message; use risingwave_common::array::DataChunk; use risingwave_common::catalog::Schema; use risingwave_common::memory::MemoryContext; @@ -28,6 +27,7 @@ use risingwave_common::util::sort_util::ColumnOrder; use risingwave_common_estimate_size::EstimateSize; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; +use risingwave_pb::Message; use super::{ BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder, diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index b8287147c6750..b897dbd813787 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -237,7 +237,7 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { let ordered = seq_scan_node.ordered; - let epoch = source.epoch.clone(); + let epoch = source.epoch; let limit = seq_scan_node.limit; let as_of = seq_scan_node .as_of @@ -341,8 +341,7 @@ impl RowSeqScanExecutor { for point_get in point_gets { let table = table.clone(); if let Some(row) = - Self::execute_point_get(table, point_get, query_epoch.clone(), histogram.clone()) - .await? + Self::execute_point_get(table, point_get, query_epoch, histogram.clone()).await? { if let Some(chunk) = data_chunk_builder.append_one_row(row) { returned += chunk.cardinality() as u64; @@ -373,7 +372,7 @@ impl RowSeqScanExecutor { table.clone(), range, ordered, - query_epoch.clone(), + query_epoch, chunk_size, limit, histogram.clone(), diff --git a/src/batch/src/spill/spill_op.rs b/src/batch/src/spill/spill_op.rs index 237ee3baf0099..b3e842a269ec7 100644 --- a/src/batch/src/spill/spill_op.rs +++ b/src/batch/src/spill/spill_op.rs @@ -22,9 +22,9 @@ use futures_util::AsyncReadExt; use opendal::layers::RetryLayer; use opendal::services::{Fs, Memory}; use opendal::Operator; -use prost::Message; use risingwave_common::array::DataChunk; use risingwave_pb::data::DataChunk as PbDataChunk; +use risingwave_pb::Message; use thiserror_ext::AsReport; use tokio::sync::Mutex; use twox_hash::XxHash64; diff --git a/src/batch/src/task/broadcast_channel.rs b/src/batch/src/task/broadcast_channel.rs index d66eda7d7d620..9781e38e7d7f6 100644 --- a/src/batch/src/task/broadcast_channel.rs +++ b/src/batch/src/task/broadcast_channel.rs @@ -86,7 +86,7 @@ pub fn new_broadcast_channel( output_channel_size: usize, ) -> (ChanSenderImpl, Vec) { let broadcast_info = match shuffle.distribution { - Some(exchange_info::Distribution::BroadcastInfo(ref v)) => v.clone(), + Some(exchange_info::Distribution::BroadcastInfo(ref v)) => *v, _ => BroadcastInfo::default(), }; diff --git a/src/batch/src/task/task_execution.rs b/src/batch/src/task/task_execution.rs index 4536dad1c031f..7186ced55febd 100644 --- a/src/batch/src/task/task_execution.rs +++ b/src/batch/src/task/task_execution.rs @@ -393,7 +393,7 @@ impl BatchTaskExecution { self.plan.root.as_ref().unwrap(), &self.task_id, self.context.clone(), - self.epoch.clone(), + self.epoch, self.shutdown_rx.clone(), ) .build(), diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index d451ef46ef838..43451ebaeb9d1 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -50,7 +50,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "time", "signal", ] } -tokio-stream = "0.1" +tokio-stream = { workspace = true } toml = "0.8" tracing = "0.1" tracing-subscriber = "0.3.17" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index a117dce645ae6..2cc1d81f1a38d 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -55,7 +55,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } governor = { version = "0.6", default-features = false, features = ["std"] } hashbrown = "0.14" hex = "0.4.3" -http = "0.2" +http = "1" humantime = "2.1" hytra = { workspace = true } itertools = { workspace = true } diff --git a/src/common/common_service/Cargo.toml b/src/common/common_service/Cargo.toml index cb43702f3f9e6..87206ab7cbc1d 100644 --- a/src/common/common_service/Cargo.toml +++ b/src/common/common_service/Cargo.toml @@ -18,7 +18,7 @@ normal = ["workspace-hack"] async-trait = "0.1" axum = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } -hyper = "0.14" # required by tonic +http = "1" prometheus = { version = "0.13" } risingwave_common = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/common/common_service/src/tracing.rs b/src/common/common_service/src/tracing.rs index 3ee4a64231c29..de6f43bbf33f3 100644 --- a/src/common/common_service/src/tracing.rs +++ b/src/common/common_service/src/tracing.rs @@ -15,8 +15,8 @@ use std::task::{Context, Poll}; use futures::Future; -use hyper::Body; use risingwave_common::util::tracing::TracingContext; +use tonic::body::BoxBody; use tower::{Layer, Service}; use tracing::Instrument; @@ -49,9 +49,9 @@ pub struct TracingExtract { inner: S, } -impl Service> for TracingExtract +impl Service> for TracingExtract where - S: Service> + Clone + Send + 'static, + S: Service> + Clone + Send + 'static, S::Future: Send + 'static, { type Error = S::Error; @@ -63,7 +63,7 @@ where self.inner.poll_ready(cx) } - fn call(&mut self, req: hyper::Request) -> Self::Future { + fn call(&mut self, req: http::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary diff --git a/src/common/metrics/Cargo.toml b/src/common/metrics/Cargo.toml index 4f3e8b20936b2..0c32b557cebb2 100644 --- a/src/common/metrics/Cargo.toml +++ b/src/common/metrics/Cargo.toml @@ -15,12 +15,16 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +auto_impl = "1" bytes = "1" clap = { workspace = true } easy-ext = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -http = "0.2" -hyper = { version = "0.14", features = ["client"] } # used by tonic +http = "1" +http-02 = { package = "http", version = "0.2" } +hyper = { version = "1" } +hyper-014 = { package = "hyper", version = "0.14" } +hyper-util = { version = "0.1", features = ["client-legacy"] } hytra = { workspace = true } itertools = { workspace = true } parking_lot = { workspace = true } @@ -32,13 +36,13 @@ serde = { version = "1", features = ["derive"] } thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio" } tonic = { workspace = true } +tower-layer = "0.3.2" +tower-service = "0.3.2" tracing = "0.1" tracing-subscriber = "0.3.17" [target.'cfg(not(madsim))'.dependencies] -http-body = "0.4.5" -tower-layer = "0.3.2" -tower-service = "0.3.2" +http-body = "1" [target.'cfg(target_os = "linux")'.dependencies] procfs = { version = "0.16", default-features = false } libc = "0.2" diff --git a/src/common/metrics/src/monitor/connection.rs b/src/common/metrics/src/monitor/connection.rs index e5774a3f16d7d..aa7c8c8d4baa3 100644 --- a/src/common/metrics/src/monitor/connection.rs +++ b/src/common/metrics/src/monitor/connection.rs @@ -24,10 +24,9 @@ use std::time::Duration; use futures::FutureExt; use http::Uri; -use hyper::client::connect::dns::{GaiAddrs, GaiFuture, GaiResolver, Name}; -use hyper::client::connect::Connection; -use hyper::client::HttpConnector; -use hyper::service::Service; +use hyper_util::client::legacy::connect::dns::{GaiAddrs, GaiFuture, GaiResolver, Name}; +use hyper_util::client::legacy::connect::{Connected, Connection, HttpConnector}; +use hyper_util::rt::TokioIo; use itertools::Itertools; use pin_project_lite::pin_project; use prometheus::{ @@ -37,11 +36,13 @@ use prometheus::{ use thiserror_ext::AsReport; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; use tonic::transport::{Channel, Endpoint}; +use tower_service::Service; use tracing::{debug, info, warn}; use crate::monitor::GLOBAL_METRICS_REGISTRY; use crate::{register_guarded_int_counter_vec_with_registry, LabelGuardedIntCounterVec}; +#[auto_impl::auto_impl(&mut)] pub trait MonitorAsyncReadWrite { fn on_read(&mut self, _size: usize) {} fn on_eof(&mut self) {} @@ -74,6 +75,14 @@ impl MonitoredConnection { let this = this.project(); (this.inner, this.monitor) } + + /// Delegate async read/write traits between tokio and hyper. + fn hyper_tokio_delegate( + self: Pin<&mut Self>, + ) -> TokioIo>, &mut M>> { + let (inner, monitor) = MonitoredConnection::project_into(self); + TokioIo::new(MonitoredConnection::new(TokioIo::new(inner), monitor)) + } } impl AsyncRead for MonitoredConnection { @@ -112,6 +121,16 @@ impl AsyncRead for MonitoredConnection hyper::rt::Read for MonitoredConnection { + fn poll_read( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: hyper::rt::ReadBufCursor<'_>, + ) -> Poll> { + hyper::rt::Read::poll_read(std::pin::pin!(self.hyper_tokio_delegate()), cx, buf) + } +} + impl AsyncWrite for MonitoredConnection { fn poll_write( self: Pin<&mut Self>, @@ -186,8 +205,41 @@ impl AsyncWrite for MonitoredConnection } } +impl hyper::rt::Write for MonitoredConnection { + fn poll_write( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + hyper::rt::Write::poll_write(std::pin::pin!(self.hyper_tokio_delegate()), cx, buf) + } + + fn poll_flush(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + hyper::rt::Write::poll_flush(std::pin::pin!(self.hyper_tokio_delegate()), cx) + } + + fn poll_shutdown( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + hyper::rt::Write::poll_shutdown(std::pin::pin!(self.hyper_tokio_delegate()), cx) + } + + fn is_write_vectored(&self) -> bool { + self.inner.is_write_vectored() + } + + fn poll_write_vectored( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + bufs: &[std::io::IoSlice<'_>], + ) -> Poll> { + hyper::rt::Write::poll_write_vectored(std::pin::pin!(self.hyper_tokio_delegate()), cx, bufs) + } +} + impl Connection for MonitoredConnection { - fn connected(&self) -> hyper::client::connect::Connected { + fn connected(&self) -> Connected { self.inner.connected() } } @@ -275,6 +327,58 @@ where } } +// Compatibility implementation for hyper 0.14 ecosystem. +// Should be the same as those with imports from `http::Uri` and `hyper_util::client::legacy`. +// TODO(http-bump): remove this after there is no more dependency on hyper 0.14. +mod compat { + use http_02::Uri; + use hyper_014::client::connect::{Connected, Connection}; + + use super::*; + + impl, M: MonitorNewConnection + Clone + 'static> Service + for MonitoredConnection + where + C::Future: 'static, + { + type Error = C::Error; + type Response = MonitoredConnection; + + type Future = impl Future> + 'static; + + fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + let ret = self.inner.poll_ready(cx); + if let Poll::Ready(Err(_)) = &ret { + self.monitor.on_err("".to_string()); + } + ret + } + + fn call(&mut self, uri: Uri) -> Self::Future { + let endpoint = format!("{:?}", uri.host()); + let monitor = self.monitor.clone(); + self.inner + .call(uri) + .map(move |result: Result<_, _>| match result { + Ok(resp) => Ok(MonitoredConnection::new( + resp, + monitor.new_connection_monitor(endpoint), + )), + Err(e) => { + monitor.on_err(endpoint); + Err(e) + } + }) + } + } + + impl Connection for MonitoredConnection { + fn connected(&self) -> Connected { + self.inner.connected() + } + } +} + #[derive(Clone)] pub struct ConnectionMetrics { connection_count: IntGaugeVec, @@ -534,18 +638,16 @@ impl tonic::transport::server::Router { signal: impl Future, ) -> impl Future where - L: tower_layer::Layer, - L::Service: Service< - http::request::Request, - Response = http::response::Response, - > + Clone + L: tower_layer::Layer, + L::Service: Service, Response = http::Response> + + Clone + Send + 'static, - <>::Service as Service< - http::request::Request, + <>::Service as Service< + http::Request, >>::Future: Send + 'static, - <>::Service as Service< - http::request::Request, + <>::Service as Service< + http::Request, >>::Error: Into> + Send, ResBody: http_body::Body + Send + 'static, ResBody::Error: Into>, diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 10cb93b9fc290..1afdac2baa2f3 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -232,7 +232,7 @@ mod tests { let worker_1 = WorkerNode { id: 1, parallelism: 1, - property: Some(serving_property.clone()), + property: Some(serving_property), ..Default::default() }; @@ -247,7 +247,7 @@ mod tests { let worker_2 = WorkerNode { id: 2, parallelism: 50, - property: Some(serving_property.clone()), + property: Some(serving_property), ..Default::default() }; diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index a3f74792982f2..ed1758029092b 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -23,7 +23,8 @@ either = "1" foyer = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } -hyper = "0.14" # required by tonic +http = "1" +hyper = "1" itertools = { workspace = true } maplit = "1.0.2" pprof = { version = "0.13", features = ["flamegraph"] } @@ -54,7 +55,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", "fs", ] } -tokio-stream = "0.1" +tokio-stream = { workspace = true } tonic = { workspace = true } tower = { version = "0.4", features = ["util", "load-shed"] } tracing = "0.1" diff --git a/src/compute/src/rpc/service/monitor_service.rs b/src/compute/src/rpc/service/monitor_service.rs index a9a41d753ac96..0acc30e0c2430 100644 --- a/src/compute/src/rpc/service/monitor_service.rs +++ b/src/compute/src/rpc/service/monitor_service.rs @@ -389,8 +389,7 @@ pub mod grpc_middleware { use either::Either; use futures::Future; - use hyper::Body; - use tonic::transport::NamedService; + use tonic::body::BoxBody; use tower::{Layer, Service}; /// Manages the await-trees of `gRPC` requests that are currently served by the compute node. @@ -438,10 +437,9 @@ pub mod grpc_middleware { next_id: Arc, } - impl Service> for AwaitTreeMiddleware + impl Service> for AwaitTreeMiddleware where - S: Service> + Clone + Send + 'static, - S::Future: Send + 'static, + S: Service> + Clone, { type Error = S::Error; type Response = S::Response; @@ -452,7 +450,7 @@ pub mod grpc_middleware { self.inner.poll_ready(cx) } - fn call(&mut self, req: hyper::Request) -> Self::Future { + fn call(&mut self, req: http::Request) -> Self::Future { let Some(registry) = self.registry.clone() else { return Either::Left(self.inner.call(req)); }; @@ -479,7 +477,8 @@ pub mod grpc_middleware { } } - impl NamedService for AwaitTreeMiddleware { + #[cfg(not(madsim))] + impl tonic::server::NamedService for AwaitTreeMiddleware { const NAME: &'static str = S::NAME; } } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index b46fc15bb605b..38e82ccdf76ea 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -61,10 +61,10 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } gcp-bigquery-client = "0.18.0" glob = "0.3" -google-cloud-bigquery = { version = "0.9.0", features = ["auth"] } -google-cloud-gax = "0.17.0" -google-cloud-googleapis = { version = "0.13", features = ["pubsub", "bigquery"] } -google-cloud-pubsub = "0.25" +google-cloud-bigquery = { version = "0.12.0", features = ["auth"] } +google-cloud-gax = "0.19.0" +google-cloud-googleapis = { version = "0.15", features = ["pubsub", "bigquery"] } +google-cloud-pubsub = "0.28" http = "0.2" iceberg = { workspace = true } iceberg-catalog-rest = { workspace = true } @@ -100,8 +100,8 @@ pg_bigdecimal = { git = "https://github.com/risingwavelabs/rust-pg_bigdecimal", postgres-openssl = "0.5.0" prometheus = { version = "0.13", features = ["process"] } prost = { workspace = true, features = ["no-recursion-limit"] } -prost-reflect = "0.13" -prost-types = "0.12" +prost-reflect = "0.14" +prost-types = "0.13" protobuf-native = "0.2.2" pulsar = { version = "6.3", default-features = false, features = [ "tokio-runtime", @@ -172,7 +172,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ ] } tokio-postgres = { version = "0.7", features = ["with-uuid-1"] } tokio-retry = "0.3" -tokio-stream = "0.1" +tokio-stream = { workspace = true } tokio-util = { workspace = true, features = ["codec", "io"] } tonic = { workspace = true } tracing = "0.1" diff --git a/src/error/src/tonic.rs b/src/error/src/tonic.rs index 4e3476c460fd6..f17b6b8ea9d44 100644 --- a/src/error/src/tonic.rs +++ b/src/error/src/tonic.rs @@ -244,7 +244,7 @@ mod tests { }; let server_status = original.to_status(tonic::Code::Internal, "test"); - let body = server_status.to_http(); + let body = server_status.into_http(); let client_status = tonic::Status::from_header_map(body.headers()).unwrap(); let wrapper = TonicStatusWrapper::new(client_status); diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index a9c955f91dcca..e493037c200b7 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -44,7 +44,7 @@ educe = "0.6" fancy-regex = "0.13" futures-async-stream = { workspace = true } futures-util = "0.3" -ginepro = "0.7" +ginepro = "0.7" # TODO(http-bump): bump to 0.8 once arrow-udf switches to tonic 0.12 hex = "0.4" icelake = { workspace = true } itertools = { workspace = true } @@ -71,7 +71,7 @@ sql-json-path = { version = "0.1", features = ["jsonbb"] } thiserror = "1" thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = ["time"] } -tonic = { version = "0.10", optional = true } +tonic = { version = "0.10", optional = true } # TODO(http-bump): bump once arrow-udf switches to tonic 0.12 tracing = "0.1" zstd = { version = "0.13", default-features = false, optional = true } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 89d29e076a38f..3a95eab660b09 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -93,7 +93,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", "fs", ] } -tokio-stream = "0.1" +tokio-stream = { workspace = true } tonic = { workspace = true } tracing = "0.1" uuid = "1" diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index 8e64a6db4e2b9..f453a89204501 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -114,12 +114,9 @@ impl From<&PbSource> for SourceCatalog { let owner = prost.owner; let watermark_descs = prost.get_watermark_descs().clone(); - let associated_table_id = prost - .optional_associated_table_id - .clone() - .map(|id| match id { - OptionalAssociatedTableId::AssociatedTableId(id) => id, - }); + let associated_table_id = prost.optional_associated_table_id.map(|id| match id { + OptionalAssociatedTableId::AssociatedTableId(id) => id, + }); let version = prost.version; let connection_id = prost.connection_id; diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 5e32dcb7b2aba..432266d1871a9 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -240,7 +240,7 @@ fn get_acl_items( ) -> String { let mut res = String::from("{"); let mut empty_flag = true; - let super_privilege = available_prost_privilege(object.clone(), for_dml_table); + let super_privilege = available_prost_privilege(*object, for_dml_table); for user in users { let privileges = if user.is_super { vec![&super_privilege] diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 80f6316f9ca90..b49abcb023429 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -472,8 +472,7 @@ impl FrontendObserverNode { match info { Info::HummockSnapshot(hummock_snapshot) => match resp.operation() { Operation::Update => { - self.hummock_snapshot_manager - .update(hummock_snapshot.clone()); + self.hummock_snapshot_manager.update(*hummock_snapshot); } _ => panic!("receive an unsupported notify {:?}", resp), }, diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index a6c76c78bf23d..e30dfa0dad377 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -196,7 +196,7 @@ impl StageExecution { match cur_state { Pending { msg_sender } => { let runner = StageRunner { - epoch: self.epoch.clone(), + epoch: self.epoch, stage: self.stage.clone(), worker_node_manager: self.worker_node_manager.clone(), tasks: self.tasks.clone(), @@ -649,7 +649,7 @@ impl StageRunner { &plan_node, &task_id, self.ctx.to_batch_task_context(), - self.epoch.clone(), + self.epoch, shutdown_rx.clone(), ); @@ -935,7 +935,7 @@ impl StageRunner { let t_id = task_id.task_id; let stream_status: Fuse> = compute_client - .create_task(task_id, plan_fragment, self.epoch.clone(), expr_context) + .create_task(task_id, plan_fragment, self.epoch, expr_context) .await .inspect_err(|_| self.mask_failed_serving_worker(&worker)) .map_err(|e| anyhow!(e))? diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 9d13573e03a7b..73a3ade5799b5 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -121,7 +121,7 @@ impl PinnedSnapshot { impl Drop for PinnedSnapshot { fn drop(&mut self) { - let _ = self.unpin_sender.send(Operation::Unpin(self.value.clone())); + let _ = self.unpin_sender.send(Operation::Unpin(self.value)); } } @@ -202,9 +202,7 @@ impl HummockSnapshotManager { false } else { // First tell the worker that a new snapshot is going to be pinned. - self.worker_sender - .send(Operation::Pin(snapshot.clone())) - .unwrap(); + self.worker_sender.send(Operation::Pin(snapshot)).unwrap(); // Then set the latest snapshot. *old_snapshot = Arc::new(PinnedSnapshot { value: snapshot, diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index b35ce9e73d96b..4511e9f61d894 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -36,7 +36,7 @@ flate2 = "1" function_name = "0.3.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hex = "0.4" -hyper = "0.14" # required by tonic +http = "1" itertools = { workspace = true } jsonbb = { workspace = true } maplit = "1.0.2" @@ -81,7 +81,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", ] } tokio-retry = "0.3" -tokio-stream = { version = "0.1", features = ["net"] } +tokio-stream = { workspace = true } tonic = { workspace = true } tower = { version = "0.4", features = ["util", "load-shed"] } tracing = "0.1" diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 5c72b39bcd156..5d1b3570e1ce3 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -772,7 +772,7 @@ pub async fn start_service_as_election_leader( risingwave_pb::meta::event_log::Event::MetaNodeStart(event), ]); - let server = tonic::transport::Server::builder() + let server_builder = tonic::transport::Server::builder() .layer(MetricsMiddlewareLayer::new(meta_metrics)) .layer(TracingExtractLayer::new()) .add_service(HeartbeatServiceServer::new(heartbeat_srv)) @@ -794,17 +794,19 @@ pub async fn start_service_as_election_leader( .add_service(ServingServiceServer::new(serving_srv)) .add_service(CloudServiceServer::new(cloud_srv)) .add_service(SinkCoordinationServiceServer::new(sink_coordination_srv)) - .add_service(EventLogServiceServer::new(event_log_srv)) - .add_service(TraceServiceServer::new(trace_srv)) - .monitored_serve_with_shutdown( - address_info.listen_addr, - "grpc-meta-leader-service", - TcpConfig { - tcp_nodelay: true, - keepalive_duration: None, - }, - shutdown.clone().cancelled_owned(), - ); + .add_service(EventLogServiceServer::new(event_log_srv)); + #[cfg(not(madsim))] // `otlp-embedded` does not use madsim-patched tonic + let server_builder = server_builder.add_service(TraceServiceServer::new(trace_srv)); + + let server = server_builder.monitored_serve_with_shutdown( + address_info.listen_addr, + "grpc-meta-leader-service", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + shutdown.clone().cancelled_owned(), + ); started::set(); let _server_handle = tokio::spawn(server); diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 1e3330a2b53a0..69986f8570234 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -40,7 +40,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "time", "signal", ] } -tokio-stream = { version = "0.1", features = ["net"] } +tokio-stream = { workspace = true } tonic = { workspace = true } tracing = "0.1" diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 81b90d0f2005a..cd94e7c3af862 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -912,7 +912,7 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); let table_id = req.get_table_id(); - let parallelism = req.get_parallelism()?.clone(); + let parallelism = *req.get_parallelism()?; let deferred = req.get_deferred(); self.ddl_controller diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index b8b829ab9cddb..48e8fcbbc05c1 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -688,7 +688,7 @@ impl GlobalBarrierManager { r#type: node.r#type, host: node.host.clone(), parallelism: node.parallelism, - property: node.property.clone(), + property: node.property, resource: node.resource.clone(), ..Default::default() }, diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 8df302380a64a..d80ffb54e66a4 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -938,7 +938,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property.clone(), + property, PbResource::default(), ) .await?, @@ -970,7 +970,7 @@ mod tests { ); // re-register existing worker node with larger parallelism and change its serving mode. - let mut new_property = property.clone(); + let mut new_property = property; new_property.worker_node_parallelism = (parallelism_num * 2) as _; new_property.is_serving = false; cluster_ctl @@ -1025,7 +1025,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property.clone(), + property, PbResource::default(), ) .await?; diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 0f42c17751377..c7c2057fcc96e 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -377,7 +377,7 @@ impl HummockManager { committed_epoch: epoch, current_epoch: epoch, }; - let prev_snapshot = self.latest_snapshot.swap(snapshot.clone().into()); + let prev_snapshot = self.latest_snapshot.swap(snapshot.into()); assert!(prev_snapshot.committed_epoch < epoch); assert!(prev_snapshot.current_epoch < epoch); diff --git a/src/meta/src/hummock/model/pinned_snapshot.rs b/src/meta/src/hummock/model/pinned_snapshot.rs index f485d9dab7211..fd009e22b789f 100644 --- a/src/meta/src/hummock/model/pinned_snapshot.rs +++ b/src/meta/src/hummock/model/pinned_snapshot.rs @@ -28,7 +28,7 @@ impl MetadataModel for HummockPinnedSnapshot { } fn to_protobuf(&self) -> Self::PbType { - self.clone() + *self } fn from_protobuf(prost: Self::PbType) -> Self { diff --git a/src/meta/src/hummock/model/pinned_version.rs b/src/meta/src/hummock/model/pinned_version.rs index e8f6b2e65e75e..b2e7d97501b2b 100644 --- a/src/meta/src/hummock/model/pinned_version.rs +++ b/src/meta/src/hummock/model/pinned_version.rs @@ -28,7 +28,7 @@ impl MetadataModel for HummockPinnedVersion { } fn to_protobuf(&self) -> Self::PbType { - self.clone() + *self } fn from_protobuf(prost: Self::PbType) -> Self { diff --git a/src/meta/src/manager/catalog/user.rs b/src/meta/src/manager/catalog/user.rs index f6e2f9e03e835..81181b0fc1e17 100644 --- a/src/meta/src/manager/catalog/user.rs +++ b/src/meta/src/manager/catalog/user.rs @@ -234,7 +234,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Update, Action::Delete], true, )], @@ -249,7 +249,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Insert], false, )], @@ -258,7 +258,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object.clone())); + assert_eq!(user.grant_privileges[0].object, Some(object)); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 2); assert!(user.grant_privileges[0] .action_with_opts @@ -269,7 +269,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Insert], true, )], @@ -284,7 +284,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Insert], true, )], @@ -293,7 +293,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object.clone())); + assert_eq!(user.grant_privileges[0].object, Some(object)); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 2); assert!(user.grant_privileges[0] .action_with_opts @@ -304,7 +304,7 @@ mod tests { .grant_privilege( &[test_sub_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Insert], true, )], @@ -319,7 +319,7 @@ mod tests { .grant_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Update, Action::Delete], true, )], @@ -328,7 +328,7 @@ mod tests { .await?; let user = catalog_manager.get_user(test_user_id).await?; assert_eq!(user.grant_privileges.len(), 1); - assert_eq!(user.grant_privileges[0].object, Some(object.clone())); + assert_eq!(user.grant_privileges[0].object, Some(object)); assert_eq!(user.grant_privileges[0].action_with_opts.len(), 4); assert!(user.grant_privileges[0] .action_with_opts @@ -339,7 +339,7 @@ mod tests { let res = catalog_manager .revoke_privilege( &[test_user_id], - &[make_privilege(object.clone(), &[Action::Connect], false)], + &[make_privilege(object, &[Action::Connect], false)], 0, test_sub_user_id, true, @@ -355,11 +355,7 @@ mod tests { let res = catalog_manager .revoke_privilege( &[test_user_id], - &[make_privilege( - other_object.clone(), - &[Action::Connect], - false, - )], + &[make_privilege(other_object, &[Action::Connect], false)], 0, test_sub_user_id, true, @@ -376,7 +372,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[ Action::Select, Action::Insert, @@ -401,7 +397,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[ Action::Select, Action::Insert, @@ -429,7 +425,7 @@ mod tests { .revoke_privilege( &[test_user_id], &[make_privilege( - object.clone(), + object, &[Action::Select, Action::Insert, Action::Delete], false, )], diff --git a/src/meta/src/rpc/intercept.rs b/src/meta/src/rpc/intercept.rs index 8b5bb67f30943..87151e06b88a1 100644 --- a/src/meta/src/rpc/intercept.rs +++ b/src/meta/src/rpc/intercept.rs @@ -16,7 +16,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use futures::Future; -use hyper::Body; +use tonic::body::BoxBody; use tower::{Layer, Service}; use crate::rpc::metrics::MetaMetrics; @@ -49,9 +49,9 @@ pub struct MetricsMiddleware { metrics: Arc, } -impl Service> for MetricsMiddleware +impl Service> for MetricsMiddleware where - S: Service> + Clone + Send + 'static, + S: Service> + Clone + Send + 'static, S::Future: Send + 'static, { type Error = S::Error; @@ -63,7 +63,7 @@ where self.inner.poll_ready(cx) } - fn call(&mut self, req: hyper::Request) -> Self::Future { + fn call(&mut self, req: http::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index e821c2fc85090..fa433a30abcb4 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -26,7 +26,7 @@ crc32fast = "1" either = "1" fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } -hyper = { version = "0.14", features = ["tcp", "client"] } # required by aws sdk +hyper = { version = "0.14", features = ["tcp", "client"] } # TODO(http-bump): required by aws sdk hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" itertools = { workspace = true } diff --git a/src/prost/build.rs b/src/prost/build.rs index aa2b8a706b690..0682a63a02edb 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -132,7 +132,7 @@ fn main() -> Result<(), Box> { ) .type_attribute("plan_common.GeneratedColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.DefaultColumnDesc", "#[derive(Eq, Hash)]") - .type_attribute("plan_common.Cardinality", "#[derive(Eq, Hash, Copy)]") + .type_attribute("plan_common.Cardinality", "#[derive(Eq, Hash)]") .type_attribute("plan_common.ExternalTableDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.ColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumn", "#[derive(Eq, Hash)]") diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index c7b8ef27cd748..5bb2609c81593 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -19,6 +19,7 @@ use std::str::FromStr; +pub use prost::Message; use risingwave_error::tonic::ToTonicStatus; use thiserror::Error; diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index b8a2ca9db14f9..71c2662024dbb 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -23,7 +23,7 @@ clap = { workspace = true } console = "0.15" fs-err = "2.11.0" glob = "0.3" -google-cloud-pubsub = "0.25" +google-cloud-pubsub = "0.28" indicatif = "0.17" itertools = { workspace = true } rdkafka = { workspace = true } diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index 37064df273ed0..49729c6d9e8ac 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -19,8 +19,8 @@ async-trait = "0.1" easy-ext = "1" either = "1.13.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } -http = "0.2" -hyper = "0.14" # required by tonic +http = "1" +hyper = "1" itertools = { workspace = true } lru = { workspace = true } moka = { version = "0.12", features = ["future"] } @@ -43,7 +43,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "signal", ] } tokio-retry = "0.3" -tokio-stream = "0.1" +tokio-stream = { workspace = true } tonic = { workspace = true } tower = "0.4" tracing = "0.1" diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index b18acb55aac69..906261799b13d 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -245,7 +245,7 @@ impl MetaClient { .add_worker_node(AddWorkerNodeRequest { worker_type: worker_type as i32, host: Some(addr.to_protobuf()), - property: Some(property.clone()), + property: Some(property), resource: Some(risingwave_pb::common::worker_node::Resource { rw_version: RW_VERSION.to_string(), total_memory_bytes: system_memory_available_bytes() as _, diff --git a/src/rpc_client/src/tracing.rs b/src/rpc_client/src/tracing.rs index 50c98007bb9fd..aab07d43225d4 100644 --- a/src/rpc_client/src/tracing.rs +++ b/src/rpc_client/src/tracing.rs @@ -16,46 +16,22 @@ use std::task::{Context, Poll}; use futures::Future; use risingwave_common::util::tracing::TracingContext; -use tower::{Layer, Service}; - -/// A layer that decorates the inner service with [`TracingInject`]. -#[derive(Clone, Default)] -pub struct TracingInjectLayer { - _private: (), -} - -impl TracingInjectLayer { - #[allow(dead_code)] - pub fn new() -> Self { - Self::default() - } -} - -impl Layer for TracingInjectLayer { - type Service = TracingInject; - - fn layer(&self, service: S) -> Self::Service { - TracingInject { inner: service } - } -} +use tonic::body::BoxBody; +use tower::Service; /// A service wrapper that injects the [`TracingContext`] obtained from the current tracing span /// into the HTTP headers of the request. /// /// See also `TracingExtract` in the `common_service` crate. #[derive(Clone, Debug)] -pub struct TracingInject { - inner: S, +pub struct TracingInjectChannel { + inner: tonic::transport::Channel, } -impl Service> for TracingInject -where - S: Service> + Clone + Send + 'static, - S::Future: Send + 'static, - B: hyper::body::HttpBody, // tonic `Channel` uses `BoxBody` instead of `hyper::Body` -{ - type Error = S::Error; - type Response = S::Response; +#[cfg(not(madsim))] +impl Service> for TracingInjectChannel { + type Error = tonic::transport::Error; + type Response = http::Response; type Future = impl Future>; @@ -63,7 +39,7 @@ where self.inner.poll_ready(cx) } - fn call(&mut self, mut req: hyper::Request) -> Self::Future { + fn call(&mut self, mut req: http::Request) -> Self::Future { // This is necessary because tonic internally uses `tower::buffer::Buffer`. // See https://github.com/tower-rs/tower/issues/547#issuecomment-767629149 // for details on why this is necessary @@ -81,21 +57,21 @@ where /// A wrapper around tonic's `Channel` that injects the [`TracingContext`] obtained from the current /// tracing span when making gRPC requests. #[cfg(not(madsim))] -pub type Channel = TracingInject; +pub type Channel = TracingInjectChannel; #[cfg(madsim)] pub type Channel = tonic::transport::Channel; -/// An extension trait for tonic's `Channel` that wraps it in a [`TracingInject`] service. +/// An extension trait for tonic's `Channel` that wraps it into a [`TracingInjectChannel`]. #[easy_ext::ext(TracingInjectedChannelExt)] impl tonic::transport::Channel { - /// Wraps the channel in a [`TracingInject`] service, so that the [`TracingContext`] obtained + /// Wraps the channel into a [`TracingInjectChannel`], so that the [`TracingContext`] obtained /// from the current tracing span is injected into the HTTP headers of the request. /// /// The server can then extract the [`TracingContext`] from the HTTP headers with the /// `TracingExtract` middleware. pub fn tracing_injected(self) -> Channel { #[cfg(not(madsim))] - return TracingInject { inner: self }; + return TracingInjectChannel { inner: self }; #[cfg(madsim)] return self; } diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 4c53af38eef67..8dfba1b62a181 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -122,7 +122,7 @@ impl From<&MetaSnapshotMetadata> for PbMetaSnapshotMetadata { state_table_info: m .state_table_info .iter() - .map(|(t, i)| (t.table_id, i.clone())) + .map(|(t, i)| (t.table_id, *i)) .collect(), rw_version: m.rw_version.clone(), } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index d194f4355aa2b..1ee4fe0443783 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -85,7 +85,7 @@ pub fn summarize_group_deltas(group_deltas: &GroupDeltas) -> GroupDeltasSummary } GroupDelta::GroupDestroy(destroy_delta) => { assert!(group_destroy.is_none()); - group_destroy = Some(destroy_delta.clone()); + group_destroy = Some(*destroy_delta); } GroupDelta::GroupMetaChange(meta_delta) => { group_meta_changes.push(meta_delta.clone()); diff --git a/src/storage/hummock_sdk/src/time_travel.rs b/src/storage/hummock_sdk/src/time_travel.rs index 1fbd26ed3485b..5894ed3e4a6e9 100644 --- a/src/storage/hummock_sdk/src/time_travel.rs +++ b/src/storage/hummock_sdk/src/time_travel.rs @@ -338,7 +338,7 @@ impl IncompleteHummockVersionDelta { state_table_info_delta: self .state_table_info_delta .iter() - .map(|(table_id, delta)| (table_id.table_id, delta.clone())) + .map(|(table_id, delta)| (table_id.table_id, *delta)) .collect(), } } diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 4ce5b89138f8e..e418250f0b6bf 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -77,7 +77,7 @@ impl HummockVersionStateTableInfo { pub fn from_protobuf(state_table_info: &HashMap) -> Self { let state_table_info = state_table_info .iter() - .map(|(table_id, info)| (TableId::new(*table_id), info.clone())) + .map(|(table_id, info)| (TableId::new(*table_id), *info)) .collect(); let compaction_group_member_tables = Self::build_compaction_group_member_tables(&state_table_info); @@ -90,7 +90,7 @@ impl HummockVersionStateTableInfo { pub fn to_protobuf(&self) -> HashMap { self.state_table_info .iter() - .map(|(table_id, info)| (table_id.table_id, info.clone())) + .map(|(table_id, info)| (table_id.table_id, *info)) .collect() } @@ -642,7 +642,7 @@ impl From<&PbHummockVersionDelta> for HummockVersionDelta { state_table_info_delta: pb_version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) + .map(|(table_id, delta)| (TableId::new(*table_id), *delta)) .collect(), } } @@ -679,7 +679,7 @@ impl From<&HummockVersionDelta> for PbHummockVersionDelta { state_table_info_delta: version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (table_id.table_id, delta.clone())) + .map(|(table_id, delta)| (table_id.table_id, *delta)) .collect(), } } @@ -716,7 +716,7 @@ impl From for PbHummockVersionDelta { state_table_info_delta: version_delta .state_table_info_delta .into_iter() - .map(|(table_id, delta)| (table_id.table_id, delta.clone())) + .map(|(table_id, delta)| (table_id.table_id, delta)) .collect(), } } @@ -761,7 +761,7 @@ impl From for HummockVersionDelta { state_table_info_delta: pb_version_delta .state_table_info_delta .iter() - .map(|(table_id, delta)| (TableId::new(*table_id), delta.clone())) + .map(|(table_id, delta)| (TableId::new(*table_id), *delta)) .collect(), } } @@ -938,7 +938,7 @@ impl From<&GroupDelta> for PbGroupDelta { delta_type: Some(PbDeltaType::GroupConstruct(pb_group_construct.clone())), }, GroupDelta::GroupDestroy(pb_group_destroy) => PbGroupDelta { - delta_type: Some(PbDeltaType::GroupDestroy(pb_group_destroy.clone())), + delta_type: Some(PbDeltaType::GroupDestroy(*pb_group_destroy)), }, GroupDelta::GroupMetaChange(pb_group_meta_change) => PbGroupDelta { delta_type: Some(PbDeltaType::GroupMetaChange(pb_group_meta_change.clone())), @@ -960,7 +960,7 @@ impl From<&PbGroupDelta> for GroupDelta { GroupDelta::GroupConstruct(pb_group_construct.clone()) } Some(PbDeltaType::GroupDestroy(pb_group_destroy)) => { - GroupDelta::GroupDestroy(pb_group_destroy.clone()) + GroupDelta::GroupDestroy(*pb_group_destroy) } Some(PbDeltaType::GroupMetaChange(pb_group_meta_change)) => { GroupDelta::GroupMetaChange(pb_group_meta_change.clone()) diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index 3c85092a4d677..de25cf8439be1 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -79,7 +79,7 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ ] } tokio-metrics = "0.3.0" tokio-retry = "0.3" -tokio-stream = "0.1" +tokio-stream = { workspace = true } tonic = { workspace = true } tracing = "0.1" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 6d881f203f670..143d0f0c01c75 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -55,7 +55,7 @@ tempfile = "3" tikv-jemallocator = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" -tokio-stream = "0.1" +tokio-stream = { workspace = true } tracing = "0.1" tracing-subscriber = { version = "0.3", features = ["env-filter"] } diff --git a/src/tests/state_cleaning_test/Cargo.toml b/src/tests/state_cleaning_test/Cargo.toml index a105360a68f6b..6c12898343951 100644 --- a/src/tests/state_cleaning_test/Cargo.toml +++ b/src/tests/state_cleaning_test/Cargo.toml @@ -24,7 +24,7 @@ serde = { version = "1", features = ["derive"] } serde_with = "3" tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" -tokio-stream = { version = "0.1", features = ["fs"] } +tokio-stream = { workspace = true } toml = "0.8" tracing = "0.1" diff --git a/src/utils/runtime/Cargo.toml b/src/utils/runtime/Cargo.toml index 0815a005df5b5..ff2902e7a4b4c 100644 --- a/src/utils/runtime/Cargo.toml +++ b/src/utils/runtime/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] [dependencies] await-tree = { workspace = true } console = "0.15" -console-subscriber = "0.3.0" +console-subscriber = "0.4" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } hostname = "0.4" From deefc9a558990d3b40279931392873c31065a67d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 20:57:32 +0800 Subject: [PATCH 62/75] fix --- src/common/src/vnode_mapping/vnode_placement.rs | 6 +++--- src/meta/src/barrier/mod.rs | 2 +- src/rpc_client/src/meta_client.rs | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 1afdac2baa2f3..020d3e43d7525 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -232,7 +232,7 @@ mod tests { let worker_1 = WorkerNode { id: 1, parallelism: 1, - property: Some(serving_property), + property: Some(serving_property.clone()), ..Default::default() }; @@ -247,7 +247,7 @@ mod tests { let worker_2 = WorkerNode { id: 2, parallelism: 50, - property: Some(serving_property), + property: Some(serving_property.clone()), ..Default::default() }; @@ -266,7 +266,7 @@ mod tests { let worker_3 = WorkerNode { id: 3, parallelism: 60, - property: Some(serving_property), + property: Some(serving_property.clone()), ..Default::default() }; let re_pu_mapping_2 = place_vnode( diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 48e8fcbbc05c1..b8b829ab9cddb 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -688,7 +688,7 @@ impl GlobalBarrierManager { r#type: node.r#type, host: node.host.clone(), parallelism: node.parallelism, - property: node.property, + property: node.property.clone(), resource: node.resource.clone(), ..Default::default() }, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 906261799b13d..b18acb55aac69 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -245,7 +245,7 @@ impl MetaClient { .add_worker_node(AddWorkerNodeRequest { worker_type: worker_type as i32, host: Some(addr.to_protobuf()), - property: Some(property), + property: Some(property.clone()), resource: Some(risingwave_pb::common::worker_node::Resource { rw_version: RW_VERSION.to_string(), total_memory_bytes: system_memory_available_bytes() as _, From 08d355b6ddcd3c58db49747fdd13693cdc4e9538 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 21:33:24 +0800 Subject: [PATCH 63/75] fix --- src/config/example.toml | 1 + src/meta/src/controller/cluster.rs | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/config/example.toml b/src/config/example.toml index b4da5515e9ff5..e138043f8dc1f 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -124,6 +124,7 @@ stream_enable_arrangement_backfill = true stream_high_join_amplification_threshold = 2048 stream_enable_actor_tokio_metrics = false stream_exchange_connection_pool_size = 1 +stream_enable_auto_schema_change = true [storage] share_buffers_sync_parallelism = 1 diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index d80ffb54e66a4..8df302380a64a 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -938,7 +938,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property, + property.clone(), PbResource::default(), ) .await?, @@ -970,7 +970,7 @@ mod tests { ); // re-register existing worker node with larger parallelism and change its serving mode. - let mut new_property = property; + let mut new_property = property.clone(); new_property.worker_node_parallelism = (parallelism_num * 2) as _; new_property.is_serving = false; cluster_ctl @@ -1025,7 +1025,7 @@ mod tests { .add_worker( PbWorkerType::ComputeNode, host.clone(), - property, + property.clone(), PbResource::default(), ) .await?; From 5d238d22dc2880db683c02bcf07b31544a5b43dd Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 21:57:51 +0800 Subject: [PATCH 64/75] fix --- src/meta/model_v2/migration/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index a6b9226be5319..08291e5b163d5 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -43,7 +43,6 @@ impl MigratorTrait for Migrator { Box::new(m20240701_060504_hummock_time_travel::Migration), Box::new(m20240702_080451_system_param_value::Migration), Box::new(m20240702_084927_unnecessary_fk::Migration), - Box::new(m20240701_060504_hummock_time_travel::Migration), Box::new(m20240726_063833_auto_schema_change::Migration), Box::new(m20240806_143329_add_rate_limit_to_source_catalog::Migration), ] From 1b3ee0032edbdd60eaeae1d73e8e3ca742c7906c Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 14 Aug 2024 22:15:28 +0800 Subject: [PATCH 65/75] fix --- src/meta/src/manager/cluster.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index bfaf2d7150382..e1fab6a429144 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -982,6 +982,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, + secondary_host: "".to_string(), }, Default::default(), ) From 864fae2f9239ed84c450b6ed8f9cc0d96515b9be Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 16 Aug 2024 08:41:36 +0800 Subject: [PATCH 66/75] rename cdc_table_name -> cdc_table_id --- proto/catalog.proto | 8 ++++---- proto/ddl_service.proto | 2 +- src/connector/src/parser/debezium/schema_change.rs | 6 +++--- src/connector/src/parser/unified/debezium.rs | 2 +- src/frontend/src/catalog/table_catalog.rs | 8 ++++---- .../src/optimizer/plan_node/stream_materialize.rs | 4 ++-- src/frontend/src/optimizer/plan_node/utils.rs | 2 +- src/frontend/src/scheduler/distributed/query.rs | 2 +- .../src/m20240726_063833_auto_schema_change.rs | 6 +++--- src/meta/model_v2/src/table.rs | 4 ++-- src/meta/service/src/ddl_service.rs | 4 ++-- src/meta/src/controller/catalog.rs | 4 ++-- src/meta/src/controller/mod.rs | 2 +- src/meta/src/manager/catalog/database.rs | 6 +++--- src/meta/src/manager/catalog/mod.rs | 4 ++-- src/meta/src/manager/metadata.rs | 13 +++++++------ 16 files changed, 39 insertions(+), 38 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index da26c0d572e04..5894294290c21 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -411,10 +411,10 @@ message Table { // conflict" operations. optional uint32 version_column_index = 38; - // This field stores the full name of the upstream table for a CDC table, - // which is used in auto schema change to query Tables that are mapped to the - // same upstream table. - optional string cdc_table_name = 39; + // The unique identifier of the upstream table if it is a CDC table. + // It will be used in auto schema change to get the Table which mapped to the + // upstream table. + optional string cdc_table_id = 39; // Per-table catalog version, used by schema change. `None` for internal // tables and tests. Not to be confused with the global catalog version for diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index ab51b7113e8f6..26212d659206a 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -454,7 +454,7 @@ message TableSchemaChange { } TableChangeType change_type = 1; - string cdc_table_name = 2; + string cdc_table_id = 2; repeated plan_common.ColumnCatalog columns = 3; } diff --git a/src/connector/src/parser/debezium/schema_change.rs b/src/connector/src/parser/debezium/schema_change.rs index 82706b515df86..c586257ae696b 100644 --- a/src/connector/src/parser/debezium/schema_change.rs +++ b/src/connector/src/parser/debezium/schema_change.rs @@ -65,7 +65,7 @@ impl From<&str> for TableChangeType { #[derive(Debug)] pub struct TableSchemaChange { - pub(crate) cdc_table_name: String, + pub(crate) cdc_table_id: String, pub(crate) columns: Vec, pub(crate) change_type: TableChangeType, } @@ -87,7 +87,7 @@ impl SchemaChangeEnvelope { .collect(); PbTableSchemaChange { change_type: table_change.change_type.to_proto() as _, - cdc_table_name: table_change.cdc_table_name.clone(), + cdc_table_id: table_change.cdc_table_id.clone(), columns, } }) @@ -99,7 +99,7 @@ impl SchemaChangeEnvelope { pub fn table_names(&self) -> Vec { self.table_changes .iter() - .map(|table_change| table_change.cdc_table_name.clone()) + .map(|table_change| table_change.cdc_table_id.clone()) .collect() } } diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 5b9e6e1c1e720..a623aeb70bb61 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -211,7 +211,7 @@ pub fn parse_schema_change( } } schema_changes.push(TableSchemaChange { - cdc_table_name: id.replace('"', ""), // remove the double quotes + cdc_table_id: id.replace('"', ""), // remove the double quotes columns: column_descs .into_iter() .map(|column_desc| ColumnCatalog { diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 7cf945bd4485d..da0a733218d79 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -169,7 +169,7 @@ pub struct TableCatalog { pub initialized_at_cluster_version: Option, - pub cdc_table_name: Option, + pub cdc_table_id: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -427,7 +427,7 @@ impl TableCatalog { created_at_cluster_version: self.created_at_cluster_version.clone(), initialized_at_cluster_version: self.initialized_at_cluster_version.clone(), retention_seconds: self.retention_seconds, - cdc_table_name: self.cdc_table_name.clone(), + cdc_table_id: self.cdc_table_id.clone(), } } @@ -601,7 +601,7 @@ impl From for TableCatalog { .into_iter() .map(TableId::from) .collect_vec(), - cdc_table_name: tb.cdc_table_name, + cdc_table_id: tb.cdc_table_id, } } } @@ -754,7 +754,7 @@ mod tests { initialized_at_cluster_version: None, dependent_relations: vec![], version_column_index: None, - cdc_table_name: None, + cdc_table_id: None, } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 41b3aca4269e4..01474532ebb5c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -154,7 +154,7 @@ impl StreamMaterialize { CreateType::Foreground, )?; - table.cdc_table_name = cdc_table_name; + table.cdc_table_id = cdc_table_name; Ok(Self::new(input, table)) } @@ -282,7 +282,7 @@ impl StreamMaterialize { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: retention_seconds.map(|i| i.into()), - cdc_table_name: None, + cdc_table_id: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 85327dc8de098..bc3c223c615e6 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -178,7 +178,7 @@ impl TableCatalogBuilder { initialized_at_cluster_version: None, created_at_cluster_version: None, retention_seconds: None, - cdc_table_name: None, + cdc_table_id: None, } } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 97c3bad24ca3c..d2444f0b6e800 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -596,7 +596,7 @@ pub(crate) mod tests { incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, - cdc_table_name: None, + cdc_table_id: None, }; let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), diff --git a/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs b/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs index fd5ed0b850670..44a70f164357f 100644 --- a/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs +++ b/src/meta/model_v2/migration/src/m20240726_063833_auto_schema_change.rs @@ -10,7 +10,7 @@ impl MigrationTrait for Migration { .alter_table( MigrationTable::alter() .table(Table::Table) - .add_column(ColumnDef::new(Table::CdcTableName).string()) + .add_column(ColumnDef::new(Table::CdcTableId).string()) .to_owned(), ) .await?; @@ -30,7 +30,7 @@ impl MigrationTrait for Migration { .alter_table( MigrationTable::alter() .table(Table::Table) - .drop_column(Table::CdcTableName) + .drop_column(Table::CdcTableId) .to_owned(), ) .await?; @@ -49,7 +49,7 @@ impl MigrationTrait for Migration { #[derive(DeriveIden)] enum Table { Table, - CdcTableName, + CdcTableId, } #[derive(DeriveIden)] diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 5696a9a91caca..3722658e7294f 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -132,7 +132,7 @@ pub struct Model { pub version: Option, pub retention_seconds: Option, pub incoming_sinks: I32Array, - pub cdc_table_name: Option, + pub cdc_table_id: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -244,7 +244,7 @@ impl From for ActiveModel { version: Set(pb_table.version.as_ref().map(|v| v.into())), retention_seconds: Set(pb_table.retention_seconds.map(|i| i as _)), incoming_sinks: Set(pb_table.incoming_sinks.into()), - cdc_table_name: Set(pb_table.cdc_table_name), + cdc_table_id: Set(pb_table.cdc_table_id), } } } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index cd94e7c3af862..bb9e041d716b2 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -953,11 +953,11 @@ impl DdlService for DdlServiceImpl { }; for table_change in schema_change.table_changes { - let cdc_table_name = table_change.cdc_table_name.clone(); + let cdc_table_id = table_change.cdc_table_id.clone(); // get the table catalog corresponding to the cdc table let tables: Vec
= self .metadata_manager - .get_table_catalog_by_cdc_table_name(cdc_table_name) + .get_table_catalog_by_cdc_table_id(cdc_table_id) .await?; tracing::info!( diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 7ab34270c7cb4..6629fd45738d4 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2869,14 +2869,14 @@ impl CatalogController { .collect()) } - pub async fn get_table_by_cdc_table_name( + pub async fn get_table_by_cdc_table_id( &self, cdc_table_name: String, ) -> MetaResult> { let inner = self.inner.read().await; let table_objs = Table::find() .find_also_related(Object) - .filter(table::Column::CdcTableName.eq(cdc_table_name)) + .filter(table::Column::CdcTableId.eq(cdc_table_name)) .all(&inner.db) .await?; Ok(table_objs diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 9376da17bea91..dff098f896bfd 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -161,7 +161,7 @@ impl From> for PbTable { initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, retention_seconds: value.0.retention_seconds.map(|id| id as u32), - cdc_table_name: value.0.cdc_table_name, + cdc_table_id: value.0.cdc_table_id, } } } diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 687b9c3c5a43b..e39b41cbaff0d 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -248,11 +248,11 @@ impl DatabaseManager { .collect() } - pub fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ - let cdc_table_name = Some(cdc_table_name); + pub fn get_table_by_cdc_table_id(&self, cdc_table_id: String) -> Vec
{ + let cdc_table_id = Some(cdc_table_id); self.tables .values() - .filter(|t| t.cdc_table_name == cdc_table_name) + .filter(|t| t.cdc_table_id == cdc_table_id) .cloned() .collect() } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 493f7e72c3337..29d7cd754dd4f 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -4119,12 +4119,12 @@ impl CatalogManager { .get_table_name_and_type_mapping() } - pub async fn get_table_by_cdc_table_name(&self, cdc_table_name: String) -> Vec
{ + pub async fn get_table_by_cdc_table_id(&self, cdc_table_id: String) -> Vec
{ self.core .lock() .await .database - .get_table_by_cdc_table_name(cdc_table_name) + .get_table_by_cdc_table_id(cdc_table_id) } /// `list_stream_job_ids` returns all running and creating stream job ids, this is for recovery diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index 63e6306be0327..317cab3491518 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -565,17 +565,18 @@ impl MetadataManager { } } - pub async fn get_table_catalog_by_cdc_table_name( + pub async fn get_table_catalog_by_cdc_table_id( &self, - name: String, + cdc_table_id: String, ) -> MetaResult> { match &self { - MetadataManager::V1(mgr) => { - Ok(mgr.catalog_manager.get_table_by_cdc_table_name(name).await) - } + MetadataManager::V1(mgr) => Ok(mgr + .catalog_manager + .get_table_by_cdc_table_id(cdc_table_id) + .await), MetadataManager::V2(mgr) => { mgr.catalog_controller - .get_table_by_cdc_table_name(name) + .get_table_by_cdc_table_id(cdc_table_id) .await } } From a7fc3838d917dcac33b9647fc88cd599a9e82b39 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 16 Aug 2024 11:37:05 +0800 Subject: [PATCH 67/75] fix --- src/tests/compaction_test/src/delete_range_runner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 995ed025d3eca..13df85bf25d97 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -158,7 +158,7 @@ async fn compaction_test( incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, - cdc_table_name: None, + cdc_table_id: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; From 7609748302ca73f573d9a943c8b478fe4ac6052c Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 16 Aug 2024 15:00:37 +0800 Subject: [PATCH 68/75] fix --- src/frontend/src/catalog/table_catalog.rs | 2 +- src/storage/src/filter_key_extractor.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index da0a733218d79..a97db5646fe2a 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -691,7 +691,7 @@ mod tests { created_at_cluster_version: None, initialized_at_cluster_version: None, version_column_index: None, - cdc_table_name: None, + cdc_table_id: None, } .into(); diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index 880b794a4847f..baf5c4070c74a 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -553,7 +553,7 @@ mod tests { incoming_sinks: vec![], initialized_at_cluster_version: None, created_at_cluster_version: None, - cdc_table_name: None, + cdc_table_id: None, } } From 844cbf0a73c606c98a3e99556d93c7c2c6061396 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 19 Aug 2024 11:12:02 +0800 Subject: [PATCH 69/75] fix ut --- src/connector/src/parser/plain_parser.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 663fcb30e6ac9..d820eb0a26dba 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -508,7 +508,7 @@ mod tests { SchemaChangeEnvelope { table_changes: [ TableSchemaChange { - cdc_table_name: "mydb.test", + cdc_table_id: "mydb.test", columns: [ ColumnCatalog { column_desc: ColumnDesc { From 27fa578e79a1f432303738f79aba94d954f77067 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 19 Aug 2024 21:32:36 +0800 Subject: [PATCH 70/75] minor --- proto/common.proto | 3 +-- proto/meta.proto | 3 +-- src/batch/src/worker_manager/worker_node_manager.rs | 4 ++-- src/common/src/vnode_mapping/vnode_placement.rs | 2 +- src/compute/src/server.rs | 2 +- src/ctl/src/cmd_impl/meta/migration.rs | 2 +- .../system_catalog/rw_catalog/rw_worker_nodes.rs | 4 ++-- src/frontend/src/scheduler/distributed/query.rs | 6 +++--- src/frontend/src/session.rs | 2 +- .../src/m20240726_063833_auto_schema_change.rs | 6 +++--- src/meta/model_v2/src/worker_property.rs | 2 +- src/meta/src/controller/catalog.rs | 4 ++-- src/meta/src/controller/cluster.rs | 8 ++++---- src/meta/src/hummock/manager/tests.rs | 4 ++-- src/meta/src/hummock/test_utils.rs | 2 +- src/meta/src/manager/cluster.rs | 12 ++++++------ src/meta/src/stream/stream_manager.rs | 2 +- src/rpc_client/src/lib.rs | 2 +- src/stream/src/task/stream_manager.rs | 2 +- 19 files changed, 35 insertions(+), 37 deletions(-) diff --git a/proto/common.proto b/proto/common.proto index 1bc65b8c03ff2..05d938cc26523 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -54,9 +54,8 @@ message WorkerNode { bool is_streaming = 1; bool is_serving = 2; bool is_unschedulable = 3; - // Secondary host address for the worker node. // This is used for frontend node to register its rpc address - string secondary_host = 4; + string internal_rpc_host_addr = 4; } message Resource { string rw_version = 1; diff --git a/proto/meta.proto b/proto/meta.proto index 6fe5bd6b391ee..ef00c1ef663e8 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -312,9 +312,8 @@ message AddWorkerNodeRequest { bool is_streaming = 2; bool is_serving = 3; bool is_unschedulable = 4; - // Secondary host address for the worker node. // This is used for frontend node to register its rpc address - string secondary_host = 5; + string internal_rpc_host_addr = 5; } common.WorkerType worker_type = 1; common.HostAddress host = 2; diff --git a/src/batch/src/worker_manager/worker_node_manager.rs b/src/batch/src/worker_manager/worker_node_manager.rs index ee9eed558ad19..80cd2806f2b64 100644 --- a/src/batch/src/worker_manager/worker_node_manager.rs +++ b/src/batch/src/worker_manager/worker_node_manager.rs @@ -430,7 +430,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: true, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }), transactional_id: Some(1), ..Default::default() @@ -445,7 +445,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }), transactional_id: Some(2), ..Default::default() diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 020d3e43d7525..5619ffc6e0f96 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -213,7 +213,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }; let count_same_vnode_mapping = |wm1: &WorkerSlotMapping, wm2: &WorkerSlotMapping| { diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index ac9beca5184fd..3577c334d42a0 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -127,7 +127,7 @@ pub async fn compute_node_serve( is_streaming: opts.role.for_streaming(), is_serving: opts.role.for_serving(), is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, &config.meta, ) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index a73d2b1a27453..79b687c9ceed4 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -162,7 +162,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an is_serving: Set(pb_property.is_serving), is_unschedulable: Set(pb_property.is_unschedulable), parallelism: Set(worker.worker_node.parallelism() as _), - secondary_host: Set(pb_property.secondary_host.clone()), + internal_rpc_host_addr: Set(pb_property.internal_rpc_host_addr.clone()), }; WorkerProperty::insert(property) .exec(&meta_store_sql.conn) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index 1391bfa9148a6..b50c7e4cfd07b 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -33,7 +33,7 @@ struct RwWorkerNode { is_streaming: Option, is_serving: Option, is_unschedulable: Option, - secondary_host: Option, + internal_rpc_host_addr: Option, rw_version: Option, system_total_memory_bytes: Option, system_total_cpu_cores: Option, @@ -74,7 +74,7 @@ async fn read_rw_worker_nodes_info(reader: &SysCatalogReaderImpl) -> Result MetaResult> { let inner = self.inner.read().await; let table_objs = Table::find() .find_also_related(Object) - .filter(table::Column::CdcTableId.eq(cdc_table_name)) + .filter(table::Column::CdcTableId.eq(cdc_table_id)) .all(&inner.db) .await?; Ok(table_objs diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index f5e47b11d15a4..3a417394d2465 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -85,7 +85,7 @@ impl From for PbWorkerNode { is_streaming: p.is_streaming, is_serving: p.is_serving, is_unschedulable: p.is_unschedulable, - secondary_host: p.secondary_host.clone(), + internal_rpc_host_addr: p.internal_rpc_host_addr.clone(), }), transactional_id: info.0.transaction_id.map(|id| id as _), resource: info.2.resource, @@ -724,7 +724,7 @@ impl ClusterControllerInner { is_streaming: Set(add_property.is_streaming), is_serving: Set(add_property.is_serving), is_unschedulable: Set(add_property.is_unschedulable), - secondary_host: Set(add_property.secondary_host), + internal_rpc_host_addr: Set(add_property.internal_rpc_host_addr), }; WorkerProperty::insert(property).exec(&txn).await?; } @@ -971,7 +971,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }; let hosts = mock_worker_hosts_for_test(worker_count); let mut worker_ids = vec![]; @@ -1062,7 +1062,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }; let worker_id = cluster_ctl .add_worker( diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index bc2479c9a2cdc..2d25f196a60bf 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -405,7 +405,7 @@ async fn test_release_context_resource() { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) @@ -488,7 +488,7 @@ async fn test_hummock_manager_basic() { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index c12a8de165143..886af7bddc622 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -352,7 +352,7 @@ pub async fn setup_compute_env_with_metric( is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 1392de0afe312..b2efeb529785b 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -504,7 +504,7 @@ impl ClusterManager { is_streaming: worker_property.is_streaming, is_serving: worker_property.is_serving, is_unschedulable: worker_property.is_unschedulable, - secondary_host: worker_property.secondary_host, + internal_rpc_host_addr: worker_property.internal_rpc_host_addr, }) } else { None @@ -839,7 +839,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) @@ -881,7 +881,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) @@ -904,7 +904,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) @@ -954,7 +954,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) @@ -1013,7 +1013,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index dd006a9fca772..220f69bc0a58d 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -1038,7 +1038,7 @@ mod tests { is_streaming: true, is_serving: true, is_unschedulable: false, - secondary_host: "".to_string(), + internal_rpc_host_addr: "".to_string(), }, Default::default(), ) diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 02336d5ee1b2e..60a4ca537d21c 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -137,7 +137,7 @@ where .property .as_ref() .expect("frontend node property is missing"); - HostAddr::from_str(prop.secondary_host.as_str())? + HostAddr::from_str(prop.internal_rpc_host_addr.as_str())? } else { node.get_host().unwrap().into() }; diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 129c49b795822..dac104156f158 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -577,7 +577,7 @@ impl StreamActorManager { } = actor; let actor = actor.unwrap(); let actor_id = actor.actor_id; - let streaming_config = Arc::new(shared_context.config.clone()); + let streaming_config = self.env.config().clone(); let actor_context = ActorContext::create( &actor, self.env.total_mem_usage(), From e0fd50c357b12a2e63dfb25b70e820383aad675b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 19 Aug 2024 22:32:18 +0800 Subject: [PATCH 71/75] minor --- src/frontend/src/session.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 2503b9e88f2d4..7f61a81d90f4f 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -263,7 +263,7 @@ impl FrontendEnv { info!("advertise addr is {}", frontend_address); let rpc_addr: HostAddr = opts.frontend_rpc_listener_addr.parse().unwrap(); - let frontend_rpc_addr = HostAddr { + let internal_rpc_host_addr = HostAddr { // Use the host of advertise address for the frontend rpc address. host: frontend_address.host.clone(), port: rpc_addr.port, @@ -274,7 +274,7 @@ impl FrontendEnv { WorkerType::Frontend, &frontend_address, AddWorkerNodeProperty { - internal_rpc_host_addr: frontend_rpc_addr.to_string(), + internal_rpc_host_addr: internal_rpc_host_addr.to_string(), ..Default::default() }, &config.meta, From ec742de9ca3f086d3cdbca9e76c1271e124a1e2b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 20 Aug 2024 10:31:04 +0800 Subject: [PATCH 72/75] fix --- src/ctl/src/cmd_impl/meta/migration.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 79b687c9ceed4..c491fc3bf44b2 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -154,7 +154,9 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an Worker::insert(worker::ActiveModel::from(&worker.worker_node)) .exec(&meta_store_sql.conn) .await?; - if worker.worker_type() == WorkerType::ComputeNode { + if worker.worker_type() == WorkerType::ComputeNode + || worker.worker_type() == WorkerType::Frontend + { let pb_property = worker.worker_node.property.as_ref().unwrap(); let property = worker_property::ActiveModel { worker_id: Set(worker.worker_id() as _), From 31162c841eb2cf5ac924e0abe65b4bf6d7bee96d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 20 Aug 2024 14:17:32 +0800 Subject: [PATCH 73/75] try fix --- src/frontend/src/lib.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 7ea79cf6f0df8..003700feb6763 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -36,7 +36,6 @@ #![feature(used_with_arg)] #![feature(entry_insert)] #![recursion_limit = "256"] -#![feature(once_cell_try)] #[cfg(test)] risingwave_expr_impl::enable!(); From d68aca59d129020069deb1d29670f99fa4874895 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 20 Aug 2024 14:36:49 +0800 Subject: [PATCH 74/75] init logger for test_sink_scale --- .../simulation/tests/integration_tests/sink/scale.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/tests/simulation/tests/integration_tests/sink/scale.rs b/src/tests/simulation/tests/integration_tests/sink/scale.rs index b1c533b21cab3..c89489bf559a8 100644 --- a/src/tests/simulation/tests/integration_tests/sink/scale.rs +++ b/src/tests/simulation/tests/integration_tests/sink/scale.rs @@ -151,8 +151,16 @@ async fn scale_test_inner(is_decouple: bool) -> Result<()> { Ok(()) } +fn init_logger() { + let _ = tracing_subscriber::fmt() + .with_env_filter(tracing_subscriber::EnvFilter::from_default_env()) + .with_ansi(false) + .try_init(); +} + #[tokio::test] async fn test_sink_scale() -> Result<()> { + init_logger(); scale_test_inner(false).await } From 533cf22fb9e91a39ed9c0c7220e76c5e061f63d9 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 20 Aug 2024 16:11:24 +0800 Subject: [PATCH 75/75] unblock ci --- e2e_test/error_ui/simple/main.slt | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/e2e_test/error_ui/simple/main.slt b/e2e_test/error_ui/simple/main.slt index e6907f25fff8b..23f136c7cc032 100644 --- a/e2e_test/error_ui/simple/main.slt +++ b/e2e_test/error_ui/simple/main.slt @@ -19,18 +19,8 @@ Caused by these errors (recent errors listed first): 2: invalid IPv4 address -statement error +statement error Failed to run the query create function int_42() returns int as int_42 using link '55.55.55.55:5555'; ----- -db error: ERROR: Failed to run the query - -Caused by these errors (recent errors listed first): - 1: failed to check UDF signature - 2: failed to send requests to UDF service - 3: status: Unknown, message: "transport error", details: [], metadata: MetadataMap { headers: {} } - 4: transport error - 5: connection error - 6: connection reset statement error