[ https://issues.apache.org/jira/browse/KAFKA-18073?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17903763#comment-17903763 ]
Tom Thornton commented on KAFKA-18073: -------------------------------------- The issue appears to be from errors.tolerance. We do not specify a config so it defaults to "none" {code:java} SourceConnectorConfig values: config.action.reload = restart connector.class = io.debezium.connector.vitess.VitessConnector errors.log.enable = false errors.log.include.messages = false errors.retry.delay.max.ms = 60000 errors.retry.timeout = 0 errors.tolerance = none exactly.once.support = requested header.converter = null key.converter = null name = vifl-4-users_prefs-connector offsets.storage.topic = null predicates = [] tasks.max = 300 tasks.max.enforce = true topic.creation.groups = [] transaction.boundary = poll transaction.boundary.interval.ms = null transforms = [filterTransactionTopicRecords, removeField, useLocalVgtid] value.converter = null {code} However the connector is still skipping messages, as we see the Grafana metric kafka_connect_task_error_metrics_total_records_skipped will start reporting skipped records when we observe data loss. This value jumps from no value to 1. > Data loss when Kafka Connect retriable exception raised > ------------------------------------------------------- > > Key: KAFKA-18073 > URL: https://issues.apache.org/jira/browse/KAFKA-18073 > Project: Kafka > Issue Type: Bug > Components: connect > Reporter: Tom Thornton > Priority: Major > > We are experiencing data loss when Kafka Connect source connector fails to > send data to Kafka topic after receiving a retriable exception from a schema > registry (when converting the record). > Kafka cluster & connect details: > # Kafka Connect version: 3.8.1 > # Kafka Version 3.6.1 > # Cluster size: 16 brokers > # Number of partitions in Kafka data topic: 32 > # Number of partitions in offsets topic: > # Replication Factor: 4 > # Min In Sync Replicas: 2 > # Unclean leader election disabled > # Uses Apicurio schema registry to convert to Avro > # Uses default error tolerance i.e., None > Producer configs > * acks = -1 > * max.in.flight.requests.per.connection = 1 > * batch.size = 524288 > Our connector uses the offsets committed by Kafka Connect (does not have any > internal checkpointing/state storage). > Data loss is noticed on occasion when a timeout is received during the > [convertTransformedRecod|https://github.com/apache/kafka/blob/3.8.1/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java#L398] > call. The timeout is from connecting to the schema registry for the schema. > Full stack trace > [here|https://gist.github.com/twthorn/5764b9dae2664ca72e71c1e6ebd4c792] > Logs (dropped record is from task 79): > {code:java} > 2024-10-22T01:37:41.059Z - INFO - WorkerSourceTask{id=keyspace1-connector-79} > Committing offsets for 3 acknowledged messages > 2024-10-22T01:37:41.066Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-232} Committing offsets for 1 > acknowledged messages > 2024-10-22T01:37:41.540Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-259} Committing offsets for 3 > acknowledged messages > 2024-10-22T01:37:41.928Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-273} Committing offsets for 2 > acknowledged messages > 2024-10-22T01:37:44Z - info - Handshake message sent > 2024-10-22T01:37:51Z - info - Handshake timed out > 2024-10-22T01:37:52Z - info - Handshake message sent > 2024-10-22T01:37:58Z - info - Handshake timed out > 2024-10-22T01:38:04Z - info - Handshake message sent > 2024-10-22T01:38:10Z - info - Handshake timed out > 2024-10-22T01:38:11.664Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-268] Node 1078 > disconnected. > 2024-10-22T01:38:15.798Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-219] Node 1089 > disconnected. > 2024-10-22T01:38:20Z - info - Handshake message sent > 2024-10-22T01:38:23.225Z - INFO - [Producer > clientId=connector-producer-keyspace1-connector-107] Node 1085 disconnected. > 2024-10-22T01:38:23.224Z - INFO - [Producer > clientId=connector-producer-keyspace1-connector-107] Node 1088 disconnected. > 2024-10-22T01:38:24.536Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-13] Node 1076 > disconnected. > 2024-10-22T01:38:27Z - info - Handshake timed out > 2024-10-22T01:38:27.491Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-237] Node 1082 > disconnected. > 2024-10-22T01:38:27.492Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-237] Node 1089 > disconnected. > 2024-10-22T01:38:27.493Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-237] Node 1085 > disconnected. > 2024-10-22T01:38:27.832Z - INFO - [Producer > clientId=connector-producer-keyspace1-table1-connector-237] Node 1088 > disconnected. > 2024-10-22T01:38:28.470Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-12} Committing offsets for 2 > acknowledged messages > 2024-10-22T01:38:28.549Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-64} Committing offsets for 14 > acknowledged messages > 2024-10-22T01:38:36.930Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-53} Committing offsets for 38 > acknowledged messages > 2024-10-22T01:38:37.701Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-13} Committing offsets for 2 > acknowledged messages > 2024-10-22T01:38:37.838Z - INFO - [AdminClient > clientId=dbzium-prod-keyspace1-shared-admin] Node 1089 disconnected. > 2024-10-22T01:38:38.369Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-25} Committing offsets for 6 > acknowledged messages > 2024-10-22T01:38:39.212Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-56} Committing offsets for 21 > acknowledged messages > 2024-10-22T01:38:39.597Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-51} Committing offsets for 11 > acknowledged messages > 2024-10-22T01:38:39.642Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-219} Committing offsets for 4 > acknowledged messages > 2024-10-22T01:38:39.647Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-237} Committing offsets for 1 > acknowledged messages > 2024-10-22T01:38:39.681Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-268} Committing offsets for 8 > acknowledged messages > 2024-10-22T01:38:41.059Z - INFO - > WorkerSourceTask{id=keyspace1-connector-107} Committing offsets for 3 > acknowledged messages > 2024-10-22T01:38:41.540Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-232} Committing offsets for 3 > acknowledged messages > 2024-10-22T01:38:41.928Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-259} Committing offsets for 2 > acknowledged messages > 2024-10-22T01:38:42.252Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-273} Committing offsets for 1 > acknowledged messages > 2024-10-22T01:38:44.913Z - ERROR - Failed to send HTTP request to endpoint: > http://schema-registry.service.prod-us-east-1-dw1.consul:8080/apis/ccompat/v6/subjects/prod.keyspace1.table1-key/versions?normalize=false > 2024-10-22T01:38:52Z - info - Handshake message sent > 2024-10-22T01:39:06.790Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Rebalance started > 2024-10-22T01:39:07.134Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Successfully joined group with generation > Generation{generationId=2477, > memberId='connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083-a174acf5-1b8f-419a-a25a-4c58dab02a8d', > protocol='sessioned'} > 2024-10-22T01:39:07.171Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Successfully synced group in generation > Generation{generationId=2477, > memberId='connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083-a174acf5-1b8f-419a-a25a-4c58dab02a8d', > protocol='sessioned'} > 2024-10-22T01:39:07.171Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Joined group at generation 2477 with protocol > version 2 and got assignment: Assignment{error=0, > leader='connect-dbzium-prod-keyspace1-b7cfc9bcc-h8gxb.node.prod-us-east-1-vitess1.consul:8083-dbbc75cb-1de9-4907-bf4c-f3f6cca6eb9a', > > leaderUrl='http://dbzium-prod-keyspace1-b7cfc9bcc-h8gxb.node.prod-us-east-1-vitess1.consul:8083/', > offset=236838, connectorIds=[], taskIds=[keyspace1-table1-connector-12, > keyspace1-table1-connector-13, keyspace1-table1-connector-25, > keyspace1-table1-connector-51, keyspace1-table1-connector-53, > keyspace1-table1-connector-56, keyspace1-table1-connector-64, > keyspace1-table1-connector-219, keyspace1-table1-connector-232, > keyspace1-table1-connector-237, keyspace1-table1-connector-259, > keyspace1-table1-connector-268, keyspace1-table1-connector-273, > keyspace1-connector-79, keyspace1-connector-107, keyspace1-connector-285], > revokedConnectorIds=[], revokedTaskIds=[], delay=180000} with rebalance > delay: 180000 > 2024-10-22T01:39:07.172Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Finished starting connectors and tasks > 2024-10-22T01:39:07.172Z - INFO - [Worker > clientId=connect-dbzium-prod-keyspace1-b7cfc9bcc-8z59m.node.prod-us-east-1-vitess1.consul:8083, > groupId=dbzium-prod-keyspace1] Starting connectors and tasks using config > offset 236838 > 2024-10-22T01:39:12Z - info - Handshake message received > 2024-10-22T01:39:12Z - info - Handshake message sent > 2024-10-22T01:39:27.534Z - INFO - [Producer > clientId=connector-producer-keyspace1-connector-107] Node 1074 disconnected. > 2024-10-22T01:39:27.535Z - INFO - [Producer > clientId=connector-producer-keyspace1-connector-107] Node 1077 disconnected. > 2024-10-22T01:39:28.550Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-12} Committing offsets for 4 > acknowledged messages > 2024-10-22T01:39:28.622Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-64} Committing offsets for 48 > acknowledged messages > 2024-10-22T01:39:37.102Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-53} Committing offsets for 30 > acknowledged messages > 2024-10-22T01:39:38.194Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-13} Committing offsets for 3 > acknowledged messages > 2024-10-22T01:39:38.556Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-25} Committing offsets for 14 > acknowledged messages > 2024-10-22T01:39:39.598Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-56} Committing offsets for 15 > acknowledged messages > 2024-10-22T01:39:40.043Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-51} Committing offsets for 8 > acknowledged messages > 2024-10-22T01:39:40.086Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-219} Committing offsets for 6 > acknowledged messages > 2024-10-22T01:39:40.096Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-237} Committing offsets for 1 > acknowledged messages > 2024-10-22T01:39:40.100Z - INFO - > WorkerSourceTask{id=keyspace1-table1-connector-268} Committing offsets for 8 > acknowledged messages > 2024-10-22T01:39:41.392Z - INFO - WorkerSourceTask{id=keyspace1-connector-79} > Committing offsets for 12 acknowledged messages{code} > Does Kafka Connect commit offsets for a batch even if some of the records > failed? Or must the whole batch be successful to commit offsets? > It is odd there is no restart, just a logged error that should have been > retried. -- This message was sent by Atlassian Jira (v8.20.10#820010)