[ 
https://issues.apache.org/jira/browse/STORM-2546?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16151889#comment-16151889
 ] 

Stig Rohde Døssing edited comment on STORM-2546 at 9/4/17 2:54 PM:
-------------------------------------------------------------------

Copy pasting from the Kafka mailing list 
http://mail-archives.apache.org/mod_mbox/kafka-users/201709.mbox/%3CCA%2BOCqnYvhDTQ_dWthg68aO0JKgAENuwDyn-LgdnEyg%2BwAAgGMw%40mail.gmail.com%3E:

{quote}
I believe the Javadoc is slightly incorrect/misleading.
When it says "offset metadata is too large", it is about the metadata
you can commit along with the offset, not the offset. See
OffsetAndMetadata:
http://kafka.apache.org/0110/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html

Regarding the offset value, we only check if it's negative and that's
only performed client side (presumably 3rd party clients could commit
a negative offset). Apart from that, no checks are made if the offset
is "in range" or not.
We had a look a while back to check if the offset is "in range" when
committing but it's complicated, see the comments on
https://issues.apache.org/jira/browse/KAFKA-4081

I opened a PR to update the Javadoc: https://github.com/apache/kafka/pull/3780
{quote}

So we don't need to worry about committing offsets that are deleted.


was (Author: srdo):
Copy pasting answer by Mickael Maison from the Kafka mailing list:

{quote}
I believe the Javadoc is slightly incorrect/misleading.
When it says "offset metadata is too large", it is about the metadata
you can commit along with the offset, not the offset. See
OffsetAndMetadata:
http://kafka.apache.org/0110/javadoc/index.html?org/apache/kafka/clients/consumer/KafkaConsumer.html

Regarding the offset value, we only check if it's negative and that's
only performed client side (presumably 3rd party clients could commit
a negative offset). Apart from that, no checks are made if the offset
is "in range" or not.
We had a look a while back to check if the offset is "in range" when
committing but it's complicated, see the comments on
https://issues.apache.org/jira/browse/KAFKA-4081

I opened a PR to update the Javadoc: https://github.com/apache/kafka/pull/3780
{quote}

So we don't need to worry about committing offsets that are deleted.

> Kafka spout can stall / get stuck due to edge case with failing tuples
> ----------------------------------------------------------------------
>
>                 Key: STORM-2546
>                 URL: https://issues.apache.org/jira/browse/STORM-2546
>             Project: Apache Storm
>          Issue Type: Bug
>          Components: storm-kafka-client
>    Affects Versions: 2.0.0, 1.x
>            Reporter: Prasanna Ranganathan
>            Assignee: Stig Rohde Døssing
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> The mechanism for replaying a failed tuple involves seeking the kafka 
> consumer to the failing offset and then re-emitting it into the topology. A 
> tuple, when emitted the first time, will have an entry created in 
> OffsetManager. This entry will be removed only after the tuple is 
> successfully acknowledged and its offset successfully committed. Till then, 
> commits for offsets beyond the failing offset for that TopicPartition will be 
> blocked.
> It is possible that when the spout seeks the consumer to the failing offset, 
> the corresponding kafka message is not returned in the poll response. This 
> can happen due to that offset being deleted or compacted away. In this 
> scenario that partition will be blocked from committing and progressing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to