Hi Guozhang,
That makes sense. I will update the KIP wiki and bump up the voting thread
to let people know about this change.
Thanks,
Jiangjie (Becket) Qin
On Tue, Jan 26, 2016 at 10:55 PM, Guozhang Wang wrote:
> One motivation of my proposal is actually to avoid any clients trying to
> read
One motivation of my proposal is actually to avoid any clients trying to
read the timestamp type from the topic metadata response and behave
differently since:
1) topic metadata response is not always in-sync with the source-of-truth
(ZK), hence when the clients realized that the config has change
My hesitation for the changed protocol is that I think If we will have
topic configuration returned in the topic metadata, the current protocol
makes more sense. Because the timestamp type is a topic level setting so we
don't need to put it into each message. That is assuming the timestamp type
cha
I'm in favor of Guozhang's proposal. I think that logic is a bit hacky, but
I agree that this is better than the alternative, and the hackiness only
effects people using log append time which I think will be pretty uncommon.
I think setting that bit will have the additional added value that
consume
Bump up this thread per discussion on the KIP hangout.
During the implementation of the KIP, Guozhang raised another proposal on
how to indicate the message timestamp type used by messages. So we want to
see people's opinion on this proposal.
The difference between current and the new proposal on
Jun,
1. I agree it would be nice to have the timestamps used in a unified way.
My concern is that if we let server change timestamp of the inner message
for LogAppendTime, that will enforce the user who are using LogAppendTime
to always pay the recompression penalty. So using LogAppendTime makes
K
Jun, Jiangjie,
I am confused about 3) here, if we use "the timestamp of the latest message"
then doesn't this mean we will roll the log whenever a message delayed by
rolling time is received as well? Just to clarify, my understanding of "the
timestamp of the latest message", for example in the fol
1. Hmm, it's more intuitive if the consumer sees the same timestamp whether
the messages are compressed or not. When message.timestamp.type=LogAppendTime,
we will need to set timestamp in each message if messages are not
compressed, so that the follower can get the same timestamp. So, it seems
that
Thanks for the explanation, Jun.
1. That makes sense. So maybe we can do the following:
(a) Set the timestamp in the compressed message to latest timestamp of all
its inner messages. This works for both LogAppendTime and CreateTime.
(b) If message.timestamp.type=LogAppendTime, the broker will over
1. I was thinking more about saving the decompression overhead in the
follower. Currently, the follower doesn't decompress the messages. To keep
it that way, the outer message needs to include the timestamp of the latest
inner message to build the time index in the follower. The simplest thing
to d
Hi Jun,
Thanks a lot for the comments. Please see inline replies.
Thanks,
Jiangjie (Becket) Qin
On Mon, Dec 14, 2015 at 10:19 AM, Jun Rao wrote:
> Hi, Becket,
>
> Thanks for the proposal. Looks good overall. A few comments below.
>
> 1. KIP-32 didn't say what timestamp should be set in a comp
Hi, Becket,
Thanks for the proposal. Looks good overall. A few comments below.
1. KIP-32 didn't say what timestamp should be set in a compressed message.
We probably should set it to the timestamp of the latest messages included
in the compressed one. This way, during indexing, we don't have to
d
Hey Jay,
Thanks for the comments.
Good point about the actions after when max.message.time.difference is
exceeded. Rejection is a useful behavior although I cannot think of use
case at LinkedIn at this moment. I think it makes sense to add a
configuration.
How about the following configurations?
Hey Becket,
That summary of pros and cons sounds about right to me.
There are potentially two actions you could take when
max.message.time.difference is exceeded--override it or reject the
message entirely. Can we pick one of these or does the action need to
be configurable too? (I'm not sure). T
It looks the format of the previous email was messed up. Send it again.
Just to recap, the last proposal Jay made (with some implementation
details added)
was:
1. Allow user to stamp the message when produce
2. When broker receives a message it take a look at the difference between
its local tim
Bump up this thread.
Just to recap, the last proposal Jay made (with some implementation details
added) was:
1. Allow user to stamp the message when produce
2. When broker receives a message it take a look at the difference
between its local time and the timestamp in the message.
-
Hi Jay,
Thanks for such detailed explanation. I think we both are trying to make
CreateTime work for us if possible. To me by "work" it means clear
guarantees on:
1. Log Retention Time enforcement.
2. Log Rolling time enforcement (This might be less a concern as you
pointed out)
3. Application sea
one more thing.. I wanted to clarify that I am not proposing stamping two
time properties per message either.
I think at this point we have a solution for "event time" (as described in
Millwheel) at Li.. (it is in the event payload).
So we just need a way to fix the log retention isssues and impr
This thread has gone rather long. So I might not be fully clear on
everybody's positions.
To make sure I understand the possible proposals. Here is a concrete and
'must have' scenario for LinkedIn.
We want to bootstrap from say a Database dump (espresso) into a kafka
topic. (Databus bootstrap)
Hey Joel,
I think we're mostly in agreement. I don't know about you but I'm not
head-over-heals in love with any of these proposals, including mine--time
is just pretty complicated and icky.
With respect to using log append time here was basically how I felt:
1. I think in a vacuum basing offset
Hey Kartik,
Yes, I agree exactly with your characterization.
The question is "what is the meaning of retention?" It could mean either:
1. "Retain data that is no more than 7 days old"
2. "Retain data for 7 days from when you get it"
I don't know if either is actually a clear winner.
Each is int
Hey Becket,
Let me see if I can address your concerns:
1. Let's say we have two source clusters that are mirrored to the same
> target cluster. For some reason one of the mirror maker from a cluster dies
> and after fix the issue we want to resume mirroring. In this case it is
> possible that whe
I’m in favor of adding the create-time in the message (although some would
argue even that should really be an application-level header), but I don’t
think it should be mutable after it leaves the client and I think we should
avoid having the server use that for any server-side indexing. The
max.a
Joel or Becket will probably respond back in more detail.. but here are my
2c.
>From the standpoint of LinkedIN, the suggested proposal works.. in essence
max.appenddelay can be used to turn "creationTime" into "logAppendTime".
this does mean that at LinkedIn we won't be able to use "creationTi
Hey Jay and Guozhang,
Thanks a lot for the reply. So if I understand correctly, Jay's proposal is:
1. Let client stamp the message create time.
2. Broker build index based on client-stamped message create time.
3. Broker only takes message whose create time is withing current time
plus/minus T (T
Just to complete Jay's option, here is my understanding:
1. For log retention: if we want to remove data before time t, we look into
the index file of each segment and find the largest timestamp t' < t, find
the corresponding timestamp and start scanning to the end of the segment,
if there is no e
I think it should be possible to index out-of-order timestamps. The
timestamp index would be similar to the offset index, a memory mapped file
appended to as part of the log append, but would have the format
timestamp offset
The timestamp entries would be monotonic and as with the offset index wo
Here's my basic take:
- I agree it would be nice to have a notion of time baked in if it were
done right
- All the proposals so far seem pretty complex--I think they might make
things worse rather than better overall
- I think adding 2x8 byte timestamps to the message is probably a
non-starter from
Bumping up this thread although most of the discussion were on the
discussion thread of KIP-31 :)
I just updated the KIP page to add detailed solution for the option (Option
3) that does not expose the LogAppendTime to user.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+CreateTim
Hi Jay,
I just copy/pastes here your feedback on the timestamp proposal that was in
the discussion thread of KIP-31. Please see the replies inline.
The main change I made compared with previous proposal is to add both
CreateTime and LogAppendTime to the message.
On Tue, Sep 8, 2015 at 10:57 AM, J
30 matches
Mail list logo