risingwavelabs / risingwave

Best-in-class stream processing, analytics, and management. Perform continuous analytics, or build event-driven applications, real-time ETL pipelines, and feature stores in minutes. Unified streaming and batch. PostgreSQL compatible.
https://go.risingwave.com/slack
Apache License 2.0
6.8k stars 564 forks source link

CDC replication stopped in chaos-mesh (ch-benchmark-pg-cdc) test #15141

Closed fuyufjh closed 5 months ago

fuyufjh commented 7 months ago

Describe the bug

After the CN being killed and restarted, the CDC tables seems to lose sync with their upstream PG.

$ psql -h localhost -p 4567 -d dev -U root

dev=> select * from new_order;
 no_o_id | no_d_id | no_w_id 
---------+---------+---------
    2288 |       1 |       1
    2299 |       1 |       1
    2357 |       1 |       1
    2366 |       1 |       1
    2389 |       1 |       1
.... too many rows ...
    2883 |      10 |       1
    2888 |      10 |       1
    2947 |      10 |       1
    2952 |      10 |       1
(8024 rows)

$ psql -h localhost -p 5432 -U postgres 

postgres=# select * from new_order;
 no_o_id | no_d_id | no_w_id 
---------+---------+---------
   13357 |      10 |       1
   13353 |       7 |       1
   13357 |       7 |       1
   13300 |       2 |       1
   12957 |       3 |       1
   13376 |       5 |       1
   13355 |       7 |       1
   13358 |       7 |       1
   13359 |       7 |       1
   13326 |       9 |       1
   13328 |       9 |       1
   13354 |       7 |       1
   13356 |       7 |       1
   13327 |       9 |       1
(14 rows)

BuildKite Grafana Logs

namespace: longcmkf-20240220-022651

Error message/log

2024-02-20T02:36:14.667064577Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/C90F020' on replication slot 'customer_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/C90F020, minimum is 0/15B79B60': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/C90F020, minimum is 0/15B79B60
2024-02-20T02:36:14.667078575Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/C9F2078' on replication slot 'new_order_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/C9F2078, minimum is 0/14EB0598': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/C9F2078, minimum is 0/14EB0598
2024-02-20T02:36:14.669016214Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/107346E0' on replication slot 'stock_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/107346E0, minimum is 0/15E74680': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/107346E0, minimum is 0/15E74680
2024-02-20T02:36:14.674653919Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/4EECA108' on replication slot 'history_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/80CB5060': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/80CB5060
2024-02-20T02:36:14.674658103Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/9831AC8' on replication slot 'item_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/9831AC8, minimum is 0/81998D50': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/9831AC8, minimum is 0/81998D50
2024-02-20T02:36:14.679035212Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/4EECA108' on replication slot 'warehouse_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/81086F00': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/81086F00
2024-02-20T02:36:15.826899745Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/C9F2078' on replication slot 'new_order_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/C9F2078, minimum is 0/14EB0598': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/C9F2078, minimum is 0/14EB0598
2024-02-20T02:36:15.849238513Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/C90F020' on replication slot 'customer_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/C90F020, minimum is 0/15B79B60': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/C90F020, minimum is 0/15B79B60
2024-02-20T02:36:15.852313048Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/107346E0' on replication slot 'stock_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/107346E0, minimum is 0/15E74680': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/107346E0, minimum is 0/15E74680
2024-02-20T02:36:16.752002682Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/9831AC8' on replication slot 'item_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/9831AC8, minimum is 0/81998D50': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/9831AC8, minimum is 0/81998D50
2024-02-20T02:36:16.768027687Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/4EECA108' on replication slot 'history_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/80CB5060': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/80CB5060
2024-02-20T02:36:17.341700037Z  WARN risingwave_connector_node: Cannot seek to the last known offset '0/4EECA108' on replication slot 'warehouse_slot'. Error from server: 'ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/81086F00': org.postgresql.util.PSQLException: ERROR: cannot advance replication slot to 0/4EECA108, minimum is 0/81086F00

To Reproduce

No response

Expected behavior

No response

How did you deploy RisingWave?

No response

The version of RisingWave

No response

Additional context

No response

StrikeW commented 7 months ago

Some background go first

Our cdc connector consumes PG cdc events while acking to the PG server at regular intervals the offset (lsn) that has been consumed. Then upstream PG will assume that wal log of those offsets can be discarded. https://github.com/risingwavelabs/risingwave/blob/e6d8d88bcef5a388a0a7a0669d9418d25194f138/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java#L189-L199

DebeziumEngine will commit those marked offsets to upstream: https://github.com/debezium/debezium/blob/4ca2a67b0d302c611b89b1931728377cf232ab6c/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java#L435-L436

Findings

After some investigation, I think the reason for the "Cannot seek to the last known offset" error is that we ack the offset to PG before the checkpoint commit. So that when the cluster recovered from a committed checkpoint, the restored offset may already been discarded by upstream PG.

Currently our framework doesn't have a checkpoint commit callback mechanism to notify the source executor. An intuitive idea is to let Meta make a broadcast RPCs to each CNs in the cluster. cc @hzxa21

