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
> > > >
> > >
> >
>

Reply via email to