Github user srdo commented on a diff in the pull request: https://github.com/apache/storm/pull/2637#discussion_r185174572 --- Diff: docs/storm-kafka-client.md --- @@ -313,4 +313,37 @@ KafkaSpoutConfig<String, String> kafkaConf = KafkaSpoutConfig .setTupleTrackingEnforced(true) ``` -Note: This setting has no effect with AT_LEAST_ONCE processing guarantee, where tuple tracking is required and therefore always enabled. \ No newline at end of file +Note: This setting has no effect with AT_LEAST_ONCE processing guarantee, where tuple tracking is required and therefore always enabled. + +# Migrating a `storm-kafka` spout to use `storm-kafka-client` + +This may not be an exhaustive list because the `storm-kafka` configs were taken from Storm 0.9.6 +[SpoutConfig](https://github.com/apache/storm/blob/v0.9.6/external/storm-kafka/src/jvm/storm/kafka/SpoutConfig.java) and +[KafkaConfig](https://github.com/apache/storm/blob/v0.9.6/external/storm-kafka/src/jvm/storm/kafka/KafkaConfig.java). +`storm-kafka-client` spout configurations were taken from Storm 1.0.6 +[KafkaSpoutConfig](https://github.com/apache/storm/blob/v1.0.6/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutConfig.java). + +| Storm-0.9.6 SpoutConfig | Storm-1.0.6 KafkaSpoutConfig name | KafkaSpoutConfig usage help | +| ------------------------- | ---------------------------- | --------------------------- | +| **Setting:** `startOffsetTime`<br><br> **Default:** `EarliestTime`<br>________________________________________________ <br> **Setting:** `forceFromStart` <br><br> **Default:** `false` <br><br> `startOffsetTime` & `forceFromStart` together determine the starting offset. `forceFromStart` determines whether the Zookeeper offset is ignored. `startOffsetTime` sets the timestamp that determines the beginning offset, in case there is no offset in Zookeeper, or the Zookeeper offset is ignored | **Setting:** [`FirstPollOffsetStrategy`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.FirstPollOffsetStrategy.html)<br><br> **Default:** `UNCOMMITTED_EARLIEST` <br><br> [Refer to the helper table](#helper-table-for-setting-firstpolloffsetstrategy) for picking `FirstPollOffsetStrategy` based on your `startOffsetTime` & `forceFromStart` settings | **Import package:** `org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy.<strategy-name>` <br><br> **Usage:** [`<KafkaSpout Config-Builder>.setFirstPollOffsetStrategy(<strategy-name>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setFirstPollOffsetStrategy-org.apache.storm.kafka.spout.KafkaSpoutConfig.FirstPollOffsetStrategy-)| +| **Setting:** `scheme`<br><br> The interface that specifies how a `ByteBuffer` from a Kafka topic is transformed into Storm tuple <br>**Default:** `RawMultiScheme` | [`Deserializers`](https://kafka.apache.org/11/javadoc/org/apache/kafka/common/serialization/Deserializer.html)| **Import package:** `import org.apache.kafka.clients.consumer.ConsumerConfig;` <br><br> **Usage:** [`<KafkaSpoutConfig-Builder>.setProp(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, <deserializer-class>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setProp-java.lang.String-java.lang.Object-)<br><br> [`<KafkaSpoutConfig-Builder>.setProp(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, <deserializer-class>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setProp-java.lang.String-java.lang.Object-)| +| **Setting:** `fetchSizeBytes`<br><br> Message fetch size -- the number of bytes to attempt to fetch in one request to a Kafka server <br> **Default:** `1MB` | **Kafka config:** [`max.partition.fetch.bytes`](https://kafka.apache.org/11/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#MAX_PARTITION_FETCH_BYTES_CONFIG)<br><br> **Default:** `1MB`| **Import package:** `import org.apache.kafka.clients.consumer.ConsumerConfig;` <br><br> **Usage:** [`<KafkaSpoutConfig-Builder>.setProp(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, <int-value>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setProp-java.lang.String-java.lang.Object-)| +| **Setting:** `bufferSizeBytes`<br><br> Buffer size (in bytes) for network requests. The buffer size which consumer has for pulling data from producer <br> **Default:** `1MB`| **Kafka config:** [`receive.buffer.bytes`](https://kafka.apache.org/11/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#RECEIVE_BUFFER_CONFIG) <br><br> 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 | **Import package:** `import org.apache.kafka.clients.consumer.ConsumerConfig;` <br><br> **Usage:** [`<KafkaSpoutConfig-Builder>.setProp(ConsumerConfig.RECEIVE_BUFFER_CONFIG, <int-value>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setProp-java.lang.String-java.lang.Object-)| +| **Setting:** `socketTimeoutMs`<br><br> **Default:** `10000` | Discontinued in `storm-kafka-client` || +| **Setting:** `useStartOffsetTimeIfOffsetOutOfRange`<br><br> **Default:** `true` | **Kafka config:** [`auto.offset.reset`](https://kafka.apache.org/11/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#AUTO_OFFSET_RESET_CONFIG)<br><br> **Possible values:** `"latest"`, `"earliest"`, `"none"`<br> **Default:** `latest`. Exception: `earliest` if [`ProcessingGuarantee`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.ProcessingGuarantee.html) is set to `AT_LEAST_ONCE` | **Import package:** `import org.apache.kafka.clients.consumer.ConsumerConfig;` <br><br> **Usage:** [`<KafkaSpoutConfig-Builder>.setProp(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, <String>)`](javadocs/org/apache/storm/kafka/spout/KafkaSpoutConfig.Builder.html#setProp-java.lang.String-java.lang.Object-)| --- End diff -- Keep in mind that our default for this setting is different from the default for the KafkaConsumer.
---