Thank you so much for the detailed explanation, Matthias! It's really helpful! Hope you have a good weekend :)
On Thu, Jun 20, 2019 at 5:30 PM Matthias J. Sax <matth...@confluent.io> wrote: > The observed behavior is expected. > > > For example, if we send 2615 events to an empty topic, we expect the end > of > >> the topic to be offset 2616. > > This is a wrong expectation. Even if Kafka behaves that way for non-EOS, > there is no "contract" that guarantees that offsets are consecutive. > Kafka only guarantees that offsets are ascending. > > In fact, if you enable log compaction, there are also "missing offsets" > after a topic was compacted. > > For EOS, each time a transaction is committed, a commit marker is > written into the topic, and the commit marked "eats" one offset. > (Similarly, if a transaction is aborted writing an abort marker). > > >> In addition, there are "missing offsets" in the topic. There seems to be > >> nothing inside message offset 5, but it still counts as a message...I > used > >> Kafkacat to print out that message, and I have specified -Z (Print NULL > >> messages and keys as "NULL"(instead of empty)), but it didn't print out > > Those markers are internal and related to your "missing offsets". You > cannot read the markers as clients will skip over them on read. > > There is also a related issue with regard to transnational markers: > https://issues.apache.org/jira/browse/KAFKA-6607 Lag will never be > reported as zero, because of the tailing transactional markers. > > > > The number of stored messages in a partition are not exposed in Kafka > and I am not aware of a way to get this information. > > > > -Matthias > > > On 6/20/19 2:42 PM, Trinity Xia wrote: > > Hi, > > > > We are using Kafka Streams 2.2.1 and Kafka 2.2.0, and we noticed that the > > end offset number is larger than the numbers of events sent to a topic if > > we set *processing guarantee* as *exactly once* in a Kafka Streams app. > > > > For example, if we send 2615 events to an empty topic, we expect the end > of > > the topic to be offset 2616. However, if we set *processing > > guarantee* as *exactly > > once,* the end of the topic is offset 2619 instead. > > > > Key: 5-****5 offset: 0 > > Key: 5-****-5 offset: 1 > > Key: 5-****-5 offset: 2 > > Key: 5-****-5 offset: 3 > > Key: 5-****-5 offset: 4 > > Key: 25-****-f offset: 6 > > Key: 25-****-f offset: 7 > > Key: 25-****-f offset: 8 > > ..... > > > > % Reached end of topic tracking_events[0] at offset 2619: exiting > > > > In addition, there are "missing offsets" in the topic. There seems to be > > nothing inside message offset 5, but it still counts as a message...I > used > > Kafkacat to print out that message, and I have specified -Z (Print NULL > > messages and keys as "NULL"(instead of empty)), but it didn't print out > > anything either... > > > > Has anyone encountered the same issue before? Are there any other > methods > > to calculate the number of messages inside a topic efficiently? Thanks > for > > the help :) > > > > Trinity > > > > -- [image: photo] Trinity Xia Software Engineer at Scribd A 333 Bush Street | San Francisco, CA 94104 E trini...@scribd.com W www.scribd.com <https://www.scribd.com/> <http://www.facebook.com/scribd> <http://twitter.com/scribd> <http://instagram.com/scribd> <http://www.linkedin.com/company/scribd/>