Hi, Jiangjie, Thanks for the updated KIP. A couple of minor comments.
1. latestOffsets() returns the next offset. So there won't be a timestamp associated with it. Would we use something like -1 for timestamp? 2. Jason mentioned that if no message has timestamp >= the provided timestamp, we return a null value for that partition. Could we document that in the wiki? BTW, we are getting close to the next release. This is a really nice feature to have. Do you think you will have a patch ready for the next release? Thanks. Jun On Wed, Sep 7, 2016 at 2:47 PM, Becket Qin <becket....@gmail.com> wrote: > That sounds reasonable to me. I'll update the KIP wiki page. > > On Wed, Sep 7, 2016 at 1:34 PM, Jason Gustafson <ja...@confluent.io> > wrote: > > > Hey Becket, > > > > I don't have a super strong preference, but I think this > > > > earliestOffset(singleton(partition)); > > > > captures the intent more clearly than this: > > > > offsetsForTimes(singletonMap(partition, -1)); > > > > I can understand the desire to keep the API footprint small, but I think > > the use case is common enough to justify separate APIs. A couple > additional > > points: > > > > 1. If we had separate methods, it might make sense to treat negative > > timestamps as illegal in offsetsForTimes. That seems safer from the user > > perspective since legitimate timestamps should always be positive. > > 2. The expected behavior of offsetsForTimes is to return the earliest > > offset which is greater than or equal to the passed offset, so having > > Long.MAX_VALUE return the latest value doesn't seem very intuitive to > me. I > > would actually expect it to return null. > > > > Given that, I think I prefer having the custom methods. What do you > think? > > > > Thanks, > > Jason > > > > On Wed, Sep 7, 2016 at 1:00 PM, Becket Qin <becket....@gmail.com> wrote: > > > > > Hi Jason, > > > > > > Thanks for the feedback. That is a good point. For the -1 and -2 > > semantics, > > > I was just thinking we will preserve the semantics in the wire > protocol. > > > For the user facing API, I agree that is not intuitive. We can do one > of > > > the following: > > > 1. Add two separate methods: earliestOffsets() and latestOffsets(). > > > 2. just have offsetsForTimes() and return the earliest if the timestamp > > is > > > negative and the latest if the timestamp is Long.MAX_VALUE. > > > > > > The good thing about doing (1) is that we kind of have symmetric > function > > > signatures like seekToBeginning() and seekToEnd(). However, even if we > do > > > (1), we may still need to do (2) to handle the negative timestamp and > the > > > Long.MAX_VALUE timestamp in offsetsForTimes(). Then they essentially > > become > > > redundant to earliestOffsets() and latestOffsets(). > > > > > > Personally I prefer option (2) because of the conciseness and it seems > > > intuitive enough. But I am open to option (1) as well. > > > > > > Thanks, > > > > > > Jiangjie (Becket) Qin > > > > > > On Wed, Sep 7, 2016 at 11:06 AM, Jason Gustafson <ja...@confluent.io> > > > wrote: > > > > > > > Hey Becket, > > > > > > > > Thanks for the KIP. As I understand, the intention is to preserve the > > > > current behavior with a timestamp of -1 indicating latest timestamp > and > > > -2 > > > > indicating earliest timestamp. So users can query these offsets using > > the > > > > offsetsForTimes API if they know the magic values. I'm wondering if > it > > > > would make the usage a little nicer to have a separate API instead > for > > > > these special cases? Sort of in the way that we expose a generic > seek() > > > and > > > > a seekToBeginning(), maybe we could have an earliestOffset() in > > addition > > > to > > > > offsetsForTimes()? > > > > > > > > Thanks, > > > > Jason > > > > > > > > On Wed, Sep 7, 2016 at 10:04 AM, Becket Qin <becket....@gmail.com> > > > wrote: > > > > > > > > > Thanks everyone for all the feedback. > > > > > > > > > > If there is no further concerns or comments I will start a voting > > > thread > > > > on > > > > > this KIP tomorrow. > > > > > > > > > > Thanks, > > > > > > > > > > Jiangjie (Becket) Qin > > > > > > > > > > On Tue, Sep 6, 2016 at 9:48 AM, Becket Qin <becket....@gmail.com> > > > wrote: > > > > > > > > > > > Hi Magnus, > > > > > > > > > > > > Thanks for the comments. I agree that querying messages within a > > time > > > > > > range is a valid use case (actually this is an example use case > in > > my > > > > > > previous email). The current proposal can achieve this by having > > two > > > > > > ListOffsetRequest, right? I think the current API already > supports > > > the > > > > > use > > > > > > cases that require the offsets for multiple timestamps. The > > question > > > is > > > > > > that whether it is worth adding more complexity to the protocol > to > > > make > > > > > it > > > > > > easier for multiple timestamp query. Personally I think given > that > > > > query > > > > > > multiple timestamps is likely an infrequent operation, there is > no > > > need > > > > > to > > > > > > optimize for it and complicates the protocol. > > > > > > > > > > > > Thanks, > > > > > > > > > > > > Jiangjie (Becket) Qin > > > > > > > > > > > > On Mon, Sep 5, 2016 at 11:21 PM, Magnus Edenhill < > > mag...@edenhill.se > > > > > > > > > > wrote: > > > > > > > > > > > >> Good write-up Qin, the API looks promising. > > > > > >> > > > > > >> I have one comment: > > > > > >> > > > > > >> 2016-09-03 5:20 GMT+02:00 Becket Qin <becket....@gmail.com>: > > > > > >> > > > > > >> > The currently offsetsForTimes() API obviously does not support > > > > > querying > > > > > >> > multiple timestamps for the same partition. It doesn't seems a > > > > feature > > > > > >> for > > > > > >> > ListOffsetRequest v0 either (sounds more like a bug). My > > intuition > > > > is > > > > > >> that > > > > > >> > it's a rare use case. Given it does not exist before and we > > don't > > > > see > > > > > a > > > > > >> > strong need from the community either, maybe it is better to > > keep > > > it > > > > > >> simple > > > > > >> > for ListOffsetRequest v1. We can add it later if it turns out > to > > > be > > > > a > > > > > >> > useful feature (that may need a interface change, but I > honestly > > > do > > > > > not > > > > > >> > think people would frequently query many different timestamps > > for > > > > the > > > > > >> same > > > > > >> > partition) > > > > > >> > > > > > > >> > > > > > >> I argue that the current behaviour of OffsetRequest with regards > > to > > > > > >> duplicate partitions is a bug > > > > > >> and think it would be a mistake to move the same semantics over > to > > > > thew > > > > > >> new > > > > > >> ListOffset API. > > > > > >> One use case is that an application may want to know the offset > > > range > > > > > >> between two timestamps, > > > > > >> e.g., for reprocessing, batching, searching, etc. > > > > > >> > > > > > >> > > > > > >> Thanks, > > > > > >> Magnus > > > > > >> > > > > > >> > > > > > >> > > > > > >> > > > > > > >> > Have a good long weekend! > > > > > >> > > > > > > >> > Thanks, > > > > > >> > > > > > > >> > Jiangjie (Becket) Qin > > > > > >> > > > > > > >> > > > > > > >> > > > > > > >> > > > > > > >> > On Fri, Sep 2, 2016 at 6:10 PM, Ismael Juma < > ism...@juma.me.uk> > > > > > wrote: > > > > > >> > > > > > > >> > > Thanks for the proposal Becket. Looks good overall, a few > > > > comments: > > > > > >> > > > > > > > >> > > ListOffsetResponse => [TopicName [PartitionOffsets]] > > > > > >> > > > PartitionOffsets => Partition ErrorCode Timestamp > [Offset] > > > > > >> > > > Partition => int32 > > > > > >> > > > ErrorCode => int16 > > > > > >> > > > Timestamp => int64 > > > > > >> > > > Offset => int > > > > > >> > > > > > > > >> > > > > > > > >> > > It should be int64 for `Offset` right? > > > > > >> > > > > > > > >> > > Implementation wise, we will migrate to > o.a.k.common.requests. > > > > > >> > > ListOffsetRequest > > > > > >> > > > class on the broker side. > > > > > >> > > > > > > > >> > > > > > > > >> > > Could you clarify what you mean here? We already > > > > > >> > > use o.a.k.common.requests.ListOffsetRequest in KafkaApis. > > > > > >> > > > > > > > >> > > long offset = consumer.offsetForTime( > > Collections.singletonMap( > > > > > >> > > topicPartition, > > > > > >> > > > targetTime)).offset; > > > > > >> > > > > > > > >> > > > > > > > >> > > The result of `offsetForTime` is a Map, so we can't just > call > > > > > >> `offset` on > > > > > >> > > it. You probably meant something like: > > > > > >> > > > > > > > >> > > long offset = consumer.offsetForTime( > > Collections.singletonMap( > > > > > >> > > topicPartition, > > > > > >> > > targetTime)).get(topicPartition).offset; > > > > > >> > > > > > > > >> > > Test searchByTimestamp with CreateTime and LogAppendTime > > > > > >> > > > > > > > > >> > > > > > > > >> > > Do you mean `Test offsetForTime`? > > > > > >> > > > > > > > >> > > And: > > > > > >> > > > > > > > >> > > 1. In KAFKA-1588, the following issue was described "When > > > > performing > > > > > >> an > > > > > >> > > OffsetRequest, if you request the same topic and partition > > > > > combination > > > > > >> > in a > > > > > >> > > single request more than once (for example, if you want to > get > > > > both > > > > > >> the > > > > > >> > > head and tail offsets for a partition in the same request), > > you > > > > will > > > > > >> get > > > > > >> > a > > > > > >> > > response for both, but they will be the same offset". Will > the > > > new > > > > > >> > request > > > > > >> > > version support the use case where multiple timestamps are > > > passed > > > > > for > > > > > >> the > > > > > >> > > same topic partition? And if we do support it at the > protocol > > > > level, > > > > > >> do > > > > > >> > we > > > > > >> > > also want to support it at the API level or do we think the > > > > > additional > > > > > >> > > complexity is not worth it? > > > > > >> > > > > > > > >> > > 2. Is `offsetForTime` the right method name given that we > are > > > > > getting > > > > > >> > > multiple offsets? Maybe it should be `offsetsForTimes` or > > > > something > > > > > >> like > > > > > >> > > that. > > > > > >> > > > > > > > >> > > Ismael > > > > > >> > > > > > > > >> > > On Wed, Aug 31, 2016 at 4:38 AM, Becket Qin < > > > becket....@gmail.com > > > > > > > > > > >> > wrote: > > > > > >> > > > > > > > >> > > > Hi Kafka devs, > > > > > >> > > > > > > > > >> > > > I created KIP-79 to allow consumer to precisely query the > > > > offsets > > > > > >> based > > > > > >> > > on > > > > > >> > > > timestamp. > > > > > >> > > > > > > > > >> > > > In short we propose to : > > > > > >> > > > 1. add a ListOffsetRequest/ListOffsetResponse v1, and > > > > > >> > > > 2. add an offsetForTime() method in new consumer. > > > > > >> > > > > > > > > >> > > > The KIP wiki is the following: > > > > > >> > > > https://cwiki.apache.org/confluence/pages/viewpage. > > > > > >> > > action?pageId=65868090 > > > > > >> > > > > > > > > >> > > > Comments are welcome. > > > > > >> > > > > > > > > >> > > > Thanks, > > > > > >> > > > > > > > > >> > > > Jiangjie (Becket) Qin > > > > > >> > > > > > > > > >> > > > > > > > >> > > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > > > >