Hi Matthias, Indeed for consumers it will be not obvious what −1 means: actual timestamp or no timestamp. Nevertheless, it's just −1 millisecond, so I thought it will be not a big deal to leave it (not clean, but acceptable).
I agree that it will much cleaner to have a different type of topics that support negative timestamp and/or threat Long.MIN_VALUE as a no-timestamp. I'll update KIP to make it a proposed solution. Thanks, Konstantin > On Dec 5, 2017, at 7:06 PM, Matthias J. Sax <matth...@confluent.io> wrote: > > Thanks for the KIP Konstantin. > > From my understanding, you propose to just remove the negative timestamp > check in KafkaProducer and KafkaStreams. If topics are configured with > `CreateTime` brokers also write negative timestamps if they are embedded > in the message. > > However, I am not sure about the overlapping semantics for -1 timestamp. > My concerns is, that this ambiguity might result in issues. Assume that > there is a topic (configured with `CreateTime`) for which an old and a > new producer are writing. The old producer uses old message format and > does not include any timestamp in the message. The broker will "upgrade" > this message to the new format and set -1. At the same time, the new > producer could write a message with valid timestamp -1. A consumer could > not distinguish between both cases... > > Also, there might be other Producer implementations that write negative > timestamps. Thus, those might already exist. For Streams, we don't > process those and we should make sure to keep it this way (to avoid > ambiguity). > > Thus, it might actually make sense to introduce a new timestamp type to > express those new semantics. The question is still, how to deal with > older producer clients that want to write to those topics. > > - We could either use `Long.MIN_VALUE` as "unknown" (this would be way > better than -1 as it's not in the middle of the range but at the very > end and it will also have well-defined semantics). > - Or we use a "mixed-mode" where we use broker wall-clock time for > older message formats (ie, append time semantics for older producers) > - Third, we would even give an error message back to older producers; > this might change the backward compatibility guarantees Kafka provides > so far when upgrading brokers. However, this would not affect exiting > topics, but only newly created ones (and we could disallow changing the > semantics to the new timestamp type to guard against miss > configuration). Thus, it might be ok. > > For Streams, we could check the topic config and process negative > timestamps only if the topic is configures with the new timestamp type. > > > Maybe I am a little bit to paranoid about overloading -1 semantics. > Curious to get feedback from others. > > > > -Matthias > > > On 12/5/17 1:24 PM, Konstantin Chukhlomin 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 >> >