MINOR: Update Consumer and Producer JavaDocs for committing offsets#18336
MINOR: Update Consumer and Producer JavaDocs for committing offsets#18336mjsax merged 4 commits intoapache:trunkfrom
Conversation
The consumer/producer JavaDocs still contain instruction for naively computing the offset to be committed. This PR updates the JavaDocs with regard to the improvements of KIP-1094.
AndrewJSchofield
left a comment
There was a problem hiding this comment.
Thanks for the PR. A few small comments, but looks like a good improvement.
| * which is at position 5 has consumed records with offsets 0 through 4 and will next receive the record with offset 5. There | ||
| * are actually two notions of position relevant to the user of the consumer: | ||
| * which is at position 5 has consumed records with offsets 0 through 4 and will next receive the record with offset 5. | ||
| * Note that offsets are not guaranteed to be consecutive (eg., for compacted topic, or—independent of "read_committed" |
There was a problem hiding this comment.
I suggest the following the parentheses "(such as compacted topic or when records have been produced using transactions)".
| * which is at position 5 has consumed records with offsets 0 through 4 and will next receive the record with offset 5. | ||
| * Note that offsets are not guaranteed to be consecutive (eg., for compacted topic, or—independent of "read_committed" | ||
| * mode— transactional topics). For example, if the consumer did read a record with offset 4, but 5 is not an offset | ||
| * with a record, it's position might advance to 6 (or higher) directly. Similarly, if the consumer's position is 5, |
| * should not be used. The committed offset should be the next message your application will consume, | ||
| * i.e. lastProcessedMessageOffset + 1. If automatic group management with {@link #subscribe(Collection)} is used, | ||
| * i.e. {@code nextRecordToBeProcessed.offset()} (or {@link ConsumerRecords#nextOffsets()}). | ||
| * You should also add the {@link ConsumerRecord#leaderEpoch()} (or {@code nextOffsets().get(...).leaderEpoch()}) |
There was a problem hiding this comment.
Maybe "You should also add the leader epoch as commit metadata, which can be obtained from {@link ConsumerRecord#leaderEpoch()} or {@link ConsumerRecords#nextOffsets}." I didn't find the nextOffsets().get(...).leaderEpoch() that easy to follow and the hyperlink to the ConsumerRecords seems nicer to me.
| * should not be used. The committed offset should be the next message your application will consume, | ||
| * i.e. lastProcessedMessageOffset + 1. If automatic group management with {@link #subscribe(Collection)} is used, | ||
| * i.e. {@code nextRecordToBeProcessed.offset()} (or {@link ConsumerRecords#nextOffsets()}). | ||
| * You should also add the {@link ConsumerRecord#leaderEpoch()} (or {@code nextOffsets().get(...).leaderEpoch()}) |
There was a problem hiding this comment.
Same point as above about nextOffsets().get(...).leaderEpoch().
| * should not be used. The committed offset should be the next message your application will consume, | ||
| * i.e. lastProcessedMessageOffset + 1. If automatic group management with {@link #subscribe(Collection)} is used, | ||
| * i.e. {@code nextRecordToBeProcessed.offset()} (or {@link ConsumerRecords#nextOffsets()}). | ||
| * You should also add the {@link ConsumerRecord#leaderEpoch()} (or {@code nextOffsets().get(...).leaderEpoch()}) |
| * be the next message your application will consume, i.e. lastProcessedMessageOffset + 1. | ||
| * be the next message your application will consume, i.e. {@code nextRecordToBeProcessed.offset()} | ||
| * (or {@link ConsumerRecords#nextOffsets()}). You should also add the {@link ConsumerRecord#leaderEpoch()} | ||
| * (or {@code nextOffsets().get(...).leaderEpoch()}) as commit metadata. |
| * } | ||
| * long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); | ||
| * consumer.commitSync(Collections.singletonMap(partition, new OffsetAndMetadata(lastOffset + 1))); | ||
| * consumer.commitSync(Collections.singletonMap(partition, partitionRecords.nextOffsets().get(partition)); |
There was a problem hiding this comment.
typo: partitionRecords -> records
There was a problem hiding this comment.
and missing parenthesis, I guess we should end with :
| * consumer.commitSync(Collections.singletonMap(partition, partitionRecords.nextOffsets().get(partition)); | |
| * consumer.commitSync(Collections.singletonMap(partition, records.nextOffsets().get(partition))); |
There was a problem hiding this comment.
partitionRecords is actually correct; nextRecords() was added to ConsumerRecord[s] (not ConsumerRecord) -- that's also why we need to call get(partition) -- ConsumerRecord is already data for a single partition, and get(partition) would not make sense.
There was a problem hiding this comment.
@mjsax Could you please take a look at following attachment? the type of partitionRecords is List<ConsumerRecord<String, String>> rather than ConsumerRecords
btw, it miss a ) also.
There was a problem hiding this comment.
Thanks for sanity checking... I did mix up the variable names.
lianetm
left a comment
There was a problem hiding this comment.
Thanks for the nice update! Just couple of comments (and agree with the previous suggestions)
| * mode— transactional topics). For example, if the consumer did read a record with offset 4, but 5 is not an offset | ||
| * with a record, it's position might advance to 6 (or higher) directly. Similarly, if the consumer's position is 5, | ||
| * but there is no record with offset 5, the consumer will return the record with the next higher offset. | ||
| * There are actually two notions of position relevant to the user of the consumer: |
There was a problem hiding this comment.
Related to this but on ln 84 right below (sorry couldn't add comment there):
The {@link #commitSync() committed position} is the last offset that has been stored securely
shouldn't that refer to committed(..) instead?
The {@link #committed(Set) committed position} is the last offset that has been stored securely
There was a problem hiding this comment.
Not sure... I did not modify this part... There are actually two notions of position relevant to the user of the consumer: did not change.
But I think it ok as is? Guess it works both ways... It's just a difference between "write path" vs "read path", right?
Let me know what you think. If you think using #commited(Set) is better, happy to update it, but it's orthogonal to what I want to do in this PR and would be some side improvement.
| * } | ||
| * long lastOffset = partitionRecords.get(partitionRecords.size() - 1).offset(); | ||
| * consumer.commitSync(Collections.singletonMap(partition, new OffsetAndMetadata(lastOffset + 1))); | ||
| * consumer.commitSync(Collections.singletonMap(partition, partitionRecords.nextOffsets().get(partition)); |
There was a problem hiding this comment.
and missing parenthesis, I guess we should end with :
| * consumer.commitSync(Collections.singletonMap(partition, partitionRecords.nextOffsets().get(partition)); | |
| * consumer.commitSync(Collections.singletonMap(partition, records.nextOffsets().get(partition))); |
|
Thanks for all the input. Pushed an updated. |
AndrewJSchofield
left a comment
There was a problem hiding this comment.
Just one final comment.
| * Thus, when calling {@link #commitSync(Map) commitSync(offsets)} you should add one to the offset of the last message processed. | ||
| * Thus, when calling {@link #commitSync(Map) commitSync(offsets)} you should use {@code nextRecordToBeProcessed.offset()} | ||
| * or if {@link ConsumerRecords} is exhausted already {@link ConsumerRecords#nextOffsets()} instead. | ||
| * You should also pass in the {@link ConsumerRecord#leaderEpoch()} (or {@code nextOffsets().get(...).leaderEpoch()}) |
There was a problem hiding this comment.
In the other similar places, you use the You should also add the leader epoch as commit metadata, which can be obtained from {@link ConsumerRecord#leaderEpoch()} or {@link ConsumerRecords#nextOffsets()}.. I think consistency would be good here.
AndrewJSchofield
left a comment
There was a problem hiding this comment.
Thanks for the PR. Looks good to me.
…18336) The consumer/producer JavaDocs still contain instruction for naively computing the offset to be committed. This PR updates the JavaDocs with regard to the improvements of KIP-1094. Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet Magrans <lmagrans@confluent.io>
|
Merged to |
…og-compaction-write-record-v2 * apache-github/trunk: (34 commits) MINOR: Bump year to 2025 in NOTICE file (apache#18427) KAFKA-18411 Remove ZkProducerIdManager (apache#18413) KAFKA-18408 tweak the 'tag' field for BrokerHeartbeatRequest.json, BrokerRegistrationChangeRecord.json and RegisterBrokerRecord.json (apache#18421) KAFKA-18414 Remove KRaftRegistrationResult (apache#18401) KAFKA-17921 Support SASL_PLAINTEXT protocol with java.security.auth.login.config (apache#17671) KAFKA-18384 Remove ZkAlterPartitionManager (apache#18364) KAFKA-10790: Add deadlock detection to producer#flush (apache#17946) KAFKA-18412: Remove EmbeddedZookeeper (apache#18399) MINOR : Improve Exception log in NotEnoughReplicasException(apache#12394) MINOR: Improve PlaintextAdminIntegrationTest#testConsumerGroups (apache#18409) MINOR: Remove unused local variable (apache#18410) MINOR: Remove RaftManager.maybeDeleteMetadataLogDir and AutoTopicCreationManagerTest.scala (apache#17365) KAFKA-18368 Remove TestUtils#MockZkConnect and remove zkConnect from TestUtils#createBrokerConfig (apache#18352) MINOR: Update Consumer group timeout default to 30 sec (apache#16406) MINOR: Fix typo in CommitRequestManager (apache#18407) MINOR: cleanup JavaDocs for deprecation warnings (apache#18402) KAFKA-18303; Update ShareCoordinator to use new record format (apache#18396) MINOR: Update Consumer and Producer JavaDocs for committing offsets (apache#18336) KAFKA-16446: Improve controller event duration logging (apache#15622) KAFKA-18388 test-kraft-server-start.sh should use log4j2.yaml (apache#18370) ...
…pache#18336) The consumer/producer JavaDocs still contain instruction for naively computing the offset to be committed. This PR updates the JavaDocs with regard to the improvements of KIP-1094. Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet Magrans <lmagrans@confluent.io>

The consumer/producer JavaDocs still contain instruction for naively computing the offset to be committed.
This PR updates the JavaDocs with regard to the improvements of KIP-1094.