Skip to content

DBZ-2386 Notify errorHandler on Postgres errors#5971

Merged
jpechane merged 6 commits intodebezium:mainfrom
brewneaux:bugfix/DBZ-2386
Jan 15, 2025
Merged

DBZ-2386 Notify errorHandler on Postgres errors#5971
jpechane merged 6 commits intodebezium:mainfrom
brewneaux:bugfix/DBZ-2386

Conversation

@brewneaux
Copy link
Contributor

During certain scenarios, especially during RDS reboots (planned or unplanned), Postgres becomes available in such a way that Debezium is not aware that Postgres went away. In many cases, this can result in connectors thinking they are healthy (with all statuses returning RUNNING), but are not actually reading from the replication slots.

By notifying the errorHandler when these occur, we are able to have the in-built retry mechanism reconnect after these failures.

Due to the inconsistent nature of how this happens - not every reboot will cause a connector to get stuck - we are catching erros happening in both the PostgresStreamingChangeEventSource, and the core ChangeEventSourceCoordinator. In my testing, I have seen only some types of events get caught by the error handler in the ChangeEventSourceCoordinator.

}
}
catch (SQLException e) {
errorHandler.setProducerThrowable(e);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brewneaux Is this necessary here? As the exception is rethrown it should be caught by the change in ChangeEventSourceCoordinator. The errorHandler object is a single one per connector so it will be set there.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also IMHO there should be the same cleanup as is in the execute method so it should be extracted there to a separate method. But at the same time I believe it is possible that the calls are done from separate threads. So maybe there should be a flag field that is set in commitOffset and the execute method will fail when it is set. This might be the safest approach.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In the testing I did on my initial PR, it appeared like it was. I just retested, though, in a few other scenarios with just the ChangeEventSourceCoordinator change and it continues to work as desired.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jpechane - just updated this PR. I did my best to try to understand what you were asking for and make it work within this context.

@jpechane
Copy link
Contributor

The falures are completely unrelated and I cannot reporudce them locally. I pulled the latest TimescaleDb image and it still works. Let's wait couple of hours to see if it is a transient issue with GHA.

@brewneaux
Copy link
Contributor Author

The falures are completely unrelated and I cannot reporudce them locally. I pulled the latest TimescaleDb image and it still works. Let's wait couple of hours to see if it is a transient issue with GHA.

Hey @jpechane - anything we can do to move this forward? My team is currently using a homebuilt version of Debezium so we can handle this, but we'd like to be on a more official build if we could

@jpechane
Copy link
Contributor

@brewneaux Hi, so I re-run this locally on the latest main. The TimescaleDbDatabaseTest is passing on the main but failing when the PR is updated so there is clear causality. You need to to have plugin.name=pgoutput sysproperty set to see it yourself.

@brewneaux
Copy link
Contributor Author

@jpechane this should be working now, but I'm having a ton of problems getting the full test suite to run. I can run little sections of it - and the Timescale tests all passed. Can you run the suite via GHA?

@brewneaux
Copy link
Contributor Author

brewneaux commented Dec 17, 2024

Hm, that result from Github Actions is strange. All of the failures I saw on there, I was able to run successfully locally. @jpechane - do you have any other suggestions? I'm still not able to get a full run locally the same way GHA would (I'm blaming Windows and my lack of Java experience), but when I run the failures individually I do get success.

@Naros
Copy link
Member

Naros commented Dec 18, 2024

Hi @brewneaux the REST test failures are related to a fix coming in PR #6059, you can safely ignore those errors for now.

But to add some explanation, those tests are executed when the -Passembly profile is set when running the build and tests, and you likely were not setting that profile which explains why they were not executed locally.

Looking in .github/workflows/debezium-workflow-pr.yml, you can see what profiles are executed here:

  build_postgresql:
    strategy:
      # Runs each combination concurrently
      matrix:
        profile: [ "assembly,postgres-12", "assembly,postgres-17,pgoutput-decoder" ]

So to execute locally in the same way with PG 12, you would set -Passembly,postgres-12 to have the same net experience as what is executed on GitHub actions.

@jpechane
Copy link
Contributor

@brewneaux The referred PR was merged so the tests should run correctly now.

@brewneaux
Copy link
Contributor Author

brewneaux commented Dec 18, 2024

Thanks everyone. I rebased this against debezium/debezium main. I'm still having some issues getting the full test suite to run.

I'm (now) on an Ubuntu machine (after giving up on my windows laptop doing stupid things with the MongoDbReplicaSetContainerIT), running

