Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-9127

Debezium Server crashes throwing ArrayIndexOutOfBoundsException

XMLWordPrintable

    • False
    • Hide

      None

      Show
      None
    • False
    • Important

      Bug report

      What Debezium connector do you use and what version?

      debezium-server-dist-3.1.1.Final-runner.jar

      What is the connector configuration?

      # Sink
      debezium.sink.type=kinesis
      debezium.sink.kinesis.region=us-east-1
      debezium.sink.kinesis.endpoint=https://kinesis.us-east-1.amazonaws.com
      # DSE Cassandra Connector
      debezium.source.offset.storage.file.filename=/debezium-server/offset
      debezium.source.connector.class=io.debezium.connector.dse.Dse6816Connector
      debezium.source.cassandra.node.id=<ip_address>
      debezium.source.cassandra.hosts=<ip_address>
      debezium.source.cassandra.port=9042
      debezium.source.cassandra.config=/etc/dse/cassandra/cassandra.yaml
      debezium.source.cassandra.driver.config.file=/debezium-server/config/lib/cassandra/driver.conf
      debezium.source.commit.log.relocation.dir=/debezium-server/relocation
      debezium.source.commit.log.real.time.processing.enabled=True
      debezium.source.commit.log.marked.complete.poll.interval.ms=2000
      debezium.source.offset.storage=org.apache.kafka.connect.storage.FileOffsetBackingStore
      debezium.source.topic.prefix=customer_cdc
      debezium.source.snapshot.mode=NEVER
      debezium.source.snapshot.consistency=LOCAL_ONE
      debezium.source.http.port=8085
      debezium.source.skipped.operations=none
      debezium.source.key.converter.schemas.enable=False
      debezium.source.value.converter.schemas.enable=False
      
      # Transforms
      debezium.transforms.EnvelopeTransformation.type=io.debezium.connector.cassandra.transforms.EnvelopeTransformation
      debezium.transforms=EnvelopeTransformation
      # Quarkus
      quarkus.log.level=INFO
      quarkus.log.console.json=False
      quarkus.http.port=8080
      quarkus.log.file.enable=True 

       

      What is the captured database version and mode of deployment?

      In AWS EC2 servers running with DataStax Enterprise 6.9.6.

      3 Data Center cluster with 6 nodes per Data Center.

      CDC is enabled in a single DC.

      What behavior do you expect?

      Debezium server should be able to send CDC events to Kinesis without issues.

      What behavior do you see?

      Debezium server sporadically fails to process the commitlog entry throwing an ArrayIndexOutOfBoundsException. When this happens, the commitlog and commitlog_cdc.idx files are relocated and deleted from the cdc_raw directory, but not from the commitlog directory. 

      This bug creates an inconsistency in the state of the commitlog and CDC events no longer continue to get sent to Kinesis consistently until the DSE Cassandra server is restarted.

      Do you see the same behaviour using the latest released Debezium version?

      This bug has been observed in 3.0.8.Final and continued in 3.1.1.Final.

      Do you have the connector logs, ideally from start till finish?

      Three 3 examples of the error thrown by the debezium connector when this bug happens:

       

      2025-05-09 20:52:57,707 ERROR [io.deb.con.cas.CommitLogIdxParser] (pool-15-thread-1) Processing of LogicalCommitLog{synced=2690594, completed=false, log=/dse_logs/cdc_raw/CommitLog-680-1746819656929.log, index=/dse_logs/cdc_raw/CommitLog-680-1746819656929_cdc.idx, commitLogSegmentId=1746819656929} errored out: io.debezium.DebeziumException: Error occurred while processing commit log /dse_logs/cdc_raw/CommitLog-680-1746819656929.log    at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:144)    at io.debezium.connector.cassandra.CommitLogIdxParser.parse(CommitLogIdxParser.java:84)    at io.debezium.connector.cassandra.CommitLogIdxParser.process(CommitLogIdxParser.java:116)    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)    at java.base/java.lang.Thread.run(Thread.java:1583)Caused by: io.debezium.DebeziumException: Failed to process PartitionUpdate [customer.customers] key=FAKE_KEY partition_deletion=LIVE columns=[[] | [customer_citizenship customer_country_of_origin customer_date_of_birth customer_emails customer_employment_information customer_name customer_phone_numbers customer_phone_numbers_v2 customer_physical_address customer_preferred_name_v2 customer_tax_identification customer_type enabled_model mismatched_data ofac_restriction relationship_since solr_query customer_mailing_addresses customer_phone_numbers_v3 model]]    Row[info=[ts=EMPTY] ]: EMPTY | , [customer_phone_numbers_v3[FAKE_NUMBER]= ts=1747227401955000] at CommitLog-680-1746819656929.log:2689444 for table customer.customers.    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:279)    at org.apache.cassandra.db.commitlog.CommitLogReadHandler.handleMutation(CommitLogReadHandler.java:87)    at org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:493)    at org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:413)    at org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:259)    at io.debezium.connector.dse.DseCommitLogSegmentReader.readCommitLogSegment(DseCommitLogSegmentReader.java:32)    at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:138)    ... 6 moreCaused by: io.debezium.DebeziumException: Failed to populate Column customer_phone_numbers_v3 with Type org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.FrozenType(org.apache.cassandra.db.marshal.UserType(customer,70686f6e655f6e756d6265725f76335f74797065,70686f6e655f6e756d626572:org.apache.cassandra.db.marshal.UTF8Type,6e69636b6e616d65:org.apache.cassandra.db.marshal.UTF8Type,6c65676163795f70686f6e655f74797065:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type),69735f7072696d617279:org.apache.cassandra.db.marshal.BooleanType,69735f776f726b:org.apache.cassandra.db.marshal.BooleanType,69735f6d6f62696c655f636f6e6669726d65645f62795f76656e646f72:org.apache.cassandra.db.marshal.BooleanType,6861735f636f6e73656e745f746f5f6469616c:org.apache.cassandra.db.marshal.BooleanType,636f6e73656e745f746f5f6469616c5f74797065:org.apache.cassandra.db.marshal.UTF8Type,636f6e73656e745f746f5f6469616c5f65737461626c69736865645f736f757263655f73797374656d:org.apache.cassandra.db.marshal.UTF8Type,636f6e73656e745f746f5f6469616c5f6c6173745f6d6f6469666965645f736f757263655f73797374656d:org.apache.cassandra.db.marshal.UTF8Type,636f6e73656e745f746f5f6469616c5f65737461626c69736865645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,636f6e73656e745f746f5f6469616c5f636f6e6669726d6174696f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,636f6e73656e745f746f5f6469616c5f636f6e6669726d6174696f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,636f6e746578745f6964:org.apache.cassandra.db.marshal.UTF8Type,61757468656e7469636174696f6e5f646570656e646162696c6974795f696e64696361746f72:org.apache.cassandra.db.marshal.BooleanType,61757468656e7469636174696f6e5f646570656e646162696c6974795f726561736f6e:org.apache.cassandra.db.marshal.UTF8Type,736d735f737461747573:org.apache.cassandra.db.marshal.UTF8Type,736d735f7374617475735f726561736f6e73:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UserType(customer,70686f6e65XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX065,726561736f6e5f636f6465:org.apache.cassandra.db.marshal.UTF8Type,726561736f6e5f6465736372697074696f6e:org.apache.cassandra.db.marshal.UTF8Type,77686974656c6973745f656d61696c:org.apache.cassandra.db.marshal.UserType(customer,77686974656c6973745f656d61696c5f74797065,77686974656c6973745f656d61696c5f61646472657373:org.apache.cassandra.db.marshal.UTF8Type,77686974656c6973745f656d61696c5f757064617465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,77686974656c6973745f737461747573:org.apache.cassandra.db.marshal.BooleanType))),69735f637469615f746578745f70656e64696e67:org.apache.cassandra.db.marshal.BooleanType,637573746f6d65725f737065636966696564:org.apache.cassandra.db.marshal.BooleanType,6f776e6572736869705f73636f7265:org.apache.cassandra.db.marshal.UTF8Type,6f776e6572736869705f73636f72655f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,70686f6e655f736572766963655f70726f7669646572:org.apache.cassandra.db.marshal.UTF8Type,61735f7265706f727465645f646174615f736f757263655f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,6d69736d6174636865645f64617461:org.apache.cassandra.db.marshal.BooleanType,65737461626c69736865645f736f757263655f73797374656d:org.apache.cassandra.db.marshal.UTF8Type,6c6173745f6d6f6469666965645f736f757263655f73797374656d:org.apache.cassandra.db.marshal.UTF8Type,65737461626c69736865645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,636f6e6669726d6174696f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,6c6173745f757064617465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType))) of Table customers in KeySpace customer.    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:579)    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowModifications(DseCommitLogReadHandlerImpl.java:437)    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowIterator(DseCommitLogReadHandlerImpl.java:335)    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.process(DseCommitLogReadHandlerImpl.java:320)    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:276)    ... 12 moreCaused by: java.lang.ArrayIndexOutOfBoundsException: Index 0 out of bounds for length 0    at java.base/java.util.Arrays$ArrayList.get(Arrays.java:4266)    at io.debezium.connector.cassandra.transforms.type.deserializer.AbstractUserDefinedTypeDeserializer.deserialize(AbstractUserDefinedTypeDeserializer.java:38)    at io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.deserialize(CassandraTypeDeserializer.java:121)    at io.debezium.connector.cassandra.transforms.type.deserializer.AbstractMapTypeDeserializer.deserialize(AbstractMapTypeDeserializer.java:54)    at io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.deserialize(CassandraTypeDeserializer.java:133)    at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:567)    ... 16 more 2025-05-09 20:52:57,814 INFO  [io.deb.con.cas.QueueProcessor] (pool-17-thread-3) Encountered EOF event for CommitLog-680-1746819656929.log ...2025-05-09 20:52:57,821 INFO  [io.deb.con.cas.CommitLogUtil] (pool-17-thread-3) Moved CommitLog file /dse_logs/cdc_raw/CommitLog-680-1746819656929.log to /debezium-server/relocation/archive.2025-05-09 20:52:57,822 INFO  [io.deb.con.cas.CommitLogUtil] (pool-17-thread-3) Moved CommitLog index file /dse_logs/cdc_raw/CommitLog-680-1746819656929_cdc.idx to /debezium-server/relocation/archive.2025-05-09 20:53:03,743 INFO  [io.deb.con.cas.CommitLogUtil] (pool-16-thread-1) Deleted CommitLog file CommitLog-680-1746819656929.log from /debezium-server/relocation/archive.2025-05-09 20:53:03,743 INFO  [io.deb.con.cas.CommitLogUtil] (pool-16-thread-1) Deleted CommitLog index file CommitLog-680-1746819656929_cdc.idx from /debezium-server/relocation/archive.2025-05-09 20:53:06,985 WARN  [io.deb.con.cas.CommitLogIdxParser] (pool-15-thread-1) Commit log LogicalCommitLog{synced=0, completed=false, log=/dse_logs/cdc_raw/CommitLog-680-1746819656929.log, index=/dse_logs/cdc_raw/CommitLog-680-1746819656929_cdc.idx, commitLogSegmentId=1746819656929} does not exist!
      2025-05-05 17:56:25,000 ERROR [io.deb.con.cas.CommitLogIdxParser] (pool-15-thread-1) Processing of LogicalCommitLog{synced=6293991, completed=false, log=/dse_logs/cdc_raw/CommitLog-680-1746462573677.log, index=/dse_logs/cdc_raw/CommitLog-680-1746462573677_cdc.idx, commitLogSegmentId=1746462573677} errored out: io.debezium.DebeziumException: Error occurred while processing commit log /dse_logs/cdc_raw/CommitLog-680-1746462573677.log at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:144) at io.debezium.connector.cassandra.CommitLogIdxParser.parse(CommitLogIdxParser.java:84) at io.debezium.connector.cassandra.CommitLogIdxParser.process(CommitLogIdxParser.java:116) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642) at java.base/java.lang.Thread.run(Thread.java:1583)Caused by: io.debezium.DebeziumException: Failed to process PartitionUpdate [customer.as_reported_customers] key=FAKE_KEY partition_deletion=LIVE columns=[[] | [account_classification_cd account_open_date account_status_cd account_type_cd account_use_type_cd acquisition_id acquisition_id_change_indicator_ts acquisition_id_generated_ts acquisition_id_type additional_mailing_address as_reported_citizenship as_reported_data_source_name as_reported_normalized_values authoritative_date_of_birth authoritative_date_of_birth_change_indicator_ts authoritative_tax_identification_change_indicator_ts authoritative_tax_identification_last4_tax_identification_text authoritative_tax_identification_text authoritative_tax_identification_type business_established_date business_legal_structure_cd canada_model_ucid card_address country_of_orign_cd created_user_id customer_group_membership_cd customer_group_membership_id customer_type_cd customer_type_desc date_of_birth date_of_birth_change_indicator_ts deterministic_key ecid email employment_information enabled_model escid fraud_risk_level fraud_risk_level_source gender household_customer_group_membership_id ingestion_source ingestion_ts is_valid isolation_change_indication_ts isolation_description isolation_level isolation_reason_code language_cd legal_affiliate_id mailing_address marital_status mismatch_deceased_status model_weekly_prob_determ_key name normalized_additional_mailing_address normalized_card_address normalized_mailing_address normalized_physical_address ofac_flag origin_date phone_numbers phone_numbers_opt_out_status_v2 physical_address product_cd profit_center_cd row_id session_context sor_account_status_cd sor_account_status_desc status_flag tax_identification updated_ts us_model_ucid validation verification_level_cd verification_ts whitelist_email]] Row[info=[ts=EMPTY] ]: sor_id=1, sor_customer_id=123456789, customer_role_type_code=PR | [employment_information=@:@:@:@:@ ts=1614798254277000] at CommitLog-680-1746462573677.log:4430381 for table customer.as_reported_customers. at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:279) at org.apache.cassandra.db.commitlog.CommitLogReadHandler.handleMutation(CommitLogReadHandler.java:87) at org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:493) at org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:413) at org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:259) at io.debezium.connector.dse.DseCommitLogSegmentReader.readCommitLogSegment(DseCommitLogSegmentReader.java:32) at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:138) ... 6 moreCaused by: io.debezium.DebeziumException: Failed to populate Column employment_information with Type org.apache.cassandra.db.marshal.FrozenType(org.apache.cassandra.db.marshal.UserType(customer,61735f7265706f727465645f656d706c6f796d656e745f74797065,656d706c6f796d656e745f737461747573:org.apache.cassandra.db.marshal.UTF8Type,656d706c6f7965725f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,696e647573747279:org.apache.cassandra.db.marshal.UTF8Type,6f636375706174696f6e:org.apache.cassandra.db.marshal.UTF8Type,696e636f6d65:org.apache.cassandra.db.marshal.UTF8Type,696e636f6d655f73747261746567795f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,696e636f6d655f7265706f727465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,6f726967696e616c5f696e636f6d655f7265706f727465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,637573746f6d65725f70726f76696465645f696e636f6d655f696e64696361746f72:org.apache.cassandra.db.marshal.BooleanType,6d6f6e74686c795f686f7573696e675f657870656e7365:org.apache.cassandra.db.marshal.UTF8Type,6368616e67655f696e64696361746f725f7473:org.apache.cassandra.db.marshal.TimestampType,63757272656e63795f636f6465:org.apache.cassandra.db.marshal.UTF8Type,6368616e6e656c5f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,656d706c6f796d656e745f65737461626c69736865645f7473:org.apache.cassandra.db.marshal.TimestampType,6d6f6e74686c795f686f7573696e675f657870656e73655f65737461626c69736865645f7473:org.apache.cassandra.db.marshal.TimestampType,6164646974696f6e616c5f696e636f6d65:org.apache.cassandra.db.marshal.UTF8Type)) of Table as_reported_customers in KeySpace customer. at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:579) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowModifications(DseCommitLogReadHandlerImpl.java:437) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowIterator(DseCommitLogReadHandlerImpl.java:335) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.process(DseCommitLogReadHandlerImpl.java:320) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:276) ... 12 moreCaused by: java.lang.ArrayIndexOutOfBoundsException: Index 5 out of bounds for length 5 at java.base/java.util.Arrays$ArrayList.get(Arrays.java:4266) at io.debezium.connector.cassandra.transforms.type.deserializer.AbstractUserDefinedTypeDeserializer.deserialize(AbstractUserDefinedTypeDeserializer.java:38) at io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.deserialize(CassandraTypeDeserializer.java:121) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:571) ... 16 more
      
      
      2025-05-20 02:21:31,608 ERROR [io.deb.con.cas.CommitLogIdxParser] (pool-15-thread-1) Processing of LogicalCommitLog{synced=314669, completed=false, log=/dse_logs/cdc_raw/CommitLog-680-1746748070510.log, index=/dse_logs/cdc_raw/CommitLog-680-1746748070510_cdc.idx, commitLogSegmentId=1746748070510} errored out: io.debezium.DebeziumException: Error occurred while processing commit log /dse_logs/cdc_raw/CommitLog-680-1746748070510.log at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:144) at io.debezium.connector.cassandra.CommitLogIdxParser.parse(CommitLogIdxParser.java:84) at io.debezium.connector.cassandra.CommitLogIdxParser.process(CommitLogIdxParser.java:116) at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642) at java.base/java.lang.Thread.run(Thread.java:1583)Caused by: io.debezium.DebeziumException: Failed to process PartitionUpdate [customer.customers] key=FAKE_KEY partition_deletion=LIVE columns=[[] | [customer_citizenship customer_country_of_origin customer_date_of_birth customer_emails customer_employment_information customer_name customer_phone_numbers customer_phone_numbers_v2 customer_physical_address customer_preferred_name_v2 customer_tax_identification customer_type enabled_model mismatched_data ofac_restriction relationship_since solr_query customer_mailing_addresses customer_phone_numbers_v3 model]] Row[info=[ts=1747227401955000] ]: EMPTY | [customer_emails=fcc1e492-b403-4e5a-bcf2-b9125c87ae6b\:test\\@email.com\:CustomerPrimaryEmail\:FLD\:2022-08-18 14\\:03Z\:2024-05-17 13\\:24Z\:2025-04-08 19\\:55Z\:2025-04-08 19\\:55Z\:\@\:Hydrae_Hard_Bounce_Fail\:\@\:true\:\@:@ ts=1746428870221148], [customer_phone_numbers_v2=4168785050\:2024-05-28 20\\:15Z\:2024-06-04 00\\:47Z\:2024-06-04 00\\:47Z\:\@\:NO_TEXT\:000000010000005600000014455850495245445f564552494649434154494f4e00000036437573746f6d657220646964206e6f742070726f766964652068616e6473657420766572696669636174696f6e20726573706f6e7365ffffffff\:ALL_CALLS\:00000000\:00000000\:true\:false\:APIV4|enterpriseapi-8F6y3s3ukev3HsLaJw4vkx2U\:EXPIRE\:true\:\@\:\@\:\@\:\@\:\@\:\@\:\@\:\@\:\@\:\@:@:@ ts=1747227401955000], , [customer_mailing_addresses[b0c52724-021e-3279-b5e1-208c299b9f6b:123 Test St::::Test:ON:N3S3X9:CAN:@:2024-01-25 17\:42Z:2024-01-25 17\:42Z:2024-01-25 17\:42Z:TS2:@:ELEVATION_PIPELINE_2024-01-24:False:true:false:Active:@:@]= ts=1706204571443695] at CommitLog-680-1746748070510.log:308895 for table customer.customers. at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:279) at org.apache.cassandra.db.commitlog.CommitLogReadHandler.handleMutation(CommitLogReadHandler.java:87) at org.apache.cassandra.db.commitlog.CommitLogReader.readMutation(CommitLogReader.java:493) at org.apache.cassandra.db.commitlog.CommitLogReader.readSection(CommitLogReader.java:413) at org.apache.cassandra.db.commitlog.CommitLogReader.readCommitLogSegment(CommitLogReader.java:259) at io.debezium.connector.dse.DseCommitLogSegmentReader.readCommitLogSegment(DseCommitLogSegmentReader.java:32) at io.debezium.connector.cassandra.CommitLogIdxParser.processCommitLog(CommitLogIdxParser.java:138) ... 6 moreCaused by: io.debezium.DebeziumException: Failed to populate Column customer_mailing_addresses with Type org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.FrozenType(org.apache.cassandra.db.marshal.UserType(customer,6d61696c696e675f616464726573735f74797065,616464726573735f6964:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6c696e65315f74657874:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6c696e65325f74657874:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6c696e65335f74657874:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6c696e65345f74657874:org.apache.cassandra.db.marshal.UTF8Type,636974795f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,73746174655f636f6465:org.apache.cassandra.db.marshal.UTF8Type,66756c6c5f706f7374616c5f636f6465:org.apache.cassandra.db.marshal.UTF8Type,636f756e7472795f636f6465:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6163636f756e745f707265666572656e636573:org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.UserType(customer,616464726573735f6163636f756e745f707265666572656e63655f74797065,6163636f756e74:org.apache.cassandra.db.marshal.UserType(customer,6163636f756e745f74797065,6163636f756e745f7265666572656e63655f6964656e74696669636174696f6e:org.apache.cassandra.db.marshal.UTF8Type,73797374656d5f6f665f7265636f72645f637573746f6d65725f6964656e74696669636174696f6e:org.apache.cassandra.db.marshal.UTF8Type),616464726573735f726f6c6573:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UserType(customer,616464726573735f726f6c655f74797065,616464726573735f726f6c655f636f6465:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f726f6c655f6465736372697074696f6e:org.apache.cassandra.db.marshal.UTF8Type)))),65737461626c69736865645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,636f6e6669726d6174696f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,6c6173745f757064617465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,61735f7265706f727465645f646174615f736f757263655f6e616d65:org.apache.cassandra.db.marshal.UTF8Type,766572696669636174696f6e5f636c61696d73:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UserType(customer,766572696669636174696f6e5f636c61696d5f74797065,766572696669636174696f6e5f636c61696d5f6964:org.apache.cassandra.db.marshal.UTF8Type,766572696669636174696f6e5f636c61696d5f7369676e6174757265:org.apache.cassandra.db.marshal.UTF8Type,766572696669636174696f6e5f636c61696d5f6d65746164617461:org.apache.cassandra.db.marshal.UTF8Type,766572696669636174696f6e5f636c61696d5f70726f76696465645f6f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,766572696669636174696f6e5f636c61696d5f636f6e6669726d6174696f6e5f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType)),6c6173745f6d6f6469666965645f736f757263655f73797374656d:org.apache.cassandra.db.marshal.UTF8Type,637573746f6d65725f737065636966696564:org.apache.cassandra.db.marshal.UTF8Type,64656661756c745f61646472657373:org.apache.cassandra.db.marshal.BooleanType,6d69736d6174636865645f64617461:org.apache.cassandra.db.marshal.BooleanType,616464726573735f737461747573:org.apache.cassandra.db.marshal.UTF8Type,64656c69766572795f706f696e74:org.apache.cassandra.db.marshal.UTF8Type,6e697869655f636f6465:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f76616c69645f696e64696361746f72:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f6e6f726d616c697a6174696f6e5f736f75726365:org.apache.cassandra.db.marshal.UTF8Type,616464726573735f76616c6964617465645f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType,616464726573735f76616c69646174696f6e5f6d65746164617461:org.apache.cassandra.db.marshal.UTF8Type,72657475726e65645f6d61696c5f666c6167:org.apache.cassandra.db.marshal.BooleanType,72657475726e65645f6d61696c5f666c61675f7374617475735f74696d657374616d70:org.apache.cassandra.db.marshal.TimestampType))) of Table customers in KeySpace customer. at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:579) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowModifications(DseCommitLogReadHandlerImpl.java:437) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleRowIterator(DseCommitLogReadHandlerImpl.java:335) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.process(DseCommitLogReadHandlerImpl.java:320) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.handleMutation(DseCommitLogReadHandlerImpl.java:276) ... 12 moreCaused by: java.lang.ArrayIndexOutOfBoundsException: Index 22 out of bounds for length 22 at java.base/java.util.Arrays$ArrayList.get(Arrays.java:4266) at io.debezium.connector.cassandra.transforms.type.deserializer.AbstractUserDefinedTypeDeserializer.deserialize(AbstractUserDefinedTypeDeserializer.java:38) at io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.deserialize(CassandraTypeDeserializer.java:121) at io.debezium.connector.cassandra.transforms.type.deserializer.AbstractSetTypeDeserializer.deserialize(AbstractSetTypeDeserializer.java:49) at io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.deserialize(CassandraTypeDeserializer.java:133) at io.debezium.connector.dse.DseCommitLogReadHandlerImpl.populateRegularColumns(DseCommitLogReadHandlerImpl.java:567) ... 16 more 2025-05-20 02:21:31,658 INFO [io.deb.con.cas.QueueProcessor] (pool-17-thread-1) Encountered EOF event for CommitLog-680-1746748070510.log ...2025-05-20 02:21:31,659 INFO [io.deb.con.cas.CommitLogUtil] (pool-17-thread-1) Moved CommitLog file /dse_logs/cdc_raw/CommitLog-680-1746748070510.log to /debezium-server/relocation/archive.2025-05-20 02:21:31,659 INFO [io.deb.con.cas.CommitLogUtil] (pool-17-thread-1) Moved CommitLog index file /dse_logs/cdc_raw/CommitLog-680-1746748070510_cdc.idx to /debezium-server/relocation/archive.2025-05-20 02:21:37,640 INFO [io.deb.con.cas.CommitLogUtil] (pool-16-thread-6) Deleted CommitLog file CommitLog-680-1746748070510.log from /debezium-server/relocation/archive.2025-05-20 02:21:37,640 INFO [io.deb.con.cas.CommitLogUtil] (pool-16-thread-6) Deleted CommitLog index file CommitLog-680-1746748070510_cdc.idx from /debezium-server/relocation/archive.2025-05-20 02:21:41,604 WARN [io.deb.con.cas.CommitLogIdxParser] (pool-15-thread-1) Commit log LogicalCommitLog{synced=0, completed=false, log=/dse_logs/cdc_raw/CommitLog-680-1746748070510.log, index=/dse_logs/cdc_raw/CommitLog-680-1746748070510_cdc.idx, commitLogSegmentId=1746748070510} does not exist!
      
      

       

      How to reproduce the issue using our tutorial deployment?

      I have not been able to intentionally reproduce this issue, occasionally this happens when an application makes an update to a row, other times, no updates are made but Debezium still encounters this bug. As far as I can tell there seems to be a correlation with UDTs and this bug as I have run this same setup in a cluster without UDTs and have not seen this error.

      This bug is preventing me from being able to consider the Debezium Server as a viable option for my CDC implementation.

              Unassigned Unassigned
              e_muniz Elber Muniz (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

                Created:
                Updated: