Your KafkaConsumer instance (which boils down to your KafkaSpout) can be in one 
of two states:

1 - Has committed to Kafka
Here, EARLIEST fetches from the first offset. LATEST fetches from the last 
offset. UNCOMMITTED_EARLIEST and UNCOMMITTED_LATEST will fetch from the last 
committed offset - the  _earliest and _latest are “kinda” disregarded.

2 - Has never committed to Kafka
EARLIEST is the same as UNCOMMITTED_EARLIEST
LATEST is the same as UNCOMMITTED_LATEST

What this means is that since this KafkaSpout (Kafka Consumer Instance) has 
never committed to Kafka, when you do the first poll, from which offset do you 
start. We give the option to start from the beginning (EARLIEST) or from the 
end (LATEST).

Best,
Hugo


> On Jun 28, 2017, at 9:47 AM, Stig Døssing <[email protected]> wrote:
> 
> No, the description is accurate.
> 
> EARLIEST and LATEST are for unconditionally starting at the beginning or
> end of the subscribed partitions. So if you configure a spout to use either
> of these, it will start at the earliest or latest offset on each partition
> every time you start it. Example: Say the spout is set to EARLIEST and
> we've just deployed it for the first time. The spout seeks to the earliest
> offset (let's say 0) and emits offsets 0-100, and commits them (marking
> them as "done" in Kafka for the spout's consumer group, this happens when
> you ack the tuples emitted by the spout). The spout then crashes for some
> reason, or you redeploy the topology. The spout will pick up at offset 0
> when it restarts, because it is configured to always start at the beginning
> of the partition.
> 
> UNCOMMITTED_EARLIEST and UNCOMMITTED_LATEST act exactly like the other two
> modes if the spout hasn't committed anything. If the spout has committed
> some offsets, the restarted spout will pick up where it left off, at the
> last committed offset. Example: Say the spout is set to
> UNCOMMITTED_EARLIEST and we've just deployed it for the first time. The
> spout seeks to the earliest offset because it hasn't previously committed
> anything, so it starts at offset 0 and emits offsets 0-100 and commits them
> once the tuples are acked. The spout crashes. The restarted spout will pick
> up at offset 100, because that was the last committed offset before it
> crashed.
> 
> I hope this helps.
> 
> 2017-06-28 8:40 GMT+02:00 Zhechao Ma <[email protected]>:
> 
>> The storm-kafka-client document explains these two values just almost the
>> same except the last word.
>> 
>> https://github.com/apache/storm/blob/master/docs/storm-kafka-client.md
>> 
>>   - UNCOMMITTED_EARLIEST (DEFAULT) means that the kafka spout polls
>>   records from the last committed offset, if any. If no offset has been
>>   committed, it behaves as EARLIEST.
>>   - UNCOMMITTED_LATEST means that the kafka spout polls records from the
>>   last committed offset, if any. If no offset has been committed, it
>> behaves
>>   as LATEST.
>> 
>> Or is that a mistake?
>> 
>> --
>> Thanks
>> Zhechao Ma
>> 

Reply via email to