./mvnw clean install -B -pl debezium-connector-postgres -am \
        -Passembly,postgres-17,pgoutput-decoder \
        -Ddebezium.test.records.waittime=5 \
        -Dcheckstyle.skip=true \
        -Dformat.skip=true \
        -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn \
        -Dmaven.wagon.http.pool=false \
        -Dmaven.wagon.httpconnectionManager.ttlSeconds=120 \
        -DfailFlakyTests=false \
        -Ddebezium.test.mongo.replica.primary.startup.timeout.seconds=120

I get through a large part of the test suite after quite a while. But I seem to get different results (failures) each time I run it. A lot of them look like they might be just flakey tests but I can't tell.

One of the big issues I'm running into though is that eventually, get to an error where it cannot build the docker image for the rest tests:
https://gist.github.com/brewneaux/1f7355ddec95a796224d40596105a08b

It is building the plugin as version 3.1.0-SNAPSHOT, and looking for 3.0.5-SNAPSHOT.

I appreciate everyones help on this. The Java world is so very different from the C# and Python I do on a daily basis.

@brewneaux
Copy link
Contributor Author

@jpechane or @Naros can we re-approve the workflow? I'm still getting some weird stuff locally, but when I run them all individually, I get valid builds

@brewneaux
Copy link
Contributor Author

2024-12-23T20:10:09.7941366Z [INFO] Running io.debezium.connector.postgresql.rest.DebeziumPostgresConnectorResourceNoDatabaseIT
2024-12-23T20:10:11.2406866Z 2024-12-23 20:10:11,155 ERROR  ||||  COPY failed: file not found in build context or excluded by .dockerignore: stat debezium-connector-postgres-3.0.5-SNAPSHOT-plugin.tar.gz: file does not exist   [tc.debezium/connect-rest-test:3.1.0-SNAPSHOT]
2024-12-23T20:10:12.3438199Z 2024-12-23 20:10:12,336 ERROR  ||||  COPY failed: file not found in build context or excluded by .dockerignore: stat debezium-connector-postgres-3.0.5-SNAPSHOT-plugin.tar.gz: file does not exist   [tc.debezium/connect-rest-test:3.1.0-SNAPSHOT]
2024-12-23T20:10:13.9487401Z 2024-12-23 20:10:13,878 ERROR  ||||  COPY failed: file not found in build context or excluded by .dockerignore: stat debezium-connector-postgres-3.0.5-SNAPSHOT-plugin.tar.gz: file does not exist   [tc.debezium/connect-rest-test:3.1.0-SNAPSHOT]
2024-12-23T20:10:15.1526087Z 2024-12-23 20:10:15,067 ERROR  ||||  COPY failed: file not found in build context or excluded by .dockerignore: stat debezium-connector-postgres-3.0.5-SNAPSHOT-plugin.tar.gz: file does not exist   [tc.debezium/connect-rest-test:3.1.0-SNAPSHOT]
2024-12-23T20:10:15.1798301Z [ERROR] Tests run: 2, Failures: 0, Errors: 2, Skipped: 0, Time elapsed: 5.447 s <<< FAILURE! -- in io.debezium.connector.postgresql.rest.DebeziumPostgresConnectorResourceNoDatabaseIT

This was one of the errors I was describing above, that I attributed to a local misconfiguration. Does anyone have any reason why this would be happening?

@brewneaux
Copy link
Contributor Author

@jpechane @Naros - can I ask for help getting this passed and merged? Our team is still using a homebuilt image in production and would really like to get onto an official release when we can.

@brewneaux
Copy link
Contributor Author

@jpechane @Naros - we've now gotten successful builds for all of the postgres targets. I'm not sure what isn't working on GHA.

brewneaux and others added 6 commits January 14, 2025 13:43
During certain scenarios, especially during RDS reboots (planned or
unplanned), Postgres becomes available in such a way that Debezium is
not aware that Postgres went away.  In many cases, this can result in
connectors thinking they are healthy (with all statuses returning
RUNNING), but are not actually reading from the replication slots.

By notifying the errorHandler when these occur, we are able to have the
in-built retry mechanism reconnect after these failures.

Due to the inconsistent nature of how this happens - not every reboot
will cause a connector to get stuck - we are catching erros happening in
both the PostgresStreamingChangeEventSource, and the
core ChangeEventSourceCoordinator.  In my testing, I have seen only some
types of events get caught by the error handler in the
ChangeEventSourceCoordinator.
In further testing, doesn't appear that the postgres-specific one was
necessary
@jpechane
Copy link
Contributor

Reabsed on the latest main to see fresh CI results

@jpechane jpechane merged commit 35ed1c1 into debezium:main Jan 15, 2025
@jpechane
Copy link
Contributor

@brewneaux Applied, thanks!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants