Skip to content

Conversation

@tiodollar
Copy link

@tiodollar tiodollar commented Jun 29, 2017

When i put a message in a partition, the storm-kafka-client consume this message.
But storm-kafka-client commit the offset -1.
storm-kafka-client: 1.1.0
storm-core : 1.1.0
kafka: 0.10.2.0
Steps to bug
#1 - Insert message in kafka
#2 - Read with storm Spout this topic
#3 - Get the offset for the consumer group and the offset is always offset -1

https://issues.apache.org/jira/browse/STORM-2607

@srdo
Copy link
Contributor

srdo commented Jun 29, 2017

@tiodollar Nice find. The fix is not workable though. We need to update OffsetManager to handle that the commit offset should be the last processed offset +1, as per the Kafka docs for commitSync: "The committed offset should be the next message your application will consume, i.e. lastProcessedMessageOffset + 1".

Would you like to take a look at fixing this? I'd also be happy to take a stab at it.

@tiodollar
Copy link
Author

@srdo I'll take look, and try fixing this!

preCommitCommittedOffsets + 1, this.committedOffset, numCommittedOffsets, tp);

return numCommittedOffsets;
return numCommittedOffsets - 1;// The committed offset should be the next message
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please fix this in L144 instead, so the log messages above print the right number as well

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srdo Ok!

OffsetAndMetadata nextCommitOffsetAndMetadata = null;
if (nextCommitMsg != null) {
//The committed offset should be the next message
nextCommitOffset = nextCommitOffset + 1;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would prefer that we fix the offsets in the code block above (L76-117) instead of adjusting the offset here. That way the log messages won't be printing the wrong offset.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srdo Ok!

Rodolfo Ribeiro and others added 2 commits June 29, 2017 19:30
… instead of last committed offset for compatibility with commitSync consumer API
STORM-2607: Switch OffsetManager to track earliest uncommitted offset… by @srdo
Copy link
Contributor

@srdo srdo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@tiodollar Thanks. Tested this with an example topology, and the spout seems to commit all the messages now.
@hmcl Could I get you to review this?

nextCommitMsg.getMetadata(Thread.currentThread()));
LOG.debug("topic-partition [{}] has offsets [{}-{}] ready to be committed",
tp, committedOffset + 1, nextCommitOffsetAndMetadata.offset());
tp, earliestUncommittedOffset, nextCommitOffsetAndMetadata.offset());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Whoops, missed a -1 here. It should be nextCommitOffsetAndMetadata.offset() - 1

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srdo I corrected that!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks

@tiodollar
Copy link
Author

@hmcl Could I get you to review this?

@hmcl
Copy link
Contributor

hmcl commented Jul 6, 2017

@tiodollar ok

@srdo
Copy link
Contributor

srdo commented Jul 19, 2017

@tiodollar Seems like there's a conflict with master. Could you fix it?

@tiodollar
Copy link
Author

@srdo i'll fix this! Tks!

@HeartSaVioR
Copy link
Contributor

@tiodollar Any updates?

@srdo
Copy link
Contributor

srdo commented Aug 14, 2017

@tiodollar
Copy link
Author

@HeartSaVioR i'll fix this merge today!

@hmcl
Copy link
Contributor

hmcl commented Aug 15, 2017

@tiodollar once you do the merge, can you please squash all the commits. Thanks.

@srdo
Copy link
Contributor

srdo commented Sep 5, 2017

@tiodollar Please let me know if you're having trouble resolving the conflicts. I'd be happy to help.

@Chandan83
Copy link

Is there any reason this merge is pending? Can I help?

@HeartSaVioR
Copy link
Contributor

Anyone could take this over since this PR looks like inactive.
Please keep in mind that you are encouraged to preserve @tiodollar commits with keeping authorship. Also encouraged to squash commits into one, and add your commits if needed.

@srdo
Copy link
Contributor

srdo commented Sep 19, 2017

Yes, we should get this in soon if @tiodollar is busy. @Chandan83 if you'd like to work on this please create an account at https://issues.apache.org/jira and let us know, so we can assign the issue to you.

With regard to squashing, I think it should be possible to squash the existing commits into one for @tiodollar's commits and one for mine and preserve authorship information that way.

@Chandan83
Copy link

@srdo I have created an apache jira account with id ChandanKrSingh. Please assign the issue to me.

@tiodollar
Copy link
Author

@srdo i'll resolve this issue today! I have been busy, but i reseved some hour to do this.

@Chandan83
Copy link

@tiodollar Great. Thanks.

@srdo
Copy link
Contributor

srdo commented Oct 10, 2017

@tiodollar Since there is still a conflict here, and we'd like this to go in 1.1.2 which we hope to release before too long, I've resolved the conflicts and addressed the review comments. There's a PR with the full set of changes here #2367. Note that your commits are still present, so you should still be credited with the fix.

Let me know if this is not okay with you.

@tiodollar
Copy link
Author

@srdo tks for this! I'm very busy, and is better that! For me it's ok!

@srdo
Copy link
Contributor

srdo commented Oct 11, 2017

@tiodollar Happy to hear it :)

@tiodollar tiodollar closed this Oct 14, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants