I have a few questions on security (sorry, only just catching up on the
updates).

1. Will the transaction coordinator check topic ACLs based on the
requesting client's credentials? Access to transaction logs, topics being
added for transaction etc?
2. If I create a transactional produce request (by hand, not using the
producer API) with a random PID (random, hence unlikely to be in use), will
the broker append a transactional message to the logs, preventing LSO from
moving forward? What validation will broker do for PIDs?
3. Will every broker check that a client sending transactional produce
requests at least has write access to transaction log topic since it is not
validating transactional.id (for every produce request)?
4. I understand that brokers cannot authorize the transactional id for each
produce request since requests contain only the PID. But since there is a
one-to-one mapping between PID and transactional.id, and a connection is
never expected to change its transactional.id, perhaps it is feasible to
add authorization and cache the results in the Session? Perhaps not for
version 1, but feels like it will be good to close the security gap here.
Obviously it would be simpler if transactional.id was in the produce
request if the overhead was acceptable.

Thank you,

Rajini


On Thu, Feb 2, 2017 at 8:37 PM, Ismael Juma <ism...@juma.me.uk> wrote:

> Yes, I'd also prefer the option where we only have a checksum at the
> message set level. I didn't suggest it due to the mentioned auditing use
> cases, but if they can be satisfied in some other way, then that would be
> great.
>
> Ismael
>
> On 2 Feb 2017 7:03 pm, "Jason Gustafson" <ja...@confluent.io> wrote:
>
> One more:
>
> 1. I did some benchmarking and CRC32C seems to be a massive win when using
> > the hardware instruction (particularly for messages larger than 65k), so
> > I'm keen on taking advantage of the message format version bump to add
> > support for it. I can write a separate KIP for this as it's not tied to
> > Exactly-once, but it would be good to include the code change in the same
> > PR that bumps the message format version. The benchmark and results can
> be
> > found in the following link:
> > https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
>
>
> Yeah, makes sense. We can add this to this KIP or do it separately,
> whichever you prefer. I have also been very interested in removing the
> individual message CRCs. The main reason we haven't done so is because some
> auditing applications depend on them, but there are cases where it's
> already unsafe to depend on the message CRCs not changing on the broker
> (message conversion and the use of log append time can both result in new
> message-level crcs). So I'm wondering a bit about the use cases that
> require the message CRCs and how they handle this. Perhaps if it is not
> dependable anyway, we can remove it and safe some space and computation.
>
> -Jason
>
>
> On Thu, Feb 2, 2017 at 10:28 AM, Jason Gustafson <ja...@confluent.io>
> wrote:
>
> > Hey Ismael,
> >
> > 2. The message timestamp field is 8 bytes. Did we consider storing the
> >> first timestamp in the message set and then storing deltas using varints
> >> in
> >> the messages like we do for offsets (the difference would be the usage
> of
> >> signed varints)? It seems like the deltas would be quite a bit smaller
> in
> >> the common case (potentially 0 for log append time, so we could even not
> >> store them at all using attributes like we do for key/value lengths). An
> >> alternative is using MaxTimestamp that is already present in the message
> >> set and computing deltas from that, but that seems more complicated. In
> >> any
> >> case, details aside, was this idea considered and rejected or is it
> worth
> >> exploring further?
> >
> >
> > Took me a while to remember why we didn't do this. The timestamp that is
> > included at the message set level is the max timestamp of all messages in
> > the message set as is the case in the current message format (I will
> update
> > the document to make this explicit). We could make the message timestamps
> > relative to the max timestamp, but that makes serialization a bit awkward
> > since the timestamps are not assumed to be increasing sequentially or
> > monotonically. Once the messages in the message set had been determined,
> we
> > would need to go back and adjust the relative timestamps.
> >
> > Here's one idea. We let the timestamps in the messages be varints, but we
> > make their values be relative to the timestamp of the previous message,
> > with the timestamp of the first message being absolute. For example, if
> we
> > had timestamps 500, 501, 499, then we would write 500 for the first
> > message, 1 for the next, and -2 for the final message. Would that work?
> Let
> > me think a bit about it and see if there are any problems.
> >
> > -Jason
> >
> > On Thu, Feb 2, 2017 at 9:04 AM, Apurva Mehta <apu...@confluent.io>
> wrote:
> >
> >> Good point Tom. We will update the KIP with the ACLs section and also
> the
> >> message format changes.
> >>
> >> > On Feb 2, 2017, at 06:45, Tom Crayford <tcrayf...@heroku.com> wrote:
> >> >
> >> > I said this in the voting thread, but can the authors include a
> section
> >> > about new ACLs if there are going to be ACLs for TransactionalId. It's
> >> > mentioned in the google doc, but I think new ACLs should be in a KIP
> >> > directly.
> >> >
> >> >> On Thu, Feb 2, 2017 at 2:42 PM, Ismael Juma <ism...@juma.me.uk>
> wrote:
> >> >>
> >> >> Thanks for the responses and updates to the document, Guozhang and
> >> Jason.
> >> >> They look good. One follow-up and one additional comment:
> >> >>
> >> >> 1. I did some benchmarking and CRC32C seems to be a massive win when
> >> using
> >> >> the hardware instruction (particularly for messages larger than 65k),
> >> so
> >> >> I'm keen on taking advantage of the message format version bump to
> add
> >> >> support for it. I can write a separate KIP for this as it's not tied
> to
> >> >> Exactly-once, but it would be good to include the code change in the
> >> same
> >> >> PR that bumps the message format version. The benchmark and results
> >> can be
> >> >> found in the following link:
> >> >> https://gist.github.com/ijuma/e58ad79d489cb831b290e83b46a7d7bb.
> >> >>
> >> >> 2. The message timestamp field is 8 bytes. Did we consider storing
> the
> >> >> first timestamp in the message set and then storing deltas using
> >> varints in
> >> >> the messages like we do for offsets (the difference would be the
> usage
> >> of
> >> >> signed varints)? It seems like the deltas would be quite a bit
> smaller
> >> in
> >> >> the common case (potentially 0 for log append time, so we could even
> >> not
> >> >> store them at all using attributes like we do for key/value lengths).
> >> An
> >> >> alternative is using MaxTimestamp that is already present in the
> >> message
> >> >> set and computing deltas from that, but that seems more complicated.
> >> In any
> >> >> case, details aside, was this idea considered and rejected or is it
> >> worth
> >> >> exploring further?
> >> >>
> >> >> Ismael
> >> >>
> >> >> On Thu, Feb 2, 2017 at 1:02 AM, Jason Gustafson <ja...@confluent.io>
> >> >> wrote:
> >> >>
> >> >>> Ismael,
> >> >>>
> >> >>> Thanks for the comments. A few responses below:
> >> >>>
> >> >>>
> >> >>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >> introduces
> >> >>> a
> >> >>>> compatibility issue with regards to existing third-party
> authorizers.
> >> >> It
> >> >>>> would be good to highlight this in the migration/compatibility
> >> section.
> >> >>>
> >> >>>
> >> >>> Ack. I added a note in the migration section.
> >> >>>
> >> >>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>> considered
> >> >>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>> 0.10.2.0)?
> >> >>>
> >> >>>
> >> >>> Thanks, I added a few notes about client compatibility to the
> >> migration
> >> >>> section. I covered the main issues that come to mind, but let me
> know
> >> if
> >> >>> you think of others.
> >> >>>
> >> >>> 7. It seems like there is a bit of inconsistency when it comes to
> >> naming
> >> >>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >> conventions.
> >> >>>> There are a few other examples like that and it would be good to
> >> clean
> >> >>> them
> >> >>>> up.
> >> >>>
> >> >>>
> >> >>> Let's go with InitPidRequest for consistency.  Haha,
> "InitPIdRequest"
> >> >> seems
> >> >>> like a compromise which satisfies no one.
> >> >>>
> >> >>>
> >> >>> -Jason
> >> >>>
> >> >>> On Wed, Feb 1, 2017 at 4:14 PM, Guozhang Wang <wangg...@gmail.com>
> >> >> wrote:
> >> >>>
> >> >>>> Ismael, thanks for your feedbacks. Replied inline.
> >> >>>>
> >> >>>>> On Wed, Feb 1, 2017 at 8:03 AM, Ismael Juma <ism...@juma.me.uk>
> >> wrote:
> >> >>>>>
> >> >>>>> Hi all,
> >> >>>>>
> >> >>>>> A few comments follow:
> >> >>>>>
> >> >>>>> 1. The document states "inter-broker communications will be
> >> increased
> >> >>> by
> >> >>>> M
> >> >>>>> * N * P round trips per sec. We need to conduct some system
> >> >> performance
> >> >>>>> test to make sure this additional inter-broker traffic would not
> >> >>> largely
> >> >>>>> impact the broker cluster". Has this testing been done? And if
> not,
> >> >> are
> >> >>>> we
> >> >>>>> planning to do it soon? It seems important to validate this sooner
> >> >>> rather
> >> >>>>> than later. This applies more generally too, it would be great to
> >> >>>>> understand how the new message format affects the producer with
> >> small
> >> >>>>> messages, for example.
> >> >>>>>
> >> >>>>>
> >> >>>> Yes we are conducting the perf tests with the message format
> changes
> >> in
> >> >>> the
> >> >>>> first stage; then the inter-broker communication with minimal
> >> >> transaction
> >> >>>> coordinator implementations in the second stage.
> >> >>>>
> >> >>>>
> >> >>>>> 2. `ProducerAppId` is a new authorization resource type. This
> >> >>> introduces
> >> >>>> a
> >> >>>>> compatibility issue with regards to existing third-party
> >> authorizers.
> >> >>> It
> >> >>>>> would be good to highlight this in the migration/compatibility
> >> >> section.
> >> >>>>>
> >> >>>>> 3. I was happy to see that default values for the new configs have
> >> >> been
> >> >>>>> added to the document since I last checked it. It would be good to
> >> >>>> explain
> >> >>>>> the motivation for the choices.
> >> >>>>>
> >> >>>>>
> >> >>>> Updated doc.
> >> >>>>
> >> >>>>
> >> >>>>> 4. The Migration plan is relatively brief at the moment. Have we
> >> >>>> considered
> >> >>>>> if there's any additional work required due to KIP-97 (introduced
> in
> >> >>>>> 0.10.2.0)?
> >> >>>>>
> >> >>>>> 5. transactional.id sounds good
> >> >>>>>
> >> >>>>> 6. Since we are keeping per message CRCs for auditing apps, have
> we
> >> >>>>> considered mitigating the performance cost by using the more
> >> >> performant
> >> >>>>> CRC32c in the new message format version?
> >> >>>>>
> >> >>>>>
> >> >>>> We have not discussed about this before. But I think it should be
> >> >> doable
> >> >>> as
> >> >>>> long as we can include the additional conversion logic in the
> >> migration
> >> >>>> plan.
> >> >>>>
> >> >>>>
> >> >>>>> Nits:
> >> >>>>>
> >> >>>>> 7. It seems like there is a bit of inconsistency when it comes to
> >> >>> naming
> >> >>>>> convention. For example, we have `InitPIDRequest`, `PidSnapshot`
> >> >>>>> and `InvalidPidMapping`. The latter two match Kafka's naming
> >> >>> conventions.
> >> >>>>> There are a few other examples like that and it would be good to
> >> >> clean
> >> >>>> them
> >> >>>>> up.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree with the inconsistency issue. About the name itself though,
> >> >>> should
> >> >>>> it be "InitPIdRequest", "PIdSnapshot" "InvalidPIdMapping" though,
> >> since
> >> >>> we
> >> >>>> need to capitalize "I" right?
> >> >>>>
> >> >>>>
> >> >>>>> 8. The document states "The first four fields of a message set in
> >> >> this
> >> >>>>> format must to be the same as the existing format because any
> fields
> >> >>>> before
> >> >>>>> the magic byte cannot be changed in order to provide a path for
> >> >>> upgrades
> >> >>>>> following a similar approach as was used in KIP-32". This makes
> >> >> things
> >> >>>>> easier, but it seems to me that the only strict requirement is
> that
> >> >> the
> >> >>>>> magic byte remains in the same offset and with the same size.
> >> >>>>>
> >> >>>>>
> >> >>>> I agree theoretically it is not required, but I think in practice
> it
> >> is
> >> >>>> actually better to make it more restrict: the three fields before
> >> magic
> >> >>>> byte are offset, length, and crc. Among them, crc needs to be
> before
> >> >>> magic
> >> >>>> byte if it wants to cover the magic byte fields; length would
> better
> >> be
> >> >>>> before the magic byte as well for pre-allocate memory to
> >> >> deser/decompress
> >> >>>> the message set, and the only field that does not matter too much
> to
> >> be
> >> >>>> after magic byte is offset, but in KIP-98 we will use it as the
> base
> >> >>> offset
> >> >>>> for message set and some validation checks can be optimized to not
> >> scan
> >> >>>> through the whole message with this field in front of the format.
> >> >>>>
> >> >>>>
> >> >>>>> On Mon, Jan 30, 2017 at 10:37 PM, Guozhang Wang <
> wangg...@gmail.com
> >> >
> >> >>>>> wrote:
> >> >>>>>
> >> >>>>>> Hello Folks,
> >> >>>>>>
> >> >>>>>> We have addressed all the comments collected so far, and would
> like
> >> >>> to
> >> >>>>>> propose a voting thread this Wednesday. If you have any further
> >> >>>> comments
> >> >>>>> on
> >> >>>>>> this KIP, please feel free to continue sending them on this
> thread
> >> >>>> before
> >> >>>>>> that.
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> Guozhang
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> On Mon, Jan 30, 2017 at 1:10 PM, Jason Gustafson <
> >> >> ja...@confluent.io
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>
> >> >>>>>>> +1 for transactional.id.
> >> >>>>>>>
> >> >>>>>>> -Jason
> >> >>>>>>>
> >> >>>>>>> On Mon, Jan 30, 2017 at 1:05 PM, Guozhang Wang <
> >> >> wangg...@gmail.com
> >> >>>>
> >> >>>>>> wrote:
> >> >>>>>>>
> >> >>>>>>>> If I have to choose between app.id and
> >> >> transactional.instance.id
> >> >>> ,
> >> >>>>> I'd
> >> >>>>>>>> choose the latter.
> >> >>>>>>>>
> >> >>>>>>>> Renaming transactional.instance.id to transactional.id sounds
> >> >>> even
> >> >>>>>>> better.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> Guozhang
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Mon, Jan 30, 2017 at 11:49 AM, Apurva Mehta <
> >> >>>> apu...@confluent.io>
> >> >>>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>>> Bumping one suggestion from Apurva above. The name "AppID"
> >> >>> has
> >> >>>>>> caused
> >> >>>>>>>>> some
> >> >>>>>>>>>> confusion. We're considering the following renaming:
> >> >>>>>>>>>>
> >> >>>>>>>>>> 1. AppID -> ProducerId (transaction.app.id -> producer.id)
> >> >>>>>>>>>> 2. PID -> IPID (internal producer ID)
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>> How about AppId -> TransactionalId (transaction.app.id ->
> >> >>>>>>>> transactional.id
> >> >>>>>>>>> )
> >> >>>>>>>>>
> >> >>>>>>>>> This makes it clear that this id just needs to be set when
> >> >> the
> >> >>>>>>>> application
> >> >>>>>>>>> wishes to use transactions. I also think it is more intuitive
> >> >>> in
> >> >>>>> the
> >> >>>>>>>>> context of how this id is used, viz. to maintain transactions
> >> >>>>> across
> >> >>>>>>>>> producer sessions.
> >> >>>>>>>>>
> >> >>>>>>>>> Thanks,
> >> >>>>>>>>> Apurva
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> --
> >> >>>>>>>> -- Guozhang
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> --
> >> >>>>>> -- Guozhang
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> --
> >> >>>> -- Guozhang
> >> >>>>
> >> >>>
> >> >>
> >>
> >
> >
>

Reply via email to