Details
-
Bug
-
Resolution: Unresolved
-
Major
-
2.0.0.Final
-
None
-
True
-
OOM error due to multiple result sets open on the same connection which causes SQL Server JDBC to load the full result set.
-
Important
Description
We have a Java application that is using the Debezium SQL Server connector 2.0.0.Final (Debezium Embedded Engine) to publish CDC to a AWS Kinesis stream. The java application is executed with a max heap of 1GB (-Xmx1000m). We are encountering error: java.lang.OutOfMemoryError: Java heap space (OOM) with the following engine properties which we will refer to as our "BASE CONFIGS".
connector.class = io.debezium.connector.sqlserver.SqlServerConnector database.dbname = ADDRESSBOOK database.hostname = ******** database.names = ADDRESSBOOK database.password = ******** database.port = 50001 database.user = user datatype.propagate.source.type = .* decimal.handling.mode = double errors.max.retries = -1 errors.retry.delay.initial.ms = 300 errors.retry.delay.max.ms = 10000 heartbeat.interval.ms = 0 include.schema.changes = false key.converter = org.apache.kafka.connect.json.JsonConverter max.batch.size = 1000 max.queue.size = 20000 name = core_addressbook offset.flush.interval.ms = 120000 offset.flush.timeout.ms = 30000 offset.storage = org.apache.kafka.connect.storage.MemoryOffsetBackingStore poll.interval.ms = 1000 retriable.restart.connector.wait.ms = 120000 schema.history.internal = io.debezium.relational.history.MemorySchemaHistory schema.history.internal.store.only.captured.tables.ddl = true schema.include.list = DBO snapshot.fetch.size = 1000 snapshot.isolation.mode = read_uncommitted snapshot.lock.timeout.ms = 10000 snapshot.mode = schema_only table.include.list = DBO.CONTACT,DBO.CONTACT_CUSTOM_FIELD,DBO.CONTACT_CUSTOM_FIELD_DETAIL task.id = 0 tasks.max = 1 time.precision.mode = adaptive tombstones.on.delete = false topic.prefix = core_ADDRESSBOOK value.converter = org.apache.kafka.connect.json.JsonConverter
The BASE CONFIGS (above) is set up to listen for record changes on 3 SQL Server tables.
Here is an image of the OOM error:
We modified some of the BASE CONFIGS which I will note below:
1) BASE CONFIGS + query.fetch.size=1, snapshot.fetch.size=1, max.batch.size=1, max.queue.size=2
RESULT: OOM Error
2) BASE CONFIGS + remove schema.include.list + remove table.include.list (list to all tables from database)
RESULT: OOM Error
3) BASE CONFIGS + snapshot.mode=initial
RESULT: OOM Error
4) BASE CONFIGS + table.include.list=DBO.CONTACT (listen to only 1 table instead of 3)
RESULT: SUCCESS, record count: 1,898,352
5) BASE CONFIGS + table.include.list=DBO.CONTACT_CUSTOM_FIELD (listen to only 1 table instead of 3)
RESULT: SUCCESS, record count: 6,457,582
6) BASE CONFIGS + table.include.list=CONTACT_CUSTOM_FIELD_DETAIL (listen to only 1 table instead of 3)
RESULT: SUCCESS, record count: 5,772
With test 4, 5, and 6 (above) working, we believe the issue may be due to the following from Microsoft documentation:
Avoid executing more than one statement on the same connection simultaneously. Executing another statement before processing the results of the previous statement may cause the unprocessed results to be buffered into the application memory. (source: https://learn.microsoft.com/en-us/sql/connect/jdbc/using-adaptive-buffering)
Based on this code (https://github.com/debezium/debezium/blob/v2.0.0.Final/debezium-core/src/main/java/io/debezium/jdbc/JdbcConnection.java#L587), we see that multiple ResultSets are opened on the same Connection object and stored in an array before the ResultSets are processed. This means that the SQL Server JDBC driver will try to load the entire ResultSet into memory with multiple ResultSets opened.
Heap Dump
Thread Stack:
debezium-sqlserverconnector-core_ADDRESSBOOK-change-event-source-coordinator at java.lang.OutOfMemoryError.<init>()V (OutOfMemoryError.java:48) at com.microsoft.sqlserver.jdbc.TDSPacket.<init>(I)V (IOBuffer.java:6241) at com.microsoft.sqlserver.jdbc.TDSReader.readPacket()Z (IOBuffer.java:6433) at com.microsoft.sqlserver.jdbc.TDSCommand.detach()V (IOBuffer.java:7280) at com.microsoft.sqlserver.jdbc.SQLServerConnection.executeCommand(Lcom/microsoft/sqlserver/jdbc/TDSCommand;)Z (SQLServerConnection.java:3032) at com.microsoft.sqlserver.jdbc.SQLServerStatement.executeCommand(Lcom/microsoft/sqlserver/jdbc/TDSCommand;)V (SQLServerStatement.java:247) at com.microsoft.sqlserver.jdbc.SQLServerStatement.executeStatement(Lcom/microsoft/sqlserver/jdbc/TDSCommand;)V (SQLServerStatement.java:222) at com.microsoft.sqlserver.jdbc.SQLServerPreparedStatement.executeQuery()Ljava/sql/ResultSet; (SQLServerPreparedStatement.java:444) at io.debezium.jdbc.JdbcConnection.prepareQuery([Ljava/lang/String;[Lio/debezium/jdbc/JdbcConnection$StatementPreparer;Lio/debezium/jdbc/JdbcConnection$BlockingMultiResultSetConsumer;)Lio/debezium/jdbc/JdbcConnection; (JdbcConnection.java:587) at io.debezium.connector.sqlserver.SqlServerConnection.getChangesForTables(Ljava/lang/String;[Lio/debezium/connector/sqlserver/SqlServerChangeTable;Lio/debezium/connector/sqlserver/Lsn;Lio/debezium/connector/sqlserver/Lsn;Lio/debezium/jdbc/JdbcConnection$BlockingMultiResultSetConsumer;)V (SqlServerConnection.java:335) at io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource.executeIteration(Lio/debezium/pipeline/source/spi/ChangeEventSource$ChangeEventSourceContext;Lio/debezium/connector/sqlserver/SqlServerPartition;Lio/debezium/connector/sqlserver/SqlServerOffsetContext;)Z (SqlServerStreamingChangeEventSource.java:211) at io.debezium.connector.sqlserver.SqlServerStreamingChangeEventSource.executeIteration(Lio/debezium/pipeline/source/spi/ChangeEventSource$ChangeEventSourceContext;Lio/debezium/pipeline/spi/Partition;Lio/debezium/pipeline/spi/OffsetContext;)Z (SqlServerStreamingChangeEventSource.java:60) at io.debezium.connector.sqlserver.SqlServerChangeEventSourceCoordinator.executeChangeEventSources(Lio/debezium/connector/common/CdcSourceTaskContext;Lio/debezium/pipeline/source/spi/SnapshotChangeEventSource;Lio/debezium/pipeline/spi/Offsets;Ljava/util/concurrent/atomic/AtomicReference;Lio/debezium/pipeline/source/spi/ChangeEventSource$ChangeEventSourceContext;)V (SqlServerChangeEventSourceCoordinator.java:98) at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(Ljava/util/concurrent/atomic/AtomicReference;Lio/debezium/connector/common/CdcSourceTaskContext;)V (ChangeEventSourceCoordinator.java:109) at io.debezium.pipeline.ChangeEventSourceCoordinator$$Lambda$977+0x000000080127f828.run()V (Unknown Source) at java.util.concurrent.Executors$RunnableAdapter.call()Ljava/lang/Object; (Executors.java:539) at java.util.concurrent.FutureTask.run()V (FutureTask.java:264) at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:635) at java.lang.Thread.run()V (Thread.java:833)
Can this be fixed?