Skip to content

Commit 7ec0c4b

Browse files
committed
Reapply "feat(stream): create cdc table reader and source data stream with retry (#19467)"
This reverts commit ae0341a.
1 parent be5d9b7 commit 7ec0c4b

File tree

9 files changed

+315
-88
lines changed

9 files changed

+315
-88
lines changed

src/compute/tests/cdc_tests.rs

+5-17
Original file line numberDiff line numberDiff line change
@@ -32,10 +32,8 @@ use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field,
3232
use risingwave_common::types::{Datum, JsonbVal};
3333
use risingwave_common::util::epoch::{test_epoch, EpochExt};
3434
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
35-
use risingwave_connector::source::cdc::external::mock_external_table::MockExternalTableReader;
36-
use risingwave_connector::source::cdc::external::mysql::MySqlOffset;
3735
use risingwave_connector::source::cdc::external::{
38-
DebeziumOffset, DebeziumSourceOffset, ExternalTableReaderImpl, SchemaTableName,
36+
CdcTableType, DebeziumOffset, DebeziumSourceOffset, ExternalTableConfig, SchemaTableName,
3937
};
4038
use risingwave_connector::source::cdc::DebeziumCdcSplit;
4139
use risingwave_connector::source::SplitImpl;
@@ -160,19 +158,6 @@ async fn test_cdc_backfill() -> StreamResult<()> {
160158
MockOffsetGenExecutor::new(source).boxed(),
161159
);
162160

