bq. Using deltas will probably cause lots of headaches

The downside of my initial proposal for using delta is that it didn't keep
the natural ordering among negative timestamps.

Here is variation of using delta :

-1 is still kept as how it is treated today.

The delta is calculated based on:

1/1/1970 - (300 years)

Meaning, if user specifies -x (x > 0), the value stored would be:

1/1/1970 - (300 years) - x

This way there are fewer corner cases which need special attention.

Cheers

On Wed, Dec 6, 2017 at 9:36 AM, Xavier Léauté <xav...@confluent.io> wrote:

> I agree with Matthias that keeping -1 special will be prone to errors. We
> should accept this is mistake resulting from lack of foresight on our part
> when adding timestamps in the first place and correct it.
>
> Using deltas will probably cause lots of headaches. It means we have to
> figure out the implications with retention and/or delete-retention in
> compacted topics, and it makes even harder than it already is to reason
> about create-time vs. log-append-time in your application. We would also
> have to maintain a separate delta per topic, since retention can be defined
> globally.
>
> Long.MIN_VALUE seems to be the obvious choice, and probably is what we
> should have picked in the first place, short of using another bit elsewhere
> in the protocol. I would be in favor of that solution, unless this
> introduces severe implementation headaches related to up/down-conversion of
> messages or special treatment of old data.
>
> If we feel strongly about backwards compatibility with -1 timestamp, there
> is also another solution. We could decide to sacrifice the sign bit in the
> timestamp and make it special, using the next highest bit as our sign bit.
> This would make it mostly backwards compatible while still giving us
> "plenty of time". The downside is some bit-twiddling for negative
> timestamps, and breaking backwards compatibility for anyone using
> timestamps >= 1 << 62 or anyone already abusing Kafka with negative
> timestamps. Hopefully those edge-cases are rare, but it still feels a bit
> kludgy compared to using Long.MinValue.
>
> On Wed, Dec 6, 2017 at 8:14 AM Bill Bejeck <bbej...@gmail.com> wrote:
>
> > I'm getting to this a little late, but as for the missing timestamp
> > semantics, it's a +1 from me for using Long.MIN_VALUE for missing
> > timestamps for the reasons outlined by Matthias previously.
> >
> > Thanks,
> > Bill
> >
> > On Wed, Dec 6, 2017 at 2:05 AM, Dong Lin <lindon...@gmail.com> wrote:
> >
> > > Sounds good. I don't think there is concern with using Long.MIN_VALUE
> to
> > > indicate that timestamp is not available.
> > >
> > > As Matthias also mentioned, using Long.MIN_VALUE to indicate missing
> > > timestamp seems better than overloading -1 semantics. Do you want to
> > update
> > > the "NO_TIMESTAMP (−1) problem" session in the KIP? It may also be
> useful
> > > to briefly mention the alternative solution we discussed (I realized
> that
> > > Ted also mentioned this alternative).
> > >
> > > Thanks,
> > > Dong
> > >
> > > On Tue, Dec 5, 2017 at 8:26 PM, Boerge Svingen <bsvin...@borkdal.com>
> > > wrote:
> > >
> > > >
> > > > Thank you for the suggestion. We considered this before. It works,
> but
> > > > it’s a hack, and we would be providing a bad user experience for our
> > > > consumers if we had to explain, “if you want to start consuming in
> > 2014,
> > > > you have to pretend to want 2214”.
> > > >
> > > > We would rather solve the underlying problem. These are perfectly
> valid
> > > > timestamps, and I can’t see any reason why Kafka shouldn’t support
> them
> > > - I
> > > > don’t think using `Long.MIN_VALUE` instead of -1 would necessarily
> add
> > > > complexity here?
> > > >
> > > >
> > > > Thanks,
> > > > Boerge.
> > > >
> > > >
> > > >
> > > > > On 2017-12-05, at 21:36, Dong Lin <lindon...@gmail.com> wrote:
> > > > >
> > > > > Hey Boerge,
> > > > >
> > > > > Thanks for the blog link. I will read this blog later.
> > > > >
> > > > > Here is another alternative solution which may be worth thinking.
> We
> > > know
> > > > > that the Unix time 0 corresponds to January 1, 1970. Let's say the
> > > > earliest
> > > > > time you may want to use as the timestamp of the Kafka message is
> > > within
> > > > X
> > > > > milliseconds before the January 1, 1970. Then you can add X to the
> > > > > timestamp before you produce Kafka message. And you can also make
> > > similar
> > > > > conversion when you use `offsetsForTimes()` or after you consume
> > > > messages.
> > > > > This seems to address your use-case without introducing negative
> > > > timestamp.
> > > > >
> > > > > IMO, this solution requires a bit more logic in your application
> > code.
> > > > But
> > > > > it keeps the Kafka timestamp logic simple and we reserve the
> > capability
> > > > to
> > > > > use timestamp -1 for messages without timestamp for most Kafka
> users
> > > who
> > > > do
> > > > > not need negative timestamp. Do you think this would be a good
> > > > alternative
> > > > > solution?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > > >
> > > > > On Tue, Dec 5, 2017 at 5:39 PM, Boerge Svingen <
> bsvin...@borkdal.com
> > >
> > > > wrote:
> > > > >
> > > > >>
> > > > >> Yes. To provide a little more detail, we are using Kafka to store
> > > > >> everything ever published by The New York Times, and to make this
> > > > content
> > > > >> available to a range of systems and applications. Assets are
> > published
> > > > to
> > > > >> Kafka chronologically, so that consumers can seek to any point in
> > time
> > > > and
> > > > >> start consuming from there, like Konstantin is describing, all the
> > way
> > > > back
> > > > >> to our beginning in 1851.
> > > > >>
> > > > >>
> > https://www.confluent.io/blog/publishing-apache-kafka-new-york-times/
> > > <
> > > > >>
> > https://www.confluent.io/blog/publishing-apache-kafka-new-york-times/
> > > >
> > > > >> has more information on the use case.
> > > > >>
> > > > >>
> > > > >> Thanks,
> > > > >> Boerge.
> > > > >>
> > > > >>
> > > > >> --
> > > > >>
> > > > >> Boerge Svingen
> > > > >> Director of Engineering
> > > > >> The New York Times
> > > > >>
> > > > >>
> > > > >>
> > > > >>
> > > > >>> On 2017-12-05, at 19:35, Dong Lin <lindon...@gmail.com> wrote:
> > > > >>>
> > > > >>> Hey Konstantin,
> > > > >>>
> > > > >>> According to KIP-32 the timestamp is also used for log rolling
> and
> > > log
> > > > >>> retention. Therefore, unless broker is configured to never delete
> > any
> > > > >>> message based on time, messages produced with negative timestamp
> in
> > > > your
> > > > >>> use-case will be deleted by the broker anyway. Do you actually
> plan
> > > to
> > > > >> use
> > > > >>> Kafka as a persistent storage system that never delete messages?
> > > > >>>
> > > > >>> Thanks,
> > > > >>> Dong
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>>
> > > > >>> On Tue, Dec 5, 2017 at 1:24 PM, Konstantin Chukhlomin <
> > > > >> chuhlo...@gmail.com>
> > > > >>> wrote:
> > > > >>>
> > > > >>>> Hi Dong,
> > > > >>>>
> > > > >>>> Currently we are storing historical timestamp in the message.
> > > > >>>>
> > > > >>>> What we are trying to achieve is to make it possible to do Kafka
> > > > lookup
> > > > >>>> by timestamp. Ideally I would do `offsetsForTimes` to find
> > articles
> > > > >>>> published
> > > > >>>> in 1910s (if we are storing articles on the log).
> > > > >>>>
> > > > >>>> So first two suggestions aren't really covering our use-case.
> > > > >>>>
> > > > >>>> We could create a new timestamp type like "HistoricalTimestamp"
> or
> > > > >>>> "MaybeNegativeTimestamp".
> > > > >>>> And the only difference between this one and CreateTime is that
> it
> > > > could
> > > > >>>> be negative.
> > > > >>>> I tend to use CreateTime for this purpose because it's easier to
> > > > >>>> understand from
> > > > >>>> user perspective as a timestamp which publisher can set.
> > > > >>>>
> > > > >>>> Thanks,
> > > > >>>> Konstantin
> > > > >>>>
> > > > >>>>> On Dec 5, 2017, at 3:47 PM, Dong Lin <lindon...@gmail.com>
> > wrote:
> > > > >>>>>
> > > > >>>>> Hey Konstantin,
> > > > >>>>>
> > > > >>>>> Thanks for the KIP. I have a few questions below.
> > > > >>>>>
> > > > >>>>> Strictly speaking Kafka actually allows you to store historical
> > > data.
> > > > >> And
> > > > >>>>> user are free to encode arbitrary timestamp field in their
> Kafka
> > > > >> message.
> > > > >>>>> For example, your Kafka message can currently have Json or Avro
> > > > format
> > > > >>>> and
> > > > >>>>> you can put a timestamp field there. Do you think that could
> > > address
> > > > >> your
> > > > >>>>> use-case?
> > > > >>>>>
> > > > >>>>> Alternatively, KIP-82 introduced Record Header in Kafka and you
> > can
> > > > >> also
> > > > >>>>> define your customized key/value pair in the header. Do you
> think
> > > > this
> > > > >>>> can
> > > > >>>>> address your use-case?
> > > > >>>>>
> > > > >>>>> Also, currently there are two types of timestamp according to
> > > KIP-32.
> > > > >> If
> > > > >>>>> the type is LogAppendTime then the timestamp value is the time
> > when
> > > > >>>> broker
> > > > >>>>> receives the message. If the type is CreateTime then the
> > timestamp
> > > > >> value
> > > > >>>> is
> > > > >>>>> determined when producer produces message. With these two
> > > > definitions,
> > > > >>>> the
> > > > >>>>> timestamp should always be positive. We probably need a new
> type
> > > here
> > > > >> if
> > > > >>>> we
> > > > >>>>> can not put timestamp in the Record Header or the message
> > payload.
> > > > Does
> > > > >>>>> this sound reasonable?
> > > > >>>>>
> > > > >>>>> Thanks,
> > > > >>>>> Dong
> > > > >>>>>
> > > > >>>>>
> > > > >>>>>
> > > > >>>>> On Tue, Dec 5, 2017 at 8:40 AM, Konstantin Chukhlomin <
> > > > >>>> chuhlo...@gmail.com>
> > > > >>>>> wrote:
> > > > >>>>>
> > > > >>>>>> Hi all,
> > > > >>>>>>
> > > > >>>>>> I have created a KIP to support negative timestamp:
> > > > >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > >>>>>> 228+Negative+record+timestamp+support <
> > https://cwiki.apache.org/
> > > > >>>>>> confluence/display/KAFKA/KIP-228+Negative+record+timestamp+
> > > support>
> > > > >>>>>>
> > > > >>>>>> Here are proposed changes: https://github.com/apache/
> > > > >>>>>> kafka/compare/trunk...chuhlomin:trunk <
> > https://github.com/apache/
> > > > >>>>>> kafka/compare/trunk...chuhlomin:trunk>
> > > > >>>>>>
> > > > >>>>>> I'm pretty sure that not cases are covered, so comments and
> > > > >> suggestions
> > > > >>>>>> are welcome.
> > > > >>>>>>
> > > > >>>>>> Thank you,
> > > > >>>>>> Konstantin
> > > > >>>>
> > > > >>>>
> > > > >>
> > > > >>
> > > >
> > > >
> > >
> >
>

Reply via email to