-
Bug
-
Resolution: Done
-
Critical
-
3.2.0.Final
-
False
-
-
False
-
Critical
In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.
Bug report
For bug reports, provide this information, please:
What Debezium connector do you use and what version?
Debezium version is 3.2
I am using connector Postgres and Event Hubs sink.
What is the connector configuration?
Here is the debezium server's entire configuration
debezium.source.connector.class=io.debezium.connector.postgresql.PostgresConnector debezium.source.offset.storage=io.debezium.storage.jdbc.offset.JdbcOffsetBackingStore # debezium.source.offset.storage.jdbc.connection.url='<FROM_ENV>' debezium.source.offset.storage.jdbc.table.name=debezium_offset_storage_test debezium.source.schema.history.internal=io.debezium.storage.jdbc.history.JdbcSchemaHistory # debezium.source.schema.history.internal.jdbc.connection.url='<FROM_ENV>' debezium.source.schema.history.internal.jdbc.table.name=debezium_schema_history_test quarkus.http.port=8080 # quarkus.log.level=<FROM_ENV> quarkus.log.console.json=true debezium.sink.type=eventhubs # debezium.sink.eventhubs.connectionstring=<FROM_ENV> debezium.sink.eventhubs.hubname=evh-debezium-cdc-test debezium.source.key.converter.schemas.enable=false debezium.source.value.converter.schemas.enable=false # Connector configuration # debezium.source.database.hostname=<FROM_ENV> # debezium.source.database.port=<FROM_ENV> # debezium.source.database.user=<FROM_ENV> # debezium.source.database.password=<FROM_ENV> # debezium.source.database.dbname=<FROM_ENV> debezium.source.database.sslmode=require debezium.source.plugin.name=pgoutput debezium.source.snapshot.mode=never debezium.source.publication.autocreate.mode=disabled debezium.source.table.include.list=public.* debezium.source.slot.name=test debezium.source.publication.name=test debezium.source.topic.prefix=postgres-test
What is the captured database version and mode of deployment?
(E.g. on-premises, with a specific cloud provider, etc.)
Postgres 16, running locally with docker-compose. We have also found this issue on Azure Container Apps.
What behavior do you expect?
I expected debezium to publish every event once while rarely duplicating events (at least once delivery)
What behavior do you see?
Debezium publishes the current event + all the other before before it, since the connector started. If I restart the connector, then it clears the old events, but as soon as we start emitting new changes the connector starts duplicating more and more. We strongly believe that this is a user error but since the sink has only 3 options we could not find anything wrong. It works with kafka sink and if we specify partitionKey or partitionId options in the azure event hubs sink so we are pretty much sure the event hubs sink (or its configuration) is the issue here.
Do you see the same behaviour using the latest released Debezium version?
(Ideally, also verify with latest Alpha/Beta/CR version)
Yes, we are already using the latest version.
Do you have the connector logs, ideally from start till finish?
(You might be asked later to provide DEBUG/TRACE level log)
As we see no errors on debezium, we started the server with warn logs, but enabled debug log for the debezium and azure event hubs package. This is the relevant log when a change event happens. Note that only a single record has changed in this transaction (the one with the highest id), but it sent every change since the connector started again.
debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.163604373Z", "sequence": 770, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Event Hubs sink adapter processing change events", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.331435002Z", "sequence": 771, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Emitting events starting from index 0", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.332101541Z", "sequence": 772, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_3cb9df_1753904890819\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"74bc5ac2-67b6-42b4-b835-55015cef653d\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.332777598Z", "sequence": 773, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Dispatching 1 events for partition key '{\"id\":7}'.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.332970133Z", "sequence": 774, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sending batch of 1 events to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.333187811Z", "sequence": 775, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.EventHubProducerAsyncClient", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Sending batch.\",\"size\":1,\"partitionKey\":\"{\\\"id\\\":7}\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.334057138Z", "sequence": 776, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_3cb9df_1753904890819\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"74bc5ac2-67b6-42b4-b835-55015cef653d\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.514890325Z", "sequence": 777, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sent record batch to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.515287124Z", "sequence": 778, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Dispatching 1 events for partition key '{\"id\":10}'.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.515406495Z", "sequence": 779, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sending batch of 1 events to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.515614685Z", "sequence": 780, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.EventHubProducerAsyncClient", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Sending batch.\",\"size\":1,\"partitionKey\":\"{\\\"id\\\":10}\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.516819956Z", "sequence": 781, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_3cb9df_1753904890819\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"74bc5ac2-67b6-42b4-b835-55015cef653d\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.699242909Z", "sequence": 782, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sent record batch to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.699675119Z", "sequence": 783, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Dispatching 1 events for partition key '{\"id\":9}'.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.699885738Z", "sequence": 784, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sending batch of 1 events to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.700085608Z", "sequence": 785, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.EventHubProducerAsyncClient", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Sending batch.\",\"size\":1,\"partitionKey\":\"{\\\"id\\\":9}\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.701201462Z", "sequence": 786, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_3cb9df_1753904890819\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"74bc5ac2-67b6-42b4-b835-55015cef653d\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.872494443Z", "sequence": 787, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sent record batch to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.873031103Z", "sequence": 788, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Dispatching 1 events for partition key '{\"id\":8}'.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.873259343Z", "sequence": 789, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sending batch of 1 events to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.873492029Z", "sequence": 790, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.EventHubProducerAsyncClient", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Sending batch.\",\"size\":1,\"partitionKey\":\"{\\\"id\\\":8}\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:47.87479574Z", "sequence": 791, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_3cb9df_1753904890819\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"74bc5ac2-67b6-42b4-b835-55015cef653d\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:48.04740911Z", "sequence": 792, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sent record batch to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:48.048039236Z", "sequence": 793, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Marking 1 records as processed.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:49:48.548036267Z", "sequence": 794, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Batch marked finished", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 }
Note that if claims to have finished processing one record, but actually sends many. We have done some investigation on this and narrowed the issue to this forEach loop having dynamic partition key batches, but have no idea why that is the case.
dynamicPartitionKeyBatches.forEach((partitionKey, batch) -> { if (batch.getCount() > 0) { LOGGER.trace("Dispatching {} events for partition key '{}'.", batch.getCount(), partitionKey); emitBatchToEventHub(batch); } });
The issue does not happen when using a fixed partition id or key. These are the logs of a change of a single record using a fixed partition key. (this one works and sends only the updated record)
debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.335746062Z", "sequence": 744, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Event Hubs sink adapter processing change events", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.336185466Z", "sequence": 745, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Emitting events starting from index 0", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.336806111Z", "sequence": 746, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_d7b9cb_1753905331795\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"2c118e9c-3074-4788-a0fc-049514079e93\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.337328443Z", "sequence": 747, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Dispatching 1 events.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.337426753Z", "sequence": 748, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sending batch of 1 events to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.337538363Z", "sequence": 749, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.EventHubProducerAsyncClient", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Sending batch.\",\"size\":1,\"partitionKey\":\"potato\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.33794537Z", "sequence": 750, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "com.azure.messaging.eventhubs.implementation.EventHubReactorAmqpConnection", "level": "DEBUG", "message": "{\"az.sdk.message\":\"Get or create producer.\",\"connectionId\":\"MF_d7b9cb_1753905331795\",\"entityPath\":\"evh-debezium-cdc-test\",\"clientIdentifier\":\"2c118e9c-3074-4788-a0fc-049514079e93\"}", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.509213072Z", "sequence": 751, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.BatchManager", "level": "TRACE", "message": "Sent record batch to Event Hubs", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:44.509548448Z", "sequence": 752, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Marking 1 records as processed.", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 } debezium-server-1 | { "timestamp": "2025-07-30T19:57:45.008078586Z", "sequence": 753, "loggerClassName": "org.slf4j.impl.Slf4jLogger", "loggerName": "io.debezium.server.eventhubs.EventHubsChangeConsumer", "level": "TRACE", "message": "Batch marked finished", "threadName": "pool-22-thread-1", "threadId": 101, "mdc": {}, "ndc": "", "hostName": "debezium-server", "processName": "/usr/lib/jvm/java-21-openjdk-21.0.8.0.9-1.el8.x86_64/bin/java", "processId": 1 }
How to reproduce the issue using our tutorial deployment?
The tutorial deployment is using kafka connect, we are using debezium server. We are running the quay.io image with a couple changes only to add applicationinsights agent.
FROM quay.io/debezium/server:3.2 USER root RUN microdnf -y install gettext USER jboss ENV APPLICATION_INSIGHTS_AGENT_VERSION="3.7.1" RUN mkdir /debezium/config/applicationinsights ADD --chown=jboss:jboss https://github.com/microsoft/ApplicationInsights-Java/releases/download/${APPLICATION_INSIGHTS_AGENT_VERSION}/applicationinsights-agent-${APPLICATION_INSIGHTS_AGENT_VERSION}.jar /debezium/config/applicationinsights/applicationinsights-agent.jar COPY ./configs/applicationinsights.json /debezium/config/applicationinsights/applicationinsights.json ENV JAVA_TOOL_OPTIONS="-javaagent:/debezium/config/applicationinsights/applicationinsights-agent.jar" ENV ENABLE_OTEL=true CMD ["/debezium/run.sh"]
Feature request or enhancement
For feature requests or enhancements, provide this information, please:
Which use case/requirement will be addressed by the proposed feature?
<Your answer>
Implementation ideas (optional)
- account is impacted by
-
DBZ-9340 Debezium constatly performs heartbeat.action.query instead of honoring heartbeat.interval.ms
-
- Closed
-