163-
let binlog_file = String::from("1.binlog");
164-
// mock binlog watermarks for backfill
165-
// initial low watermark: 1.binlog, pos=2 and expected behaviors:
166-
// - ignore events before (1.binlog, pos=2);
167-
// - apply events in the range of (1.binlog, pos=2, 1.binlog, pos=4) to the snapshot
168-
let binlog_watermarks = vec![
169-
MySqlOffset::new(binlog_file.clone(), 2), // binlog low watermark
170-
MySqlOffset::new(binlog_file.clone(), 4),
171-
MySqlOffset::new(binlog_file.clone(), 6),
172-
MySqlOffset::new(binlog_file.clone(), 8),
173-
MySqlOffset::new(binlog_file.clone(), 10),
174-
];
175-
176161
let table_name = SchemaTableName {
177162
schema_name: "public".to_string(),
178163
table_name: "mock_table".to_string(),
@@ -183,11 +168,14 @@ async fn test_cdc_backfill() -> StreamResult<()> {
183168
]);
184169
let table_pk_indices = vec![0];
185170
let table_pk_order_types = vec![OrderType::ascending()];
171+
let config = ExternalTableConfig::default();
172+
186173
let external_table = ExternalStorageTable::new(
187174
TableId::new(1234),
188175
table_name,
189176
"mydb".to_string(),
190-
ExternalTableReaderImpl::Mock(MockExternalTableReader::new(binlog_watermarks)),
177+
config,
178+
CdcTableType::Mock,
191179
table_schema.clone(),
192180
table_pk_order_types,
193181
table_pk_indices.clone(),

src/connector/src/source/cdc/external/mock_external_table.rs

+13-1
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,19 @@ pub struct MockExternalTableReader {
3131
}
3232

3333
impl MockExternalTableReader {
34-
pub fn new(binlog_watermarks: Vec<MySqlOffset>) -> Self {
34+
pub fn new() -> Self {
35+
let binlog_file = String::from("1.binlog");
36+
// mock binlog watermarks for backfill
37+
// initial low watermark: 1.binlog, pos=2 and expected behaviors:
38+
// - ignore events before (1.binlog, pos=2);
39+
// - apply events in the range of (1.binlog, pos=2, 1.binlog, pos=4) to the snapshot
40+
let binlog_watermarks = vec![
41+
MySqlOffset::new(binlog_file.clone(), 2), // binlog low watermark
42+
MySqlOffset::new(binlog_file.clone(), 4),
43+
MySqlOffset::new(binlog_file.clone(), 6),
44+
MySqlOffset::new(binlog_file.clone(), 8),
45+
MySqlOffset::new(binlog_file.clone(), 10),
46+
];
3547
Self {
3648
binlog_watermarks,
3749
snapshot_cnt: AtomicUsize::new(0),

src/connector/src/source/cdc/external/mod.rs

+4-2
Original file line numberDiff line numberDiff line change
@@ -45,9 +45,10 @@ use crate::source::cdc::external::sql_server::{
4545
use crate::source::cdc::CdcSourceType;
4646
use crate::WithPropertiesExt;
4747

48-
#[derive(Debug)]
48+
#[derive(Debug, Clone)]
4949
pub enum CdcTableType {
5050
Undefined,
51+
Mock,
5152
MySql,
5253
Postgres,
5354
SqlServer,
@@ -97,6 +98,7 @@ impl CdcTableType {
9798
Self::SqlServer => Ok(ExternalTableReaderImpl::SqlServer(
9899
SqlServerExternalTableReader::new(config, schema, pk_indices).await?,
99100
)),
101+
Self::Mock => Ok(ExternalTableReaderImpl::Mock(MockExternalTableReader::new())),
100102
_ => bail!("invalid external table type: {:?}", *self),
101103
}
102104
}
@@ -214,7 +216,7 @@ pub enum ExternalTableReaderImpl {
214216
Mock(MockExternalTableReader),
215217
}
216218

217-
#[derive(Debug, Clone, Deserialize)]
219+
#[derive(Debug, Default, Clone, Deserialize)]
218220
pub struct ExternalTableConfig {
219221
pub connector: String,
220222

src/stream/src/executor/backfill/cdc/cdc_backfill.rs

+91-12
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@
1212
// See the License for the specific language governing permissions and
1313
// limitations under the License.
1414

15+
use std::future::Future;
1516
use std::pin::Pin;
1617

1718
use either::Either;
@@ -27,9 +28,11 @@ use risingwave_connector::parser::{
2728
ByteStreamSourceParser, DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties,
2829
ProtocolProperties, SourceStreamChunkBuilder, SpecificParserConfig,
2930
};
30-
use risingwave_connector::source::cdc::external::CdcOffset;
31+
use risingwave_connector::source::cdc::external::{CdcOffset, ExternalTableReaderImpl};
3132
use risingwave_connector::source::{SourceColumnDesc, SourceContext};
3233
use rw_futures_util::pausable;
34+
use thiserror_ext::AsReport;
35+
use tracing::Instrument;
3336

3437
use crate::executor::backfill::cdc::state::CdcBackfillState;
3538
use crate::executor::backfill::cdc::upstream_table::external::ExternalStorageTable;
@@ -42,6 +45,7 @@ use crate::executor::backfill::utils::{
4245
use crate::executor::backfill::CdcScanOptions;
4346
use crate::executor::monitor::CdcBackfillMetrics;
4447
use crate::executor::prelude::*;
48+
use crate::executor::source::get_infinite_backoff_strategy;
4549
use crate::executor::UpdateMutation;
4650
use crate::task::CreateMviewProgressReporter;
4751

@@ -140,7 +144,6 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
140144
let upstream_table_name = self.external_table.qualified_table_name();
141145
let schema_table_name = self.external_table.schema_table_name().clone();
142146
let external_database_name = self.external_table.database_name().to_owned();
143-
let upstream_table_reader = UpstreamTableReader::new(self.external_table);
144147

145148
let additional_columns = self
146149
.output_columns
@@ -168,29 +171,85 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
168171
// if not, we should bypass the backfill directly.
169172
let mut state_impl = self.state_impl;
170173

171-
let mut upstream = transform_upstream(upstream, &self.output_columns)
172-
.boxed()
173-
.peekable();
174-
175174
state_impl.init_epoch(first_barrier_epoch).await?;
176175

177176
// restore backfill state
178177
let state = state_impl.restore_state().await?;
179178
current_pk_pos = state.current_pk_pos.clone();
180179

181-
let to_backfill = !self.options.disable_backfill && !state.is_finished;
180+
let need_backfill = !self.options.disable_backfill && !state.is_finished;
182181

183182
// Keep track of rows from the snapshot.
184183
let mut total_snapshot_row_count = state.row_count as u64;
185184

185+
// After init the state table and forward the initial barrier to downstream,
186+
// we now try to create the table reader with retry.
187+
// If backfill hasn't finished, we can ignore upstream cdc events before we create the table reader;
188+
// If backfill is finished, we should forward the upstream cdc events to downstream.
189+
let mut table_reader: Option<ExternalTableReaderImpl> = None;
190+
let external_table = self.external_table.clone();
191+
let mut future = Box::pin(async move {
192+
let backoff = get_infinite_backoff_strategy();
193+
tokio_retry::Retry::spawn(backoff, || async {
194+
match external_table.create_table_reader().await {
195+
Ok(reader) => Ok(reader),
196+
Err(e) => {
197+
tracing::warn!(error = %e.as_report(), "failed to create cdc table reader, retrying...");
198+
Err(e)
199+
}
200+
}
201+
})
202+
.instrument(tracing::info_span!("create_cdc_table_reader_with_retry"))
203+
.await
204+
.expect("Retry create cdc table reader until success.")
205+
});
206+
loop {
207+
if let Some(msg) =
208+
build_reader_and_poll_upstream(&mut upstream, &mut table_reader, &mut future)
209+
.await?
210+
{
211+
match msg {
212+
Message::Barrier(barrier) => {
213+
// commit state to bump the epoch of state table
214+
state_impl.commit_state(barrier.epoch).await?;
215+
yield Message::Barrier(barrier);
216+
}
217+
Message::Chunk(chunk) => {
218+
if need_backfill {
219+
// ignore chunk if we need backfill, since we can read the data from the snapshot
220+
} else {
221+
// forward the chunk to downstream
222+
yield Message::Chunk(chunk);
223+
}
224+
}
225+
Message::Watermark(_) => {
226+
// ignore watermark
227+
}
228+
}
229+
} else {
230+
assert!(table_reader.is_some(), "table reader must created");
231+
tracing::info!(
232+
table_id,
233+
upstream_table_name,
234+
"table reader created successfully"
235+
);
236+
break;
237+
}
238+
}
239+
240+
let upstream_table_reader = UpstreamTableReader::new(
241+
self.external_table.clone(),
242+
table_reader.expect("table reader must created"),
243+
);
244+
245+
let mut upstream = transform_upstream(upstream, &self.output_columns)
246+
.boxed()
247+
.peekable();
186248
let mut last_binlog_offset: Option<CdcOffset> = state
187249
.last_cdc_offset
188250
.map_or(upstream_table_reader.current_cdc_offset().await?, Some);
189251

190-
let offset_parse_func = upstream_table_reader
191-
.inner()
192-
.table_reader()
193-
.get_cdc_offset_parser();
252+
let offset_parse_func = upstream_table_reader.reader.get_cdc_offset_parser();
194253
let mut consumed_binlog_offset: Option<CdcOffset> = None;
195254

196255
tracing::info!(
@@ -227,7 +286,7 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
227286
// finished.
228287
//
229288
// Once the backfill loop ends, we forward the upstream directly to the downstream.
230-
if to_backfill {
289+
if need_backfill {
231290
// drive the upstream changelog first to ensure we can receive timely changelog event,
232291
// otherwise the upstream changelog may be blocked by the snapshot read stream
233292
let _ = Pin::new(&mut upstream).peek().await;
@@ -702,6 +761,26 @@ impl<S: StateStore> CdcBackfillExecutor<S> {
702761
}
703762
}
704763

764+
async fn build_reader_and_poll_upstream(
765+
upstream: &mut BoxedMessageStream,
766+
table_reader: &mut Option<ExternalTableReaderImpl>,
767+
future: &mut Pin<Box<impl Future<Output = ExternalTableReaderImpl>>>,
768+
) -> StreamExecutorResult<Option<Message>> {
769+
if table_reader.is_some() {
770+
return Ok(None);
771+
}
772+
tokio::select! {
773+
biased;
774+
reader = &mut *future => {
775+
*table_reader = Some(reader);
776+
Ok(None)
777+
}
778+
msg = upstream.next() => {
779+
msg.transpose()
780+
}
781+
}
782+
}
783+
705784
#[try_stream(ok = Message, error = StreamExecutorError)]
706785
pub async fn transform_upstream(upstream: BoxedMessageStream, output_columns: &[ColumnDesc]) {
707786
let props = SpecificParserConfig {

src/stream/src/executor/backfill/cdc/upstream_table/external.rs

+30-6
Original file line numberDiff line numberDiff line change
@@ -14,9 +14,14 @@
1414

1515
use risingwave_common::catalog::{Schema, TableId};
1616
use risingwave_common::util::sort_util::OrderType;
17-
use risingwave_connector::source::cdc::external::{ExternalTableReaderImpl, SchemaTableName};
17+
use risingwave_connector::error::ConnectorResult;
18+
use risingwave_connector::source::cdc::external::{
19+
CdcOffset, CdcTableType, ExternalTableConfig, ExternalTableReader, ExternalTableReaderImpl,
20+
SchemaTableName,
21+
};
1822

1923
/// This struct represents an external table to be read during backfill
24+
#[derive(Debug, Clone)]
2025
pub struct ExternalStorageTable {
2126
/// Id for this table.
2227
table_id: TableId,
@@ -28,7 +33,9 @@ pub struct ExternalStorageTable {
2833

2934
database_name: String,
3035

31-
table_reader: ExternalTableReaderImpl,
36+
config: ExternalTableConfig,
37+
38+
table_type: CdcTableType,
3239

3340
/// The schema of the output columns, i.e., this table VIEWED BY some executor like
3441
/// `RowSeqScanExecutor`.
@@ -43,14 +50,16 @@ pub struct ExternalStorageTable {
4350
}
4451

4552
impl ExternalStorageTable {
53+
#[allow(clippy::too_many_arguments)]
4654
pub fn new(
4755
table_id: TableId,
4856
SchemaTableName {
4957
table_name,
5058
schema_name,
5159
}: SchemaTableName,
5260
database_name: String,
53-
table_reader: ExternalTableReaderImpl,
61+
config: ExternalTableConfig,
62+
table_type: CdcTableType,
5463
schema: Schema,
5564
pk_order_types: Vec<OrderType>,
5665
pk_indices: Vec<usize>,
@@ -60,7 +69,8 @@ impl ExternalStorageTable {
6069
table_name,
6170
schema_name,
6271
database_name,
63-
table_reader,
72+
config,
73+
table_type,
6474
schema,
6575
pk_order_types,
6676
pk_indices,
@@ -90,8 +100,14 @@ impl ExternalStorageTable {
90100
}
91101
}
92102

93-
pub fn table_reader(&self) -> &ExternalTableReaderImpl {
94-
&self.table_reader
103+
pub async fn create_table_reader(&self) -> ConnectorResult<ExternalTableReaderImpl> {
104+
self.table_type
105+
.create_table_reader(
106+
self.config.clone(),
107+
self.schema.clone(),
108+
self.pk_indices.clone(),
109+
)
110+
.await
95111
}
96112

97113
pub fn qualified_table_name(&self) -> String {
@@ -101,4 +117,12 @@ impl ExternalStorageTable {
101117
pub fn database_name(&self) -> &str {
102118
self.database_name.as_str()
103119
}
120+
121+
pub async fn current_cdc_offset(
122+
&self,
123+
table_reader: &ExternalTableReaderImpl,
124+
) -> ConnectorResult<Option<CdcOffset>> {
125+
let binlog = table_reader.current_cdc_offset().await?;
126+
Ok(Some(binlog))
127+
}
104128
}

0 commit comments

Comments
 (0)