You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
[DBZ-PGYB] Bug fix for inability to alter publication upon transitioning to streaming (#174)
## Problem
When #172 got merged, we added the logic to begin a read only
transaction so that we can use `USE_SNAPSHOT` and capture the snapshot
from the same. However, upon transition to streaming, when
`PostgresReplicationConnection#startStreaming` during streaming phase,
it internally calls `PostgresReplicationConnection#initConnection` and
then ``PostgresReplicationConnection#initPublication` which tries to
execute an `ALTER` command on the publication.
The `ALTER` command fails owing to the fact that we're still in a
read-only transaction opened at the time of slot creation. The exception
is similar to:
```
Caused by: org.apache.kafka.connect.errors.ConnectException: Unable to update filtered publication dbz_publication for "public"."test"
at io.debezium.connector.postgresql.connection.PostgresReplicationConnection.createOrUpdatePublicationModeFilterted(PostgresReplicationConnection.java:232)
at io.debezium.connector.postgresql.connection.PostgresReplicationConnection.initPublication(PostgresReplicationConnection.java:196)
at io.debezium.connector.postgresql.connection.PostgresReplicationConnection.initConnection(PostgresReplicationConnection.java:496)
at io.debezium.connector.postgresql.connection.PostgresReplicationConnection.startStreaming(PostgresReplicationConnection.java:396)
at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:182)
... 9 more
Caused by: com.yugabyte.util.PSQLException: ERROR: cannot execute ALTER PUBLICATION in a read-only transaction
at com.yugabyte.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2675)
at com.yugabyte.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:2365)
at com.yugabyte.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:355)
at com.yugabyte.jdbc.PgStatement.executeInternal(PgStatement.java:490)
at com.yugabyte.jdbc.PgStatement.execute(PgStatement.java:408)
at com.yugabyte.jdbc.PgStatement.executeWithFlags(PgStatement.java:329)
at com.yugabyte.jdbc.PgStatement.executeCachedSql(PgStatement.java:315)
at com.yugabyte.jdbc.PgStatement.executeWithFlags(PgStatement.java:291)
at com.yugabyte.jdbc.PgStatement.execute(PgStatement.java:286)
at io.debezium.connector.postgresql.connection.PostgresReplicationConnection.createOrUpdatePublicationModeFilterted(PostgresReplicationConnection.java:229)
```
## Solution
As a fix, we will now commit at the time of snapshot completion and
rollback at the time of snapshot abortion so that we ensure that we are
not going into the streaming phase with any open transaction. Another
added guardrail in this PR is that we only open the read-only
transaction when `streaming.mode` is set to `parallel`.
Copy file name to clipboardexpand all lines: debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSnapshotChangeEventSource.java
Copy file name to clipboardexpand all lines: debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresReplicationConnection.java
+8-2
Original file line number
Diff line number
Diff line change
@@ -533,8 +533,14 @@ public Optional<SlotCreationResult> createReplicationSlot() throws SQLException
0 commit comments