To confirm the findings, I increase the offset flush interval to 30mins which is much large than the time required for the test and rerun the chaos test (stresschaos only and w/o memtable spill: 599, 603 ), the results show that the "Cannot seek" error is gone and btw mv check is passed.

But! when I run chaos test with 3CNs (601) , even the error is gone, the source table is still unsynced with PG, I have no idea of this one right now.

fuyufjh commented 7 months ago

Currently our framework doesn't have a checkpoint commit callback mechanism to notify the source executor.

+1 for this.

But that will certainly take some time. Before that, shall we use some hacky way to work around the problem to unblock the CH-Benchmark chaos test?

For example, set PG's wal_keep_size to higher number to let PG keeps more WAL. Note that this only works for our testing env. Users will still run into this problem.

StrikeW commented 6 months ago

For example, set PG's wal_keep_size to higher number to let PG keeps more WAL. Note that this only works for our testing env. Users will still run into this problem.

Will confirm whether it can work.

hzxa21 commented 6 months ago

To confirm the findings, I increase the offset flush interval to 30mins which is much large than the time required for the test and rerun the chaos test (stresschaos only and w/o memtable spill: 599, 603 ), the results show that the "Cannot seek" error is gone and btw mv check is passed.

But! when I run chaos test with 3CNs (601) , even the error is gone, the source table is still unsynced with PG, I have no idea of this one right now.

Is it possible that when the actor/executor is dropped during recovery, the offset is force flushed regardless of what the offset flush interval is?

lmatz commented 6 months ago

https://github.com/risingwavelabs/kube-bench/pull/408 wal_keep_size now allows customization

StrikeW commented 6 months ago

But! when I run chaos test with 3CNs (601) , even the error is gone, the source table is still unsynced with PG, I have no idea of this one right now.

I reproduced the problem in job 646. And the new_order in RW has more rows than PG.

image

After confirmed with @cyliu0, the workload has DELETE operations on upstream PG. So the problem is why upstream DELETE events doesn't ingested into RW.

The conclusion should be the dedicated source is not tolerant recovery (by design) during the progress of initial snapshot loading. And the cluster is crashed during snapshot loading, then after recovery the cdc source will initiate a new snapshot again and consume from a new lsn offset so that the rows in RW cannot be deleted.

image

We can try to use PG share source which has recoverable backfill to confirm the above findings. https://risingwave-labs.slack.com/archives/C064SBT0ASF/p1709632464607879

Since dedicated cdc source doesn't support recoverable initial snapshot loading, we should ensure the historical data is empty in chaos-mesh test for it . cc @lmatz @cyliu0

hzxa21 commented 6 months ago

But! when I run chaos test with 3CNs (601) , even the error is gone, the source table is still unsynced with PG, I have no idea of this one right now.

I reproduced the problem in job 646. And the new_order in RW has more rows than PG. image After confirmed with @cyliu0, the workload has DELETE operations on upstream PG. So the problem is why upstream DELETE events doesn't ingested into RW.

The conclusion should be the dedicated source is not tolerant recovery (by design) during the progress of initial snapshot loading. And the cluster is crashed during snapshot loading, then after recovery the cdc source will initiate a new snapshot again and consume from a new lsn offset so that the rows in RW cannot be deleted. image

We can try to use PG share source which has recoverable backfill to confirm the above findings. https://risingwave-labs.slack.com/archives/C064SBT0ASF/p1709632464607879

Since dedicated cdc source doesn't support recoverable initial snapshot loading, we should ensure the historical data is empty in chaos-mesh test for it . cc @lmatz @cyliu0

Let me see whether I understand it correct. What happened was:

  1. INSERT row1 (wal offset = 0) into PG table.
  2. PG -> RW CDC table created.
  3. RW CDC table perform snapshot read and then consuming wal offset = 0. RW table = [row1]
  4. RW recovery starts.
  5. DELETE row1 (wal offset = 1) from PG.
  6. INSERT row2 (wal offset = 2) into PG,
  7. RW recovery finishes.
  8. RW CDC table perform snapshot read (read row2) and then consuming wal offset = 2.
  9. RW table = [row1, row2]. row1 delete is missing
hzxa21 commented 6 months ago

Will the CREATE TABLE statement be blocked until the initial snapshot loading finishes for dedicated source CDC? I think we can simply resolve this issue by dropping/clearing the table if it fails before initial snapshot loading and starts clean after recovery.

StrikeW commented 6 months ago

Will the CREATE TABLE statement be blocked until the initial snapshot loading finishes for dedicated source CDC?

No. it won't block. For chaos test purpose, I recommend ensuring historical data is empty when create RW cdc tables for dedicated cdc sources.

StrikeW commented 5 months ago

Since dedicated cdc source doesn't support recoverable initial snapshot loading, we should ensure the historical data is empty in chaos-mesh test for it . cc @lmatz @cyliu0

As I mentioned, it is not a bug and this issue can be closed.