Thanks Jun, let's go with the simple approach (not using attributes) and
revise later if we find a compelling reason. I'll update the docs.

-Jason

On Tue, Feb 21, 2017 at 6:23 PM, Jun Rao <j...@confluent.io> wrote:

> It seems that it's simpler and more consistent to avoid optional keys and
> values. Not sure if it's worth squeezing every byte at the expense of
> additional complexity. Other than that, +1 from me.
>
> Also, since this is a large KIP, minor changes may arise as we start the
> implementation. It would be good if we can keep the community posted of
> those changes, if any.
>
> Thanks,
>
> Jun
>
> On Tue, Feb 21, 2017 at 4:33 PM, Michael Pearce <michael.pea...@ig.com>
> wrote:
>
> > If the argument and objective within this KIP is to keep the overhead of
> > the protocol as small as possible and remove redundancy, and every byte
> is
> > being counted and the introduction of varInts, then it would make sense
> to
> > use attributes to me.
> >
> >
> > On 22/02/2017, 00:14, "Jason Gustafson" <ja...@confluent.io> wrote:
> >
> >     Done. I've left the key and value as optional since we may not have
> > reached
> >     consensus on whether to use attributes or not. Perhaps we should just
> > keep
> >     it simple and not do it? The benefit seems small.
> >
> >     -Jason
> >
> >     On Tue, Feb 21, 2017 at 4:05 PM, Michael Pearce <
> michael.pea...@ig.com
> > >
> >     wrote:
> >
> >     > Ok, no worries, can you add it back ValueLen on this KIP, and
> update
> > the
> >     > doc, then we can work from that ☺
> >     >
> >     > Cheers
> >     > Mike
> >     >
> >     > On 22/02/2017, 00:02, "Jason Gustafson" <ja...@confluent.io>
> wrote:
> >     >
> >     >     I feel it was a little odd to leave out the value length
> anyway,
> > so I
> >     > would
> >     >     rather add it back and put headers at the end. This is more
> > consistent
> >     > with
> >     >     the rest of the Kafka protocol.
> >     >
> >     >     -Jason
> >     >
> >     >     On Tue, Feb 21, 2017 at 3:58 PM, Michael Pearce <
> > michael.pea...@ig.com
> >     > >
> >     >     wrote:
> >     >
> >     >     > Or we keep as is (valuelen removed), and headers are added
> with
> >     > headers
> >     >     > length..
> >     >     >
> >     >     > On 21/02/2017, 23:38, "Apurva Mehta" <apu...@confluent.io>
> > wrote:
> >     >     >
> >     >     >     Right now, we don't need the value length: since it is
> the
> > last
> >     > item
> >     >     > in the
> >     >     >     message, and we have the message length, we can deduce
> the
> > value
> >     >     > length.
> >     >     >     However, if we are adding record headers to the end, we
> > would
> >     > need to
> >     >     >     introduce the value length along with that change.
> >     >     >
> >     >     >     On Tue, Feb 21, 2017 at 3:34 PM, Michael Pearce <
> >     > michael.pea...@ig.com
> >     >     > >
> >     >     >     wrote:
> >     >     >
> >     >     >     > It seems I cannot add comment on the doc.
> >     >     >     >
> >     >     >     > In the section around the message protocol.
> >     >     >     >
> >     >     >     > It has stated:
> >     >     >     >
> >     >     >     > Message =>
> >     >     >     > Length => uintVar
> >     >     >     > Attributes => int8
> >     >     >     > TimestampDelta => intVar
> >     >     >     > OffsetDelta => uintVar
> >     >     >     > KeyLen => uintVar [OPTIONAL]
> >     >     >     > Key => data [OPTIONAL]
> >     >     >     > Value => data [OPTIONAL]
> >     >     >     >
> >     >     >     > Should it not be: (added missing value len)
> >     >     >     >
> >     >     >     > Message =>
> >     >     >     > Length => uintVar
> >     >     >     > Attributes => int8
> >     >     >     > TimestampDelta => intVar
> >     >     >     > OffsetDelta => uintVar
> >     >     >     > KeyLen => uintVar [OPTIONAL]
> >     >     >     > Key => data [OPTIONAL]
> >     >     >     > ValueLen => uintVar [OPTIONAL]
> >     >     >     > Value => data [OPTIONAL]
> >     >     >     >
> >     >     >     >
> >     >     >     >
> >     >     >     > On 21/02/2017, 23:07, "Joel Koshy" <
> jjkosh...@gmail.com>
> >     > wrote:
> >     >     >     >
> >     >     >     >     I left a couple of comments/questions directly on
> the
> >     > google-doc
> >     >     >     >     <https://docs.google.com/document/d/11Jqy_
> >     >     >     > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0wSw9ra8>
> >     >     >     >     - I found it much more tractable for a proposal of
> > this
> >     > size to
> >     >     >     > discuss in
> >     >     >     >     context within the doc. The permissions on the doc
> > don't
> >     > let
> >     >     > everyone
> >     >     >     > view
> >     >     >     >     comments, so if there are any material changes that
> > come
> >     > out of
> >     >     > the
> >     >     >     >     discussions in those comment threads we can
> > summarize here.
> >     >     >     >
> >     >     >     >     Thanks,
> >     >     >     >
> >     >     >     >     Joel
> >     >     >     >
> >     >     >     >     On Mon, Feb 20, 2017 at 4:08 PM, Becket Qin <
> >     >     > becket....@gmail.com>
> >     >     >     > wrote:
> >     >     >     >
> >     >     >     >     > Thanks for the explanation, Guozhang. That makes
> > sense.
> >     >     >     >     >
> >     >     >     >     > On Sun, Feb 19, 2017 at 7:28 PM, Guozhang Wang <
> >     >     > wangg...@gmail.com>
> >     >     >     > wrote:
> >     >     >     >     >
> >     >     >     >     > > Thanks Becket.
> >     >     >     >     > >
> >     >     >     >     > > Actually sequence is associated with a message,
> > not a
> >     >     > message set.
> >     >     >     > For
> >     >     >     >     > > example if a message set sent by producer
> > contains 100
> >     >     > messages,
> >     >     >     > and the
> >     >     >     >     > > first message's sequence is 5, then the last
> > message's
> >     >     > sequence
> >     >     >     > number
> >     >     >     >     > > would be 104, and the next message set's first
> >     > sequence is
> >     >     >     > expected to be
> >     >     >     >     > > 105.
> >     >     >     >     > >
> >     >     >     >     > >
> >     >     >     >     > > Guozhang
> >     >     >     >     > >
> >     >     >     >     > >
> >     >     >     >     > > On Sun, Feb 19, 2017 at 4:48 PM, Becket Qin <
> >     >     > becket....@gmail.com>
> >     >     >     >     > wrote:
> >     >     >     >     > >
> >     >     >     >     > > > +1. Thanks for the great work on the KIP!
> >     >     >     >     > > >
> >     >     >     >     > > > I have only one minor question, in the wiki
> > (and the
> >     > doc)
> >     >     > the new
> >     >     >     >     > message
> >     >     >     >     > > > set format has a "FirstSequence" field,
> should
> > it
> >     > just be
> >     >     >     > "Sequence" if
> >     >     >     >     > > the
> >     >     >     >     > > > sequence is always associated with a message
> > set?
> >     >     >     >     > > >
> >     >     >     >     > > > On Fri, Feb 17, 2017 at 3:28 AM, Michael
> > Pearce <
> >     >     >     > michael.pea...@ig.com
> >     >     >     >     > >
> >     >     >     >     > > > wrote:
> >     >     >     >     > > >
> >     >     >     >     > > > > +0
> >     >     >     >     > > > >
> >     >     >     >     > > > > I think need some unified agreement on the
> > VarInts.
> >     >     >     >     > > > >
> >     >     >     >     > > > > Would this also change in all other area’s
> > of the
> >     >     > protocol,
> >     >     >     > e.g.
> >     >     >     >     > value
> >     >     >     >     > > > and
> >     >     >     >     > > > > key length in message protocol, to keep
> this
> >     > uniform
> >     >     > across all
> >     >     >     >     > > protocols
> >     >     >     >     > > > > going forwards?
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > > On 17/02/2017, 00:23, "Apurva Mehta" <
> >     >     > apu...@confluent.io>
> >     >     >     > wrote:
> >     >     >     >     > > > >
> >     >     >     >     > > > >     Hi Jun,
> >     >     >     >     > > > >
> >     >     >     >     > > > >     Thanks for the reply. Comments inline.
> >     >     >     >     > > > >
> >     >     >     >     > > > >     On Thu, Feb 16, 2017 at 2:29 PM, Jun
> Rao
> > <
> >     >     > j...@confluent.io
> >     >     >     > >
> >     >     >     >     > wrote:
> >     >     >     >     > > > >
> >     >     >     >     > > > >     > Hi, Apurva,
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     > Thanks for the reply. A couple of
> > comment
> >     > below.
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     > On Wed, Feb 15, 2017 at 9:45 PM,
> Apurva
> >     > Mehta <
> >     >     >     >     > > apu...@confluent.io
> >     >     >     >     > > > >
> >     >     >     >     > > > > wrote:
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     > > Hi Jun,
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > > Answers inline:
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > > 210. Pid snapshots: Is the number
> of
> > pid
> >     > snapshot
> >     >     >     >     > configurable
> >     >     >     >     > > or
> >     >     >     >     > > > >     > hardcoded
> >     >     >     >     > > > >     > > > with 2? When do we decide to roll
> > a new
> >     >     > snapshot?
> >     >     >     > Based on
> >     >     >     >     > > > time,
> >     >     >     >     > > > > byte,
> >     >     >     >     > > > >     > or
> >     >     >     >     > > > >     > > > offset? Is that configurable too?
> >     >     >     >     > > > >     > > >
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > >     > When a replica becomes a follower, we
> > do a
> >     > bit log
> >     >     >     > truncation.
> >     >     >     >     > > > > Having an
> >     >     >     >     > > > >     > older snapshot allows us to recover
> the
> >     >     > PID->sequence
> >     >     >     > mapping
> >     >     >     >     > > much
> >     >     >     >     > > > > quicker
> >     >     >     >     > > > >     > than rescanning the whole log.
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > >     This is a good point. I have updated
> the
> > doc
> >     > with a
> >     >     > more
> >     >     >     > detailed
> >     >     >     >     > > > > proposal.
> >     >     >     >     > > > >     Essentially, snapshots will be created
> > on a
> >     > periodic
> >     >     >     > basis. A
> >     >     >     >     > > > > reasonable
> >     >     >     >     > > > >     period would be every 30 or 60 seconds.
> > We
> >     > will keep
> >     >     > at
> >     >     >     > most 2
> >     >     >     >     > > copies
> >     >     >     >     > > > > of
> >     >     >     >     > > > >     the snapshot file. With this setup, we
> > would
> >     > have to
> >     >     >     > replay at
> >     >     >     >     > most
> >     >     >     >     > > > 60
> >     >     >     >     > > > > or
> >     >     >     >     > > > >     120 seconds of the log in the event of
> > log
> >     > truncation
> >     >     >     > during
> >     >     >     >     > leader
> >     >     >     >     > > > >     failover.
> >     >     >     >     > > > >
> >     >     >     >     > > > >     If we need to make any of this
> > configurable,
> >     > we can
> >     >     > expose
> >     >     >     > a
> >     >     >     >     > config
> >     >     >     >     > > > in
> >     >     >     >     > > > > the
> >     >     >     >     > > > >     future. It would be easier to add a
> > config we
> >     > need
> >     >     > than
> >     >     >     > remove
> >     >     >     >     > one
> >     >     >     >     > > > with
> >     >     >     >     > > > >     marginal utility.
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     > > >
> >     >     >     >     > > > >     > > > 211. I am wondering if we should
> > store
> >     >     >     > ExpirationTime in
> >     >     >     >     > the
> >     >     >     >     > > > > producer
> >     >     >     >     > > > >     > > > transactionalId mapping message
> as
> > we do
> >     > in the
> >     >     >     > producer
> >     >     >     >     > > > > transaction
> >     >     >     >     > > > >     > > status
> >     >     >     >     > > > >     > > > message. If a producer only calls
> >     >     >     > initTransactions(), but
> >     >     >     >     > > never
> >     >     >     >     > > > >     > publishes
> >     >     >     >     > > > >     > > > any data, we still want to be
> able
> > to
> >     > expire
> >     >     > and
> >     >     >     > remove the
> >     >     >     >     > > > > producer
> >     >     >     >     > > > >     > > > transactionalId mapping message.
> >     >     >     >     > > > >     > > >
> >     >     >     >     > > > >     > > >
> >     >     >     >     > > > >     > > Actually, the document was
> > inaccurate. The
> >     >     >     > transactionalId
> >     >     >     >     > will
> >     >     >     >     > > > be
> >     >     >     >     > > > >     > expired
> >     >     >     >     > > > >     > > only if there is no active
> > transaction,
> >     > and the
> >     >     > age of
> >     >     >     > the
> >     >     >     >     > last
> >     >     >     >     > > > >     > transaction
> >     >     >     >     > > > >     > > with that transactionalId is older
> > than the
> >     >     >     > transactioanlId
> >     >     >     >     > > > > expiration
> >     >     >     >     > > > >     > > time. With these semantics, storing
> > the
> >     >     > expiration
> >     >     >     > time in
> >     >     >     >     > the
> >     >     >     >     > > > >     > > transactionalId mapping message
> > won't be
> >     > useful,
> >     >     > since
> >     >     >     > the
> >     >     >     >     > > > > expiration
> >     >     >     >     > > > >     > time
> >     >     >     >     > > > >     > > is a moving target based on
> > transaction
> >     > activity.
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > > I have updated the doc with a
> >     > clarification.
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > >
> >     >     >     >     > > > >     > Currently, the producer
> transactionalId
> >     > mapping
> >     >     > message
> >     >     >     > doesn't
> >     >     >     >     > > > carry
> >     >     >     >     > > > >     > ExpirationTime, but the producer
> > transaction
> >     > status
> >     >     >     > message
> >     >     >     >     > does.
> >     >     >     >     > > > > It would
> >     >     >     >     > > > >     > be useful if they are consistent.
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     >
> >     >     >     >     > > > >     You are right. The document has been
> > updated to
> >     >     > remove the
> >     >     >     >     > > > > ExpirationTime
> >     >     >     >     > > > >     from the transaction status messages as
> > well.
> >     > Any
> >     >     > utility
> >     >     >     > for
> >     >     >     >     > this
> >     >     >     >     > > > > field
> >     >     >     >     > > > >     can be achieved by using the timestamp
> > of the
> >     > message
> >     >     >     > itself
> >     >     >     >     > along
> >     >     >     >     > > > with
> >     >     >     >     > > > >     another expiration time (like
> > transactionalId
> >     >     > expiration
> >     >     >     > time,
> >     >     >     >     > > > > transaction
> >     >     >     >     > > > >     expiration time, etc.).
> >     >     >     >     > > > >
> >     >     >     >     > > > >     Thanks,
> >     >     >     >     > > > >     Apurva
> >     >     >     >     > > > >
> >     >     >     >     > > > >
> >     >     >     >     > > > > The information contained in this email is
> > strictly
> >     >     >     > confidential and
> >     >     >     >     > > for
> >     >     >     >     > > > > the use of the addressee only, unless
> > otherwise
> >     >     > indicated. If
> >     >     >     > you are
> >     >     >     >     > > not
> >     >     >     >     > > > > the intended recipient, please do not read,
> > copy,
> >     > use or
> >     >     >     > disclose to
> >     >     >     >     > > > others
> >     >     >     >     > > > > this message or any attachment. Please also
> > notify
> >     > the
> >     >     > sender
> >     >     >     > by
> >     >     >     >     > > replying
> >     >     >     >     > > > > to this email or by telephone (+44(020 7896
> > 0011)
> >     > and
> >     >     > then
> >     >     >     > delete the
> >     >     >     >     > > > email
> >     >     >     >     > > > > and any copies of it. Opinions, conclusion
> > (etc)
> >     > that do
> >     >     > not
> >     >     >     > relate
> >     >     >     >     > to
> >     >     >     >     > > > the
> >     >     >     >     > > > > official business of this company shall be
> >     > understood as
> >     >     >     > neither
> >     >     >     >     > given
> >     >     >     >     > > > nor
> >     >     >     >     > > > > endorsed by it. IG is a trading name of IG
> > Markets
> >     >     > Limited (a
> >     >     >     > company
> >     >     >     >     > > > > registered in England and Wales, company
> > number
> >     >     > 04008957) and
> >     >     >     > IG
> >     >     >     >     > Index
> >     >     >     >     > > > > Limited (a company registered in England
> and
> > Wales,
> >     >     > company
> >     >     >     > number
> >     >     >     >     > > > > 01190902). Registered address at Cannon
> > Bridge
> >     > House, 25
> >     >     >     > Dowgate
> >     >     >     >     > Hill,
> >     >     >     >     > > > > London EC4R 2YA. Both IG Markets Limited
> > (register
> >     > number
> >     >     >     > 195355) and
> >     >     >     >     > > IG
> >     >     >     >     > > > > Index Limited (register number 114059) are
> >     > authorised and
> >     >     >     > regulated
> >     >     >     >     > by
> >     >     >     >     > > > the
> >     >     >     >     > > > > Financial Conduct Authority.
> >     >     >     >     > > > >
> >     >     >     >     > > >
> >     >     >     >     > >
> >     >     >     >     > >
> >     >     >     >     > >
> >     >     >     >     > > --
> >     >     >     >     > > -- Guozhang
> >     >     >     >     > >
> >     >     >     >     >
> >     >     >     >
> >     >     >     >
> >     >     >     > The information contained in this email is strictly
> >     > confidential and
> >     >     > for
> >     >     >     > the use of the addressee only, unless otherwise
> > indicated. If
> >     > you
> >     >     > are not
> >     >     >     > the intended recipient, please do not read, copy, use
> or
> >     > disclose to
> >     >     > others
> >     >     >     > this message or any attachment. Please also notify the
> > sender
> >     > by
> >     >     > replying
> >     >     >     > to this email or by telephone (+44(020 7896 0011) and
> > then
> >     > delete
> >     >     > the email
> >     >     >     > and any copies of it. Opinions, conclusion (etc) that
> do
> > not
> >     > relate
> >     >     > to the
> >     >     >     > official business of this company shall be understood
> as
> >     > neither
> >     >     > given nor
> >     >     >     > endorsed by it. IG is a trading name of IG Markets
> > Limited (a
> >     > company
> >     >     >     > registered in England and Wales, company number
> > 04008957) and
> >     > IG
> >     >     > Index
> >     >     >     > Limited (a company registered in England and Wales,
> > company
> >     > number
> >     >     >     > 01190902). Registered address at Cannon Bridge House,
> 25
> >     > Dowgate
> >     >     > Hill,
> >     >     >     > London EC4R 2YA. Both IG Markets Limited (register
> number
> >     > 195355)
> >     >     > and IG
> >     >     >     > Index Limited (register number 114059) are authorised
> and
> >     > regulated
> >     >     > by the
> >     >     >     > Financial Conduct Authority.
> >     >     >     >
> >     >     >
> >     >     >
> >     >     > The information contained in this email is strictly
> > confidential and
> >     > for
> >     >     > the use of the addressee only, unless otherwise indicated. If
> > you
> >     > are not
> >     >     > the intended recipient, please do not read, copy, use or
> > disclose to
> >     > others
> >     >     > this message or any attachment. Please also notify the sender
> > by
> >     > replying
> >     >     > to this email or by telephone (+44(020 7896 0011) and then
> > delete
> >     > the email
> >     >     > and any copies of it. Opinions, conclusion (etc) that do not
> > relate
> >     > to the
> >     >     > official business of this company shall be understood as
> > neither
> >     > given nor
> >     >     > endorsed by it. IG is a trading name of IG Markets Limited (a
> > company
> >     >     > registered in England and Wales, company number 04008957) and
> > IG
> >     > Index
> >     >     > Limited (a company registered in England and Wales, company
> > number
> >     >     > 01190902). Registered address at Cannon Bridge House, 25
> > Dowgate
> >     > Hill,
> >     >     > London EC4R 2YA. Both IG Markets Limited (register number
> > 195355)
> >     > and IG
> >     >     > Index Limited (register number 114059) are authorised and
> > regulated
> >     > by the
> >     >     > Financial Conduct Authority.
> >     >     >
> >     >
> >     >
> >     > The information contained in this email is strictly confidential
> and
> > for
> >     > the use of the addressee only, unless otherwise indicated. If you
> > are not
> >     > the intended recipient, please do not read, copy, use or disclose
> to
> > others
> >     > this message or any attachment. Please also notify the sender by
> > replying
> >     > to this email or by telephone (+44(020 7896 0011) and then delete
> > the email
> >     > and any copies of it. Opinions, conclusion (etc) that do not relate
> > to the
> >     > official business of this company shall be understood as neither
> > given nor
> >     > endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> >     > registered in England and Wales, company number 04008957) and IG
> > Index
> >     > Limited (a company registered in England and Wales, company number
> >     > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > Hill,
> >     > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> > and IG
> >     > Index Limited (register number 114059) are authorised and regulated
> > by the
> >     > Financial Conduct Authority.
> >     >
> >
> >
> > The information contained in this email is strictly confidential and for
> > the use of the addressee only, unless otherwise indicated. If you are not
> > the intended recipient, please do not read, copy, use or disclose to
> others
> > this message or any attachment. Please also notify the sender by replying
> > to this email or by telephone (+44(020 7896 0011) and then delete the
> email
> > and any copies of it. Opinions, conclusion (etc) that do not relate to
> the
> > official business of this company shall be understood as neither given
> nor
> > endorsed by it. IG is a trading name of IG Markets Limited (a company
> > registered in England and Wales, company number 04008957) and IG Index
> > Limited (a company registered in England and Wales, company number
> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate Hill,
> > London EC4R 2YA. Both IG Markets Limited (register number 195355) and IG
> > Index Limited (register number 114059) are authorised and regulated by
> the
> > Financial Conduct Authority.
> >
>

Reply via email to