[ https://issues.apache.org/jira/browse/IGNITE-19910?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Ilya Shishkov updated IGNITE-19910: ----------------------------------- Description: Currently, in CDC through Kafka applications, single timeout property ({{kafkaRequestTimeout)}} is used for all Kafka related operations instead of built-in timeouts of Kafka clients (moreover, default value of 3 seconds does not correspond to Kafka clients defaults): ||Client||Timeout||Default value, s|| |{{KafkaProducer}}|{{delivery.timeout.ms}}|120| |{{KafkaProducer}}|{{request.timeout.ms}}|30| |{{KafkaConsumer}}|{{default.api.timeout.ms}}|60| |{{KafkaConsumer}}|{{request.timeout.ms}}|30| ---- Kafka clients use this defaults in some of client API, also timeouts are used in recovery process. A table below describe places where {{kafkaRequestTimeout}} is used instead of default values: ||CDC application||API||Description||Default|| |ignite-cdc.sh: {{IgniteToKafkaCdcStreamer}}|{{KafkaProducer#send}}|Waits for future during specified timeout|Future fail itself if producer can no send message in {{delivery.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteCdcStreamerApplier}}|{{KafkaConsumer#commitSync}}| \- |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#commitSync}}| \- |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#partitionsFor}}| \- |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#endOffsets}}| \- |{{request.timeout.ms}}| All above methods will fail, when specified timeout will be exceeded. They will raise an exception, thus, timeout _should not be to low for them_. On the other hand, kafka-to-ignite.sh also invokes {{KafkaConsumer#poll}} with timeout {{kafkaRequestTimeout}}, but it just waits for data until specified timeout expires. So, {{#poll}} should be called quite often and if we set too large timeout for it, then we can face with delays of replication, when some topic partitions have no new data in topics. It is not desired behavior, because in this situation we can have pending updates which will wait to be processed. ---- Links: # https://kafka.apache.org/27/documentation.html#producerconfigs_delivery.timeout.ms # https://kafka.apache.org/27/documentation.html#producerconfigs_request.timeout.ms # https://kafka.apache.org/27/documentation.html#consumerconfigs_default.api.timeout.ms # https://kafka.apache.org/27/documentation.html#consumerconfigs_request.timeout.ms was: Currently, in CDC through Kafka applications, single timeout property ({{kafkaRequestTimeout)}} is used for all Kafka related operations instead of built-in timeouts of Kafka clients (moreover, default value of 3 seconds does not correspond to Kafka clients defaults): ||Client||Timeout||Default value, s|| |{{KafkaProducer}}|{{delivery.timeout.ms}}|120| |{{KafkaProducer}}|{{request.timeout.ms}}|30| |{{KafkaConsumer}}|{{default.api.timeout.ms}}|60| |{{KafkaConsumer}}|{{request.timeout.ms}}|30| ---- Kafka clients use this defaults in some of client API, also timeouts are used in recovery process. A table below describe places where {{kafkaRequestTimeout}} is used instead of default values: ||CDC application||API||Description||Default|| |ignite-cdc.sh: {{IgniteToKafkaCdcStreamer}}|{{KafkaProducer#send}}|Waits for future during specified timeout|Future fail itself if producer can no send message in {{delivery.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteCdcStreamerApplier}}|{{KafkaConsumer#commitSync}}| - |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#commitSync}}| \- |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#partitionsFor}}| \- |{{default.api.timeout.ms}}| |kafka-to-ignite.sh: {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#endOffsets}}| \- |{{request.timeout.ms}}| All above methods will fail, when specified timeout will be exceeded. They will raise an exception, thus, timeout _should not be to low for them_. On the other hand, kafka-to-ignite.sh also invokes {{KafkaConsumer#poll}} with timeout {{kafkaRequestTimeout}}, but it just waits for data until specified timeout expires. So, {{#poll}} should be called quite often and if we set too large timeout for it, then we can face with delays of replication, when some topic partitions have no new data in topics. It is not desired behavior, because in this situation we can have pending updates which will wait to be processed. ---- Links: # https://kafka.apache.org/27/documentation.html#producerconfigs_delivery.timeout.ms # https://kafka.apache.org/27/documentation.html#producerconfigs_request.timeout.ms # https://kafka.apache.org/27/documentation.html#consumerconfigs_default.api.timeout.ms # https://kafka.apache.org/27/documentation.html#consumerconfigs_request.timeout.ms > CDC through Kafka: refactor timeouts > ------------------------------------ > > Key: IGNITE-19910 > URL: https://issues.apache.org/jira/browse/IGNITE-19910 > Project: Ignite > Issue Type: Task > Components: extensions > Reporter: Ilya Shishkov > Priority: Minor > Labels: IEP-59, ise > > Currently, in CDC through Kafka applications, single timeout property > ({{kafkaRequestTimeout)}} is used for all Kafka related operations instead of > built-in timeouts of Kafka clients (moreover, default value of 3 seconds does > not correspond to Kafka clients defaults): > ||Client||Timeout||Default value, s|| > |{{KafkaProducer}}|{{delivery.timeout.ms}}|120| > |{{KafkaProducer}}|{{request.timeout.ms}}|30| > |{{KafkaConsumer}}|{{default.api.timeout.ms}}|60| > |{{KafkaConsumer}}|{{request.timeout.ms}}|30| > ---- > Kafka clients use this defaults in some of client API, also timeouts are used > in recovery process. A table below describe places where > {{kafkaRequestTimeout}} is used instead of default values: > ||CDC application||API||Description||Default|| > |ignite-cdc.sh: {{IgniteToKafkaCdcStreamer}}|{{KafkaProducer#send}}|Waits for > future during specified timeout|Future fail itself if producer can no send > message in {{delivery.timeout.ms}}| > |kafka-to-ignite.sh: > {{KafkaToIgniteCdcStreamerApplier}}|{{KafkaConsumer#commitSync}}| \- > |{{default.api.timeout.ms}}| > |kafka-to-ignite.sh: > {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#commitSync}}| \- > |{{default.api.timeout.ms}}| > |kafka-to-ignite.sh: > {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#partitionsFor}}| \- > |{{default.api.timeout.ms}}| > |kafka-to-ignite.sh: > {{KafkaToIgniteMetadataUpdater}}|{{KafkaConsumer#endOffsets}}| \- > |{{request.timeout.ms}}| > All above methods will fail, when specified timeout will be exceeded. They > will raise an exception, thus, timeout _should not be to low for them_. > On the other hand, kafka-to-ignite.sh also invokes {{KafkaConsumer#poll}} > with timeout {{kafkaRequestTimeout}}, but it just waits for data until > specified timeout expires. So, {{#poll}} should be called quite often and if > we set too large timeout for it, then we can face with delays of replication, > when some topic partitions have no new data in topics. It is not desired > behavior, because in this situation we can have pending updates which will > wait to be processed. > ---- > Links: > # > https://kafka.apache.org/27/documentation.html#producerconfigs_delivery.timeout.ms > # > https://kafka.apache.org/27/documentation.html#producerconfigs_request.timeout.ms > # > https://kafka.apache.org/27/documentation.html#consumerconfigs_default.api.timeout.ms > # > https://kafka.apache.org/27/documentation.html#consumerconfigs_request.timeout.ms -- This message was sent by Atlassian Jira (v8.20.10#820010)