-
Notifications
You must be signed in to change notification settings - Fork 742
Description
Product: RisingWave
Version: 2.8.0-rc.1 (commit 437856795c6c2b820e488c62c0ce80f793e991c5)
Component: Connector Node / Shared CDC Source
Severity: High
Date: 2026-02-24
Title
postgres-cdc shared source silently stops streaming after Debezium engine crash — actor shows RUNNING, no error surfaced to catalog views or event logs
Summary
When the Debezium engine crashes inside the RisingWave connector node during the streaming phase (specifically during PostgresStreamingChangeEventSource.initSchema()), the failure is not propagated to any RisingWave catalog view or event log. The source actor continues to report RUNNING, the database reports recovery_state = RUNNING / SUCCESS, and rw_event_logs contains no error entries. The result is silent, permanent data loss — streaming stops and the system presents as fully healthy.
Background
We operate a multi-tenant RisingWave deployment where each tenant has a shared postgres-cdc source. One tenant's source consistently failed to enter streaming mode after completing its initial snapshot. Investigation revealed the following:
- The Debezium engine crashed during
initSchema()(schema introspection viaPgDatabaseMetaData.getColumns()) due to an infrastructure-level TCP timeout — the AWS NLB drops connections idle for >350 seconds, and the introspection query for a large publication covering many schemas exceeded this threshold. - The crash was visible only in raw connector node JVM logs (
kubectl logs), not in any RisingWave catalog view. - Debezium entered its retry loop (
WARN: Retry 1 of unlimited retries will be attempted) and silently looped, causing the engine to never stabilise. - This condition persisted across 4+ compute node restarts without any error being surfaced.
The infrastructure issue has been worked around by adding debezium.table.include.list to scope schema introspection. However, the RisingWave-level silent failure is a separate, independent bug that would affect any user whose Debezium engine crashes post-snapshot for any reason.
Steps to Reproduce
- Create a shared
postgres-cdcsource against a Postgres database wherePgDatabaseMetaData.getColumns()takes longer than 350 seconds (achievable with a large publication covering many schemas, or by simulating viastatement_timeouton the upstream database). - Let the snapshot complete successfully — downstream materialized views will populate.
- Observe that the Debezium engine crashes in the connector node logs with:
io.debezium.DebeziumException: Error while executing initial schema load at PostgresStreamingChangeEventSource.initSchema(...) Caused by: PSQLException: An I/O error occurred while sending to the backend. Caused by: java.net.SocketException: Socket closed - Query the RisingWave catalog.
Observed Behavior
All catalog views report the source as healthy despite the streaming engine being dead:
-- Actor shows RUNNING
SELECT actor_id, state FROM rw_actors WHERE fragment_id = <fragment_id>;
-- actor_id | state
-- ---------+-------
-- <id> | RUNNING
-- Streaming job shows CREATED (normal operational state)
SELECT id, name, status FROM rw_streaming_jobs WHERE id = <source_id>;
-- id | name | status
-- ---------+-------------+---------
-- <id> | <source> | CREATED
-- Database reports healthy
SELECT database_name, recovery_state, last_database_event
FROM rw_recovery_info
WHERE database_name = '<tenant_db>';
-- database_name | recovery_state | last_database_event
-- ---------------+----------------+--------------------
-- <tenant_db> | RUNNING | SUCCESS
-- No CDC progress tracked
SELECT * FROM rw_cdc_progress WHERE job_id = <source_id>;
-- (0 rows)
-- No error events
SELECT * FROM rw_event_logs ORDER BY timestamp DESC LIMIT 10;
-- (no entries related to the source failure)The only indicator visible from RisingWave's own observability surface is data staleness — querying a downstream MV and observing that max(modified_date) is frozen at the snapshot completion timestamp.
The actor executor emits exactly one log line and then goes permanently silent:
INFO actor{actor_id=<id>}:executor{Source <hex_id>}:
risingwave_stream::executor::source::source_executor:
Reported CDC source offset updated to meta (first time only)
actor_id=<id> source_id=<id>
No subsequent log output from the actor or connector node for this source, across 4 compute restarts spanning 17+ hours.
Expected Behavior
When the Debezium engine fails to start or crashes during the streaming phase, RisingWave should:
- Surface the error to
rw_event_logswith the exception message and source ID. - Mark the actor or source as degraded/failed —
rw_actors.stateshould transition fromRUNNINGtoFAILEDor similar, orrw_streaming_jobs.statusshould reflect the failure. - Either trigger automatic recovery with a bounded retry policy, or surface the failure so the operator can take action.
- At minimum:
cdc.source.wait.streaming.start.timeout(currently30sdefault) should cause a logged error and a clear catalog state change if the engine hasn't confirmed startup within that window.
Impact
- Silent data loss — streaming stops permanently with no alert.
- Undetectable without external data freshness monitoring — all built-in health indicators report green.
- Persists across restarts — the condition reproduces identically after every compute node restart.
- Affects any deployment where the Debezium engine can fail post-snapshot (network instability, upstream DB issues, schema introspection timeouts, etc.).
Environment
| RisingWave version | 2.8.0-rc.1 (commit 437856795c6c2b820e488c62c0ce80f793e991c5) |
| Deployment | Kubernetes (EKS), StatefulSet, namespace risingwave |
| Connector | postgres-cdc (shared source, is_shared = true) |
| Source parallelism | FIXED(1) |
| Upstream DB | Amazon Aurora PostgreSQL 15 (via AWS NLB TCP proxy) |
ssl.mode |
required |
postgres.is.aws.rds |
true |
Source DDL (Minimal Reproduction)
CREATE SOURCE <source_name> WITH (
connector = 'postgres-cdc',
hostname = '<proxy-host>',
port = '<non-standard-port>',
username = '<user>',
password = '<password>',
database.name = '<db>',
schema.name = '<schema>',
auto.schema.change = 'true',
ssl.mode = 'required',
slot.name = '<slot>',
publication.name = '<publication>',
publication.create.enable = 'false',
"postgres.is.aws.rds" = 'true',
"debezium.database.tcpKeepAlive" = 'true'
-- NOTE: without debezium.table.include.list, getColumns() scans all schemas
-- and may exceed infrastructure-level timeouts
);Questions for the RisingWave Team
-
After
"Reported CDC source offset updated to meta (first time only)", what is the expected call sequence from the actor executor to the connector node gRPC service to start the Debezium engine? At which point can a failure occur silently? -
When the connector node returns an error (or the Debezium engine enters its internal retry loop without notifying RisingWave), does the actor executor receive any signal? Is there a code path where
StreamSourceCoreor the gRPC stream simply stops producing records and the actor treats this as "no new data" rather than "engine failed"? -
Should
cdc.source.wait.streaming.start.timeout = 30cause an observable failure (logged error, actor state change) if the engine does not confirm startup within that window? Currently it appears to have no effect. -
Is
rw_cdc_progressexpected to have 0 rows for a shared CDC source after the backfill completes? Or does 0 rows indicate the source is not being tracked? -
Is there a planned mechanism for surfacing connector node JVM-level exceptions (Debezium errors) to
rw_event_logsor a similar RisingWave catalog view?