[
https://issues.apache.org/jira/browse/KAFKA-3840?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Ismael Juma resolved KAFKA-3840.
--------------------------------
Resolution: Fixed
Fix Version/s: 0.10.1.0
Issue resolved by pull request 1507
[https://github.com/apache/kafka/pull/1507]
> OS auto tuning for socket buffer size in clients not allowed through
> configuration
> ----------------------------------------------------------------------------------
>
> Key: KAFKA-3840
> URL: https://issues.apache.org/jira/browse/KAFKA-3840
> Project: Kafka
> Issue Type: Bug
> Reporter: Sébastien Launay
> Priority: Minor
> Fix For: 0.10.1.0
>
>
> KAFKA-724 added the following documentation for clients in order to provide
> OS auto tuning of socket buffer sizes:
> {noformat}
> diff --git
> a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
> b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
> index 750b8a1..3327815 100644
> --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
> +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
> @@ -43,10 +43,10 @@
> public static final String METADATA_MAX_AGE_DOC = "The period of time in
> milliseconds after which we force a refresh of metadata even if we haven't
> seen any partition leadership changes to proactively discover any new brokers
> or partitions.";
>
> public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
> - public static final String SEND_BUFFER_DOC = "The size of the TCP send
> buffer (SO_SNDBUF) to use when sending data.";
> + public static final String SEND_BUFFER_DOC = "The size of the TCP send
> buffer (SO_SNDBUF) to use when sending data. If the value is -1, the OS
> default will be used.";
>
> public static final String RECEIVE_BUFFER_CONFIG =
> "receive.buffer.bytes";
> - public static final String RECEIVE_BUFFER_DOC = "The size of the TCP
> receive buffer (SO_RCVBUF) to use when reading data.";
> + public static final String RECEIVE_BUFFER_DOC = "The size of the TCP
> receive buffer (SO_RCVBUF) to use when reading data. If the value is -1, the
> OS default will be used.";
>
> public static final String CLIENT_ID_CONFIG = "client.id";
> public static final String CLIENT_ID_DOC = "An id string to pass to the
> server when making requests. The purpose of this is to be able to track the
> source of requests beyond just ip/port by allowing a logical application name
> to be included in server-side request logging.";
> {noformat}
> Unfortunately, the value {{-1}} is not allowed for clients configuration and
> the following exception is thrown:
> {noformat}
> Exception in thread "main" org.apache.kafka.common.config.ConfigException:
> Invalid value -1 for configuration receive.buffer.bytes: Value must be at
> least 0
> at
> org.apache.kafka.common.config.ConfigDef$Range.ensureValid(ConfigDef.java:308)
> at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:153)
> at
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:49)
> at
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:56)
> at
> org.apache.kafka.clients.producer.ProducerConfig.<init>(ProducerConfig.java:315)
> at
> org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:156)
> ...
> {noformat}
> This is caused by an {{atLeast(0)}} validation in {{ProducerConfig.java}} and
> {{ConsumerConfig.java}}::
> {noformat}
> ...
> .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0),
> Importance.MEDIUM, CommonClientConfigs.SEND_BUFFER_DOC)
> .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0),
> Importance.MEDIUM, CommonClientConfigs.RECEIVE_BUFFER_DOC)
> ...
> {noformat}
> Note that the broker configuration for OS auto tuning is fine as there is no
> such checks in {{KafkaConfig.scala}}.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)