Wow, thanks so much for the detailed explanation. I've verified that I can replicate your results, and adapted it for inclusion in pykafka's documentation, since this API is a constant source of confusion for users. https://github.com/Parsely/pykafka/commit/340f8e16c8b0d9830b6fd889f3be7570015ba3eb
On Thu, Apr 12, 2018 at 1:19 PM, Andras Beni <andrasb...@cloudera.com> wrote: > Hi Emmett, > > ListOffsets API tells you about the log segments belonging to (the given > partitions of) a topic. > I think I better explain how it behaves by example. > I have a topic called 'test2' with three partitions (0..2). I produced 2665 > messages to its partition 0. I set up the topic so that it rolls a new > segment after around 530 messages. > My two oldest segments were deleted, so now in my kafka/data/test2-0 > directory I have these files: > /var/local/kafka/data/test2-0/00000000000000001059.log > /var/local/kafka/data/test2-0/00000000000000002119.log > /var/local/kafka/data/test2-0/00000000000000001589.log > /var/local/kafka/data/test2-0/00000000000000002649.log > Let's say I call ListOffset API version 0 now with following parameters > Topic: test2, partition: 0, timestamp: -1 ( == latest), numberOfOffsets : > 100 > This will result in an offset list of [2665,2649,2119,1589,1059] One item > for each first offset in a file and the last offset (but not more than 100 > items) > If I set numberOfOffsets to 3, I only get 3 items: [2665,2649,2119] > Now let's say I use timestamp -2 ( == oldest) this way I only get back the > oldest offset [1059] regardless of numberOfOffsets. > Using a real timestamp (e.g. 1523559480000) will show me offsets that were > appended to the log before that time (based on last modified date of the > file). > If the timestamp means a moment before the oldest message, the returned > array will be empty. > > In version 1 and 2 request and response have changed: > If timestamp == -1, the latest offset is returned (except if isolation > level is read committed, because then the latest stable offset is returned) > if timestamp == -2, the oldest offset is returned > if timestamp is a real time value, the log is searched for the last offset > whose timestamp is smaller than given timestamp. > > I hope this helps, > Andras > > > > On Mon, Mar 26, 2018 at 6:33 PM, Emmett Butler <emm...@parsely.com> wrote: > > > Hi users, > > > > I'm the maintainer of the PyKafka <https://github.com/parsely/pykafka> > > library and I'm working on improving its support for the ListOffsets API. > > Some questions: > > > > Kafka version: 1.0.0 > > I'm using this documentation > > <https://kafka.apache.org/protocol#The_Messages_ListOffsets> for > > reference. > > > > In some cases, ListOffsets requests return an empty array of offsets. Is > > this expected behavior? If so, when? > > > > What format is the Timestamp parameter expected in? I've been using > > milliseconds since epoch (python: time.time() * 1000), but I haven't > > managed to get a response that isn't either [0] or [] using this > approach. > > Could this have to do with the number of log segments on my topic, or the > > presence of a time index? How do I make a request including a Timestamp > > (not special values -1 or -2) that returns a valid offset? What is the > > meaning of a [0] response in this context? > > > > What is the MaxNumberOfOffsets parameter supposed to do? When I request > max > > 10 offsets, I get back [12345, 0] (two offsets). Again, could this have > to > > do with the number of log segments on my topic? > > > > Related PyKafka issue tickets for reference: > > https://github.com/Parsely/pykafka/issues/728 > > https://github.com/Parsely/pykafka/issues/733 > > > > Thanks for your help. > > > > -- > > Emmett Butler | Software Engineer > > <http://www.parsely.com/?utm_source=Signature&utm_medium= > > emmett-butler&utm_campaign=Signature> > > > -- Emmett Butler | Software Engineer <http://www.parsely.com/?utm_source=Signature&utm_medium=emmett-butler&utm_campaign=Signature>