Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

fix(cdc): fix some cdc backfill issues #13397

Merged
merged 3 commits into from
Nov 14, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
55 changes: 24 additions & 31 deletions src/connector/src/source/external.rs
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
// limitations under the License.

use std::collections::HashMap;
use std::future::Future;

use anyhow::anyhow;
use futures::stream::BoxStream;
Expand Down Expand Up @@ -62,14 +61,14 @@ impl CdcTableType {
matches!(self, Self::MySql)
}

pub fn create_table_reader(
pub async fn create_table_reader(
&self,
properties: HashMap<String, String>,
schema: Schema,
) -> ConnectorResult<ExternalTableReaderImpl> {
match self {
Self::MySql => Ok(ExternalTableReaderImpl::MySql(
MySqlExternalTableReader::new(properties, schema)?,
MySqlExternalTableReader::new(properties, schema).await?,
)),
_ => bail!(ConnectorError::Config(anyhow!(
"invalid external table type: {:?}",
Expand Down Expand Up @@ -212,7 +211,7 @@ impl MySqlOffset {
pub trait ExternalTableReader {
fn get_normalized_table_name(&self, table_name: &SchemaTableName) -> String;

fn current_cdc_offset(&self) -> impl Future<Output = ConnectorResult<CdcOffset>> + Send + '_;
async fn current_cdc_offset(&self) -> ConnectorResult<CdcOffset>;

fn parse_binlog_offset(&self, offset: &str) -> ConnectorResult<CdcOffset>;

Expand All @@ -232,10 +231,11 @@ pub enum ExternalTableReaderImpl {

#[derive(Debug)]
pub struct MySqlExternalTableReader {
pool: mysql_async::Pool,
config: ExternalTableConfig,
rw_schema: Schema,
field_names: String,
// use mutex to provide shared mutable access to the connection
conn: tokio::sync::Mutex<mysql_async::Conn>,
}

#[derive(Debug, Clone, Deserialize)]
Expand All @@ -260,11 +260,7 @@ impl ExternalTableReader for MySqlExternalTableReader {
}

async fn current_cdc_offset(&self) -> ConnectorResult<CdcOffset> {
let mut conn = self
.pool
.get_conn()
.await
.map_err(|e| ConnectorError::Connection(anyhow!(e)))?;
let mut conn = self.conn.lock().await;

let sql = "SHOW MASTER STATUS".to_string();
let mut rs = conn.query::<mysql_async::Row, _>(sql).await?;
Expand Down Expand Up @@ -294,7 +290,10 @@ impl ExternalTableReader for MySqlExternalTableReader {
}

impl MySqlExternalTableReader {
pub fn new(properties: HashMap<String, String>, rw_schema: Schema) -> ConnectorResult<Self> {
pub async fn new(
properties: HashMap<String, String>,
rw_schema: Schema,
) -> ConnectorResult<Self> {
tracing::debug!(?rw_schema, "create mysql external table reader");

let config = serde_json::from_value::<ExternalTableConfig>(
Expand All @@ -308,7 +307,8 @@ impl MySqlExternalTableReader {
"mysql://{}:{}@{}:{}/{}",
config.username, config.password, config.host, config.port, config.database
);
let pool = mysql_async::Pool::from_url(database_url)?;
let opts = mysql_async::Opts::from_url(&database_url).map_err(mysql_async::Error::Url)?;
let conn = mysql_async::Conn::new(opts).await?;

let field_names = rw_schema
.fields
Expand All @@ -318,18 +318,13 @@ impl MySqlExternalTableReader {
.join(",");

Ok(Self {
pool,
config,
rw_schema,
field_names,
conn: tokio::sync::Mutex::new(conn),
})
}

pub async fn disconnect(&self) -> ConnectorResult<()> {
self.pool.clone().disconnect().await?;
Ok(())
}

#[try_stream(boxed, ok = OwnedRow, error = ConnectorError)]
async fn snapshot_read_inner(
&self,
Expand All @@ -356,11 +351,7 @@ impl MySqlExternalTableReader {
)
};

let mut conn = self
.pool
.get_conn()
.await
.map_err(|e| ConnectorError::Connection(anyhow!(e)))?;
let mut conn = self.conn.lock().await;

// Set session timezone to UTC
conn.exec_drop("SET time_zone = \"+00:00\"", ()).await?;
Expand Down Expand Up @@ -428,7 +419,7 @@ impl MySqlExternalTableReader {

let rs_stream = sql
.with(Params::from(params))
.stream::<mysql_async::Row, _>(&mut conn)
.stream::<mysql_async::Row, _>(&mut *conn)
.await?;

let row_stream = rs_stream.map(|row| {
Expand Down Expand Up @@ -587,10 +578,10 @@ mod tests {
sink_id: Default::default(),
properties: Default::default(),
columns: vec![
ColumnDesc::unnamed(ColumnId::new(1), DataType::Int32),
ColumnDesc::unnamed(ColumnId::new(2), DataType::Decimal),
ColumnDesc::unnamed(ColumnId::new(3), DataType::Varchar),
ColumnDesc::unnamed(ColumnId::new(4), DataType::Date),
ColumnDesc::named("v1".into(), ColumnId::new(1), DataType::Int32),
ColumnDesc::named("v2".into(), ColumnId::new(2), DataType::Decimal),
ColumnDesc::named("v3".into(), ColumnId::new(3), DataType::Varchar),
ColumnDesc::named("v4".into(), ColumnId::new(4), DataType::Date),
],
downstream_pk: vec![0],
sink_type: SinkType::AppendOnly,
Expand All @@ -605,15 +596,17 @@ mod tests {
"port" => "8306",
"username" => "root",
"password" => "123456",
"database.name" => "mydb",
"database.name" => "mytest",
"table.name" => "t1"));

let reader = MySqlExternalTableReader::new(props, rw_schema).unwrap();
let reader = MySqlExternalTableReader::new(props, rw_schema)
.await
.unwrap();
let offset = reader.current_cdc_offset().await.unwrap();
println!("BinlogOffset: {:?}", offset);

let table_name = SchemaTableName {
schema_name: "mydb".to_string(),
schema_name: "mytest".to_string(),
table_name: "t1".to_string(),
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,8 @@
) from mysql_mydb table 'mydb.t1';
explain_output: |
StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite }
└─StreamDml { columns: [v1, v2] }
└─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] }
└─StreamExchange { dist: HashShard(mydb.t1.v1) }
└─StreamDml { columns: [v1, v2] }
└─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] }
with_config_map:
CDC_BACKFILL: 'true'
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ pub struct StreamCdcTableScan {
impl StreamCdcTableScan {
pub fn new(core: generic::Scan) -> Self {
let batch_plan_id = core.ctx.next_plan_node_id();
let distribution = Distribution::Single;
let distribution = Distribution::SomeShard;
let base = PlanBase::new_stream_with_core(
&core,
distribution,
Expand Down
4 changes: 4 additions & 0 deletions src/sqlparser/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -2,3 +2,7 @@

This parser is a fork of <https://github.com/andygrove/sqlparser-rs>.


## Add a new test case
1. Copy an item in the yaml file and edit the `input` to the sql you want to test
2. Run `./risedev do-apply-parser-test` to regenerate the `formatted_sql` whicih is the expected output
2 changes: 1 addition & 1 deletion src/sqlparser/src/ast/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1573,7 +1573,7 @@ impl fmt::Display for Statement {
}
if let Some(info) = cdc_table_info {
write!(f, " FROM {}", info.source_name)?;
write!(f, " TABLE {}", info.external_table_name)?;
write!(f, " TABLE '{}'", info.external_table_name)?;
}
Ok(())
}
Expand Down
4 changes: 4 additions & 0 deletions src/sqlparser/tests/testdata/create.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,10 @@
error_msg: 'sql parser error: expect description of the format'
- input: CREATE SOURCE src FORMAT PLAIN ENCODE JSON
formatted_sql: CREATE SOURCE src FORMAT PLAIN ENCODE JSON
- input: CREATE SOURCE mysql_src with ( connector = 'mysql-cdc', hostname = 'localhost', port = '3306', database.name = 'mytest', server.id = '5601' )
formatted_sql: CREATE SOURCE mysql_src WITH (connector = 'mysql-cdc', hostname = 'localhost', port = '3306', database.name = 'mytest', server.id = '5601') FORMAT PLAIN ENCODE JSON
- input: CREATE TABLE sbtest10 (id INT PRIMARY KEY, k INT, c CHARACTER VARYING, pad CHARACTER VARYING) FROM sbtest TABLE 'mydb.sbtest10'
formatted_sql: CREATE TABLE sbtest10 (id INT PRIMARY KEY, k INT, c CHARACTER VARYING, pad CHARACTER VARYING) FROM sbtest TABLE 'mydb.sbtest10'
- input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://')
formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://')
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }'
Expand Down
3 changes: 3 additions & 0 deletions src/stream/src/executor/backfill/cdc/cdc_backfill.rs
Original file line number Diff line number Diff line change
Expand Up @@ -446,6 +446,9 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
state_impl.mutate_state(last_binlog_offset).await?;
}

// drop reader to release db connection
drop(upstream_table_reader);

tracing::info!(
actor = self.actor_id,
"CdcBackfill has already finished and forward messages directly to the downstream"
Expand Down
2 changes: 1 addition & 1 deletion src/stream/src/from_proto/source/trad_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ impl ExecutorBuilder for SourceExecutorBuilder {
.collect_vec();

let table_reader = table_type
.create_table_reader(source.properties.clone(), table_schema.clone())?;
.create_table_reader(source.properties.clone(), table_schema.clone()).await?;
let external_table = ExternalStorageTable::new(
TableId::new(source.source_id),
upstream_table_name,
Expand Down
5 changes: 3 additions & 2 deletions src/stream/src/from_proto/stream_scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -76,8 +76,9 @@ impl ExecutorBuilder for StreamScanExecutorBuilder {
.map(|(k, v)| (k.clone(), v.clone()))
.collect();
let table_type = CdcTableType::from_properties(&properties);
let table_reader =
table_type.create_table_reader(properties.clone(), table_schema.clone())?;
let table_reader = table_type
.create_table_reader(properties.clone(), table_schema.clone())
.await?;

let table_pk_order_types = table_desc
.pk
Expand Down
Loading