I think we should just keep -1 as a special value, and allow negative 
timestamps.  It just means one missing millisecond in 1969, right?

There is just a lot of code everywhere checking for -1, and changing it now 
would be really likely to be buggy.

We probably also want to limit the range of timestamps as well, so that we can 
calculate timestamp1 - timestamp2 without overflowing a 64-bit long.  If we cap 
timestamps to be in the range (-2**62,  2**62), we can still represent dates 
about 146 million years into the future or past.

best,
Colin


On Tue, Jan 16, 2018, at 13:03, Konstantin Chukhlomin wrote:
> Hi all,
> 
> I tried to summarize below all approaches we were discussing.
> 
> In case there are some difficulties in email formatting, see GoogleDocs:
> https://docs.google.com/document/d/1RjlcebpigOj9DyLCedyRxki9nZcFdkBchy-k7BiThMc/edit?usp=sharing
>  
> <https://docs.google.com/document/d/1RjlcebpigOj9DyLCedyRxki9nZcFdkBchy-k7BiThMc/edit?usp=sharing>
> 
> Personally, I find options A01 and A02/B03 very convenient.
> 
> Please, let me know what you think and feel free to add more options.
> 
> Thanks,
> Konstantin
> 
> –––––––––––––––––––––––––––––––––––––
> 
> Preconditions:
> we would need to remote negative timestamp check from producer anyway
> no existing data should have -1 timestamp unless it is NO_TIMESTAMP
> older client are not embedding a timestamp: default to -1 on the broker 
> side...
> APPEND_TIME would still work for older producers
> 
> Can we still use −1 as NO_TIMESTAMP value?
> Yes: A01 Keep using −1ms as a NO_TIMESTAMP value
> No, A02 We want to change the semantics of −1ms as an actual timestamp.
> For example, use Long.MIN_VALUE as "NO_TIMESTAMP".
> How do we indicate new timestamp format?
> A02 In the message itself. That way one topic may contain messages of 
> different types.
> How?
> B01 Create new timestamp type and use one of the reserved bits
> B02 Use one reserved bit in the message as a flag
> B03 Introduce new message format version
> By the topic property: `allow.negative.timestamp`.
> That way messages with CreateTime could have negative value.
> "false" by default for existing topics.
> Should `allow.negative.timestamp` be enabled by default for new topics?
> Yes. C01: We want it to be future standard
> No. C02: We want it to be client specific case by case decision.
> Do we allow changing that property for existing topics?
> Yes
> Yes, but only if existing topic have no records with negative timestamps 
> already
> No: old behavior and new behavior are not mixed
> Can old producers write to topics with that flag enabled?
> Yes, as long as we can differentiate old and new messages: B01-03
> No, broker will write old "NO_TIMESTAMP" as a −1ms
> No, broker will throw an error.
> By the broker config: `allow.negative.timestamp`.
> What to do with old topics?
> D01: Convert all messages in all topics to the new format (B01-03) 
> during the update phase.
> D02: Keep old messages, but write new messages in the new format.
> D03: Keep old messages, but new topics will have messages with new 
> format.
> A01 Keep using −1ms as a NO_TIMESTAMP value
> Cost of change: minimal
> 
> Pros
> Cons
> Realistically, the scope of impact for that -1 is pretty tiny, as has 
> been pointed out. A single millisecond gap in available timestamps in 
> 1969.
> Keeping -1 as "unknown" leaving a time gap gives a lot of headaches
> 
> Streams drops records with timestamp -1 but processes other records with 
> negative timestamps which may cause a number of question on the mailing 
> list and will confuse users that don't pay attention and "loose" data. 
> A02 Indicate that message may have negative timestamp in the message 
> itself
> Cost of change: moderate
> A03 Add topic config that allows messages with negative timestamp
> Cost of change: high
> 
> Kafka Streams could check the topic config and only drop negative 
> timestamps is they are not enabled
> 
> Pros
> Cons
> Relaxes compatibility concerns
> We already have way too many configs
> 
> Introducing the new config seems like it has significant compatibility 
> concerns that need to be sorted out. 
> A04 Add broker config that allows messages with negative timestamp
> Cost of change: high
> –
> B01 Add new timestamp type that can be negative
> Cost of change: moderate
> Pros
> Cons
> 
> Changing message format might not be worth it
> B02 Use one of reserved bit to indicate negative timestamp (aka flag)
> Cost of change: moderate
> Pros
> Cons
> 
> Using a flag bit might not be worth it
> B03 Introduce new message format version
> Cost of change: low
> 
> Old producers will still be able to write messages in old version
> Pros
> Cons
> Clear semantics: in new version CreateTime can be negative.
> Using a flag bit might not be worth it
> 
> > On Jan 2, 2018, at 11:26 PM, Ewen Cheslack-Postava <e...@confluent.io> 
> > wrote:
> > 
> > On Tue, Jan 2, 2018 at 8:04 PM, Matthias J. Sax <matth...@confluent.io>
> > wrote:
> > 
> >> I was thinking about a broker/topic config.
> >> 
> >> However, I am not sure if we only need to worry about data written in
> >> the future (this would only be true, if there would be no records with
> >> -1 timestamp already). Assume that we you an existing topic that
> >> contains data with -1 = UNKNOWN records -- for this case, we would give
> >> those timestamps a new semantics if we suddenly allow negative
> >> timestamps. (Assuming that we don't allow -1 as a gap in the timeline
> >> what I would rather not do.)
> >> 
> > 
> > Using the Java producer you cannot have a negative timestamp today. So
> > (modulo comment about being dependent on the client implementation), no
> > existing data should have -1 timestamp unless it is NO_TIMESTAMP.
> > 
> > When you say you'd rather not like to have -1 as a gap in the timeline, can
> > you explain the potential scale of impact? I view it as a relatively
> > unlikely value and something that people who are really concerned with
> > negative timestamps can easily work around. Probably many users won't care
> > as they will not be using pre-1970s data where they actually set the Kafka
> > timestamp (rather than having timestamps embedded in the data) anyway. I
> > agree it isn't ideal, but to me it looks like a reasonable tradeoff. What
> > are the effects/use cases that make you concerned that we'd see significant
> > user pain as a result?
> > 
> > 
> >> 
> >> Also note, that it's not really client specific IMHO, as one could
> >> implement their own clients. There are many third party clients and we
> >> don't know if they check for negative timestamps (applications could
> >> even assign their own special meaning to negative timestamps as those
> >> are unused atm) -- furthermore, all older client not embedding a
> >> timestamp default to -1 on the broker side...
> >> 
> > 
> > I said "client-specific" because some of the checks are done on the
> > client-side, which means they are dependent on the specific client
> > implementation being used. Based on the rest of your comment, I think we're
> > in agreement except for how we are naming things :) I'd have to double
> > check if the same level of enforcement is done broker-side. I only mention
> > that because we tend to discuss these proposals in the context of only the
> > Java clients, but it is worth thinking through the impact to other clients
> > as well.
> > 
> > 
> >> 
> >>> The implementation could easily be made to map
> >>> those values into a range that is less likely to be utilized (e.g. use
> >> the
> >>> values near Long.MIN_VALUE and have the consumer convert back as needed).
> >>> The sentinel for NO_TIMESTAMP could be changed between versions as long
> >> as
> >>> it is handled consistently between client versions.
> >> 
> >> This opens Pandora's box IMHO.
> >> 
> > 
> > Why? There should be a small number of values that need to be mapped and
> > someone could think through the different compatibility issues that are
> > possible to determine if there are any significant issues/drawbacks.
> > 
> > 
> >> 
> >>> Introducing the new
> >>> config seems like it has significant compatibility concerns that need to
> >> be
> >>> sorted out.
> >> 
> >> I cannot follow here -- from my point of view, it relaxes compatibility
> >> concerns. If we only allow new topic to enable negative timestamps, old
> >> behavior and new behavior are not mixed. IMHO, mixing both would be a
> >> real issue. Thus, for new topics we can change "unknown" from -1 to
> >> Long.MIN_VALUE and don't mix two different approaches within a single
> >> topic.
> >> 
> > 
> > What's the mechanism for this? Is the new config only allowed in
> > CreateTopics requests? If you use existing tooling to set topic configs,
> > you would just be able to set any valid config. Are the semantics just
> > undefined if you do? Unless it is impossible to do certain things, we have
> > to deal with the compatibility concerns regardless of intended use. Might
> > be fine to just say the behavior is undefined, but there's still work to be
> > done there. Regardless, I didn't (and probably still don't) have a concrete
> > understanding of the proposed setting, so hard for me to reason about it.
> > 
> > -Ewen
> > 
> > 
> >> 
> >> I see your point that we do have too many configs -- we could also make
> >> it a new value for existing `message.timestamp.type`.
> >> 
> >> 
> >> -Matthias
> >> 
> >> 
> >> On 1/2/18 7:48 PM, Ewen Cheslack-Postava wrote:
> >>> For `allow.negative.timestamps`, do you mean this as a broker config? I'm
> >>> not entirely clear on what the proposal would entail.
> >>> 
> >>> I think taking into account whether we're talking about compatibility
> >> with
> >>> existing data in Kafka vs enabling use of negative timestamps is
> >> important
> >>> here. If they're effectively not supported today (though admittedly this
> >> is
> >>> really client-specific), then we need only concern ourselves with data
> >> that
> >>> hasn't been produced into Kafka yet. In that case, we can always handle
> >>> sentinel values in special ways if we really want to. For example, the
> >> Java
> >>> producer does not accept any values < 0 and the API supports passing null
> >>> rather than the sentinels. The implementation could easily be made to map
> >>> those values into a range that is less likely to be utilized (e.g. use
> >> the
> >>> values near Long.MIN_VALUE and have the consumer convert back as needed).
> >>> The sentinel for NO_TIMESTAMP could be changed between versions as long
> >> as
> >>> it is handled consistently between client versions.
> >>> 
> >>> IMO we already have way too many configs, so we should think about where
> >>> the impact is and if a not ideal, but also not significant compromise can
> >>> be made and avoid most of the additional complexity. Introducing the new
> >>> config seems like it has significant compatibility concerns that need to
> >> be
> >>> sorted out. In contrast, I suspect the use cases we need to support that
> >>> have come up so far can handle 1 or 2 special cases and the necessary
> >>> munging could be handled safely by interceptors such that it is trivial
> >> to
> >>> make sure all your apps do the right thing. I appreciate the pain of a
> >> ton
> >>> of mailing list questions about an issue like this, but given the
> >>> likelihood of encountering that particular value, I just find it unlikely
> >>> it would be that common and I think it's a reasonable tradeoff to tell a
> >>> user they might need to handle that one special case.
> >>> 
> >>> -Ewen
> >>> 
> >>> On Thu, Dec 28, 2017 at 12:58 PM, Matthias J. Sax <matth...@confluent.io
> >>> 
> >>> wrote:
> >>> 
> >>>> I agree that changing message format or using a flag bit might not be
> >>>> worth it.
> >>>> 
> >>>> However, just keeping -1 as "unknown" leaving a time gap give me a lot
> >>>> of headache, too. Your arguments about "not an issue in practice" kinda
> >>>> make sense to me, but I see the number of question on the mailing list
> >>>> already if we really follow this path... It will confuse users that
> >>>> don't pay attention and "loose" data if Kafka Streams drops records with
> >>>> timestamp -1 but processes other records with negative timestamps.
> >>>> 
> >>>> Thus, I was wondering if a new topic config (maybe
> >>>> `allow.negative.timestamps` with default `false`) that allows for enable
> >>>> negative timestamps would be the better solution? With this new config,
> >>>> we would not have any sentinel value for "unknown" and all timestamps
> >>>> would be valid. Old producers, can't write to those topics if they are
> >>>> configured with CREATE_TIME though; APPEND_TIME would still work for
> >>>> older producers but with APPEND_TIME no negative timestamps are possible
> >>>> in the first place, so this config would not have any impact anyway.
> >>>> 
> >>>> Kafka Streams could check the topic config and only drop negative
> >>>> timestamps is they are not enabled. Or course, existing topic should not
> >>>> enable negative timestamps if there are records with -1 in them already
> >>>> -- otherwise, semantics break down -- but this would be a config error
> >>>> we cannot prevent. However, I would expect that mostly newly created
> >>>> topics would enable this config anyway.
> >>>> 
> >>>> 
> >>>> -Matthias
> >>>> 
> >>>> On 12/18/17 10:47 PM, Ewen Cheslack-Postava wrote:
> >>>>> I think the trivial change of just recognizing using -1 was a mistake
> >>>> for a
> >>>>> sentinel value and special casing it while allowing other negative
> >> values
> >>>>> through is the most practical, reasonable change.
> >>>>> 
> >>>>> Realistically, the scope of impact for that -1 is pretty tiny, as has
> >>>> been
> >>>>> pointed out. A single millisecond gap in available timestamps in 1969.
> >>>> For
> >>>>> producers that really want to be careful (as the NYT data might want to
> >>>>> be), having the producer layer adjust accordingly is unlikely to be an
> >>>>> issue (you can't assume these timestamps are unique anyway, so they
> >>>> cannot
> >>>>> reasonably used for ordering; adjusting by 1ms is a practical
> >> tradeoff).
> >>>>> 
> >>>>> Other approaches where we modify the semantics of the timestamp from
> >> the
> >>>>> two existing modes require eating up valuable flags in the message
> >>>> format,
> >>>>> or ramping the message format version, all of which make things
> >>>>> significantly messier. Hell, timezones, leap seconds, and ms
> >> granularity
> >>>>> probably make that 1ms window pretty much moot for any practical
> >>>>> applications, and for the extremely rare case that an application might
> >>>>> care, they are probably willing to pay the cost of a secondary index if
> >>>>> they needed to store timestamp values in the payload rather than in the
> >>>>> metadata.
> >>>>> 
> >>>>> Given that we have the current system in place, I suspect that any
> >>>>> translation to using Long.MIN_VALUE as the sentinel is probably just
> >> more
> >>>>> confusing to users, adds more implementation overhead to client
> >>>> libraries,
> >>>>> and is more likely to introduce bugs.
> >>>>> 
> >>>>> Warts like these always feel wrong when approached from pure design
> >>>>> principles, but the fact is that the constraints are already there. To
> >>>> me,
> >>>>> none of the proposals to move to an encoding we'd prefer seem to add
> >>>> enough
> >>>>> value to outweigh the migration, compatibility, and implementation
> >> costs.
> >>>>> 
> >>>>> @Dong -- your point about special timestamp values is a very good one.
> >>>> The
> >>>>> issue may extend to other cases in the protocol where we use
> >> timestamps.
> >>>> Is
> >>>>> this the scope we need to worry about (2 values instead of just 1) or
> >> are
> >>>>> there others? This also might be something we want to look out for in
> >> the
> >>>>> future -- using special values relative to <SignedIntType>.MIN_VALUE
> >>>>> instead of relative to 0.
> >>>>> 
> >>>>> -Ewen
> >>>>> 
> >>>>> On Tue, Dec 12, 2017 at 11:12 AM, Dong Lin <lindon...@gmail.com>
> >> wrote:
> >>>>> 
> >>>>>> Hey Konstantin,
> >>>>>> 
> >>>>>> Thanks for updating the KIP.
> >>>>>> 
> >>>>>> If we were to support negative timestamp in the message, we probably
> >>>> also
> >>>>>> want to support negative timestamp in ListOffsetRequest. Currently in
> >>>>>> ListOffsetRequest, timestamp value -2 is used to indicate earliest
> >>>>>> timestamp and timestamp value -1 is used to indicate latest timestamp.
> >>>> It
> >>>>>> seems that we should make changes accordingly so that -1 and -2 can be
> >>>>>> supported as valid timestamp in ListOffsetRequest. What do you think?
> >>>>>> 
> >>>>>> Thanks,
> >>>>>> Dong
> >>>>>> 
> >>>>>> 
> >>>>>> 
> >>>>>> On Mon, Dec 11, 2017 at 12:55 PM, Konstantin Chukhlomin <
> >>>>>> chuhlo...@gmail.com
> >>>>>>> wrote:
> >>>>>> 
> >>>>>>> Hi all,
> >>>>>>> 
> >>>>>>> I've updated KIP with few more details:
> >>>>>>> Added (proposed) Changes in binary message format <
> >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-228+
> >>>>>>> Negative+record+timestamp+support#KIP-228Negativerecordtimes
> >>>>>>> tampsupport-Changesinbinarymessageformat>
> >>>>>>> Added Changes from producer perspective <
> >> https://cwiki.apache.org/conf
> >>>>>>> luence/display/KAFKA/KIP-228+Negative+record+timestamp+supp
> >>>>>>> ort#KIP-228Negativerecordtimestampsupport-
> >>>> Changesfromproducerperspective
> >>>>>>> 
> >>>>>>> Added Changes from consumer perspective <
> >> https://cwiki.apache.org/conf
> >>>>>>> luence/display/KAFKA/KIP-228+Negative+record+timestamp+supp
> >>>>>>> ort#KIP-228Negativerecordtimestampsupport-
> >>>> Changesfromconsumerperspective
> >>>>>>> 
> >>>>>>> 
> >>>>>>> Let me know if it makes sense to you.
> >>>>>>> 
> >>>>>>> -Konstantin
> >>>>>>> 
> >>>>>>>> On Dec 7, 2017, at 2:46 PM, Konstantin Chukhlomin <
> >>>> chuhlo...@gmail.com
> >>>>>>> 
> >>>>>>> wrote:
> >>>>>>>> 
> >>>>>>>> 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
> >>>>>>>>>> 
> >>>>>>>>> 
> >>>>>>>> 
> >>>>>>> 
> >>>>>>> 
> >>>>>> 
> >>>>> 
> >>>> 
> >>>> 
> >>> 
> >> 
> >> 
> 

Reply via email to