--
You received this message because you are subscribed to a topic in the Google Groups "debezium" group.
To unsubscribe from this topic, visit https://groups.google.com/d/topic/debezium/B0-HIJk0VHo/unsubscribe.
To unsubscribe from this group and all its topics, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/a2d11efa-64c4-4f58-a361-0181e4e476b7n%40googlegroups.com.
We were able to resolve the flush timeout issue by increasing the following Kafka Connect configuration: status.storage.partitions from 5 to 25 (https://docs.confluent.io/platform/current/installation/configuration/connect/index.html#connectconfigs_status.storage.partitions).
Here are the steps we took which prompted us to update the value:
We observed various metrics for our connectors and noticed that the buffer-available-bytes metric for all of our connectors that were stuck had reached 0 but never recovered. At first glance this was somewhat expected since we could see that data was backing up though it seemed like a symptom of the problem, and not the cause.
As per the documentation, buffer.memory is:
"The total bytes of memory the producer can use to buffer records waiting to be sent to the server. If records are sent faster than they can be delivered to the server the producer will block for max.block.ms after which it will throw an exception.
This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if compression is enabled) as well as for maintaining in-flight requests."
We set the logging level to TRACE for the connectors for all producer loggers to see if we could find any exceptions that were being thrown related to the buffer memory.
This was done using the following command:
curl -s -X PUT -H "Content-Type:application/json" http://localhost:8083/admin/loggers/org.apache.kafka.clients.producer -d '{"level": "TRACE"}' | jq '.'
Then, we noticed the following message within the logs:
[2021-11-23 00:48:10,071] DEBUG [Producer clientId=producer-2] Exception occurred during message send: (org.apache.kafka.clients.producer.KafkaProducer) org.apache.kafka.clients.producer.BufferExhaustedException: Failed to allocate memory within the configured max blocking time 60000 ms.
Instead, we looked for other configurations that we had the ability to modify. We looked at the associated configurations for kafka connect and found that the number of partitions defaults to 5 based on this configuration.
Given the number of connectors/tables/topics for our use case, 5 partitions seemed too small. We decided to increase the number of partitions to 25 (based on the default configuration value for another internally managed topic: offset.storage.partitons). Once we increased the number of partitions to 25, both Failed to commit offsets and BufferExhaustedException messages were no longer thrown. In summary, our problem seems to have been that kafka connect needed further partitioning of the status storage topic to keep up when running as many workers and connectors as we are in the same worker group. After restarting all our connectors from scratch, we have not seen the same problem reoccur again.
Lingering questions and suggestions (For kafka connect, not specific to debezium)::
1. Why does the org.apache.kafka.connect.runtime.WorkerSourceTask execute loop produce no log output in this situation even while in TRACE? It's not clear to us what code path is being executed or where it is blocked.
2. The names of producer-1, producer-2, producer-3 would be more helpful if they included something about their function.
--
You received this message because you are subscribed to a topic in the Google Groups "debezium" group.
To unsubscribe from this topic, visit https://groups.google.com/d/topic/debezium/B0-HIJk0VHo/unsubscribe.
To unsubscribe from this group and all its topics, send an email to debezium+u...@googlegroups.com.
To view this discussion on the web visit https://groups.google.com/d/msgid/debezium/a2d11efa-64c4-4f58-a361-0181e4e476b7n%40googlegroups.com.