The broker still accepts that version and the Scala API still includes
support for that timestamp. Note that the way this worked in previous
versions was by looking only at the timestamp for each log segment instead
of using a timestamp index within the segment.

Note that the new consumer now includes an offsetsForTimes method. Using
that would be the recommended path forward, especially if you are writing
new code and are working with recent enough versions of Kafka.

-Ewen

On Thu, Jan 19, 2017 at 10:02 AM, Vignesh <vignesh.v...@gmail.com> wrote:

> Another question, with getOffsetsBefore, we used to be able to get offsets
> for time in older versions.
> .10 doesn't have an equivalent method.
>
> Is there any other way to achieve the same functionality as
> getOffsetsBefore
> in .10 ? Does a .10 server respond to ListOffsetRequestV0 request?
>
>
> On Fri, Jan 6, 2017 at 1:26 PM, Ewen Cheslack-Postava <e...@confluent.io>
> wrote:
>
> > It would return the earlier one, offset 0.
> >
> > -Ewen
> >
> > On Thu, Jan 5, 2017 at 10:15 PM, Vignesh <vignesh.v...@gmail.com> wrote:
> >
> > > Thanks. I didn't realize ListOffsetRequestV1 is only available 0.10.1
> > > (which has KIP-33, time index).
> > > When timestamp is set by user (CreationTime), and it is not always
> > > increasing, would this method still return the offset of first message
> > with
> > > timestamp greater than equal to the provided timestamp?
> > >
> > >
> > > For example, in below scenario
> > >
> > > Message1, Timestamp = T1, Offset = 0
> > > Message2, Timestamp = T0 (or T2), Offset = 1
> > > Message3, Timestamp = T1, Offset = 2
> > >
> > >
> > > Would offsetForTimestamp(T1) return offset for earliest message with
> > > timestamp T1 (i.e. Offset 0 in above example) ?
> > >
> > >
> > > -Vignesh.
> > >
> > > On Thu, Jan 5, 2017 at 8:19 PM, Ewen Cheslack-Postava <
> e...@confluent.io
> > >
> > > wrote:
> > >
> > > > On Wed, Jan 4, 2017 at 11:54 PM, Vignesh <vignesh.v...@gmail.com>
> > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > offsetsForTimes
> > > > > <https://kafka.apache.org/0101/javadoc/org/apache/kafka/
> > > > clients/consumer/
> > > > > KafkaConsumer.html#offsetsForTimes(java.util.Map)>
> > > > > function
> > > > > returns offset for a given timestamp. Does it use message's
> timestamp
> > > > > (which could be LogAppendTime or set by user) or creation time of
> > > > > logsegment file?
> > > > >
> > > > >
> > > > This is actually tied to how the ListOffsetsRequest is handled. But
> if
> > > > you're on a recent version, then the KIP
> > > > https://cwiki.apache.org/confluence/pages/viewpage.
> > > action?pageId=65868090
> > > > made it use the more accurate version based on message timestamps.
> > > >
> > > >
> > > > >
> > > > > KIP-33
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > 33+-+Add+a+time+based+log+index>
> > > > > adds timestamp based index, and it is available only from 0.10.1 .
> > Does
> > > > >  above function work on 0.10.0 ? If so, are there any differences
> in
> > > how
> > > > it
> > > > > works between versions 0.10.0 and 0.10.1 ?
> > > > >
> > > > >
> > > > The KIP was only adopted and implemented in 0.10.1+. It is not
> > available
> > > in
> > > > 0.10.0.
> > > >
> > > >
> > > > > Thanks,
> > > > > Vignesh.
> > > > >
> > > >
> > >
> >
>

Reply via email to