Hi Jay,

I can't go too deep into exact implantation due to no NDA. So apologies here.

Essentially we have multiple processes each owning selection of accounts so on 
general flows an action for an account just needs to be managed local to the 
owning node, happy days ever change is handled as a tick tock change.

Unfortunately when a transfer occurs we need the two processes to co-ordinate 
their transaction, we also need to ensure both don't continue other 
actions/changesl, we do this using a data grid technology. This grid technology 
supports transaction manager that we couple into currently our jms provider 
which supports full XA transactions as such we can manage the production of the 
change messages out the system transactionally as well as the in grid state.

The obvious arguement here is should we even look to move this flow off JMS 
then. We prob shouldn't nor will do this.

The point is that I think saying Kafka supports transactions but then not 
supporting it as per the traditional sense leads to developers expecting 
similar behaviour and will cause issues in prod when they find it doesn't work 
as they're used to.

As my other response earlier, is there a better name to describe this feature, 
if we're not implementing transactions to the traditional transaction expected, 
to avoid this confusion?


Sent using OWA for iPhone
________________________________________
From: Jay Kreps <j...@confluent.io>
Sent: Friday, December 9, 2016 6:08:07 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional Messaging

Hey Michael,

Doesn't that example have more to do with applying the update against two
rows in a single transaction? That is, clearly the write to Kafka needs to
be "transactional" and the write to the destination needs to be
transactional, but it's not clear to me that you need isolation that spans
both operations. Can you dive into the system architecture a bit more and
explain why Kafka needs to participate in the same transaction as the
destination system?

-Jay

On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <michael.pea...@ig.com>
wrote:

> Usecase in IG:
>
> Fund transfer between accounts. When we debit one account and fund another
> we must ensure the records to both occur as an acid action, and as a single
> transaction.
>
> Today we achieve this because we have jms, as such we can do the actions
> needed in an xa transaction across both the accounts. To move this flow to
> Kafka we would need support of XA transaction.
>
>
>
> Sent using OWA for iPhone
> ________________________________________
> From: Michael Pearce <michael.pea...@ig.com>
> Sent: Friday, December 9, 2016 6:09:06 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hi Jay,
>
> For me having an XA transaction allows for ensuring ACID across my
> application.
>
> I believe it is part of the JMS api, and obviously JMS still is in
> enterprise very widely adopted for Messaging transport , so obviously to
> say it isn't widely used i think is ignoring a whole range of users. Like
> wise I believe frameworks like spring etc fully support it more evidence of
> its wide adoption.
>
> On this note personally we try to avoid transactions entirely in our flows
> for performance and simplicity. but we do alas unfortunately have one or
> two places we cannot ignore it.
>
> Cheers
> Mike
>
> Sent using OWA for iPhone
> ________________________________________
> From: Jay Kreps <j...@confluent.io>
> Sent: Thursday, December 8, 2016 11:25:53 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> Messaging
>
> Hey Edoardo,
>
> For (3) can you outline what you think the benefit and use cases for a more
> general cross-system XA feature would be an what changes to the proposal
> would be required to enable it? When I have asked people who wanted
> cross-system XA in the past what they wanted it for, I haven't really
> gotten any answers that made sense. Every person really wanted something
> that would be better solved by a transactional (or idempotent) write to
> Kafka followed by an independent transactional (or idempotent) consumption
> (which this proposal enables). For the use cases they described tying these
> two things together had no advantage and many disadvantages.
>
> I have one use case which would be accomplished by cross-system XA which is
> allowing the producer to block on the synchronous processing of the message
> by (all? some?) consumers. However I'm not convinced that cross-system XA
> is the best solution to this problem, and I'm also not convinced this is an
> important problem to solve. But maybe you have something in mind here.
>
> -Jay
>
>
>
> On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <eco...@uk.ibm.com> wrote:
>
> > Hi,
> > thanks, very interesting KIP ... I haven't fully digested it yet.
> >
> > We have many users who choose not to use the Java client,  so I have
> > concerns about the added complexity in developing the clients.
> > A few questions.
> >
> > 1 - is mixing transactional and non transactional messages on the *same
> > topic-partition* really a requirement ?
> > What use case does it satisfy?
> >
> > 2 - I guess some clients may only be interested to implement the producer
> > idempotency.
> > It's not clear how they could be implemented without having to add the
> > transaction capabilities.
> > As others on this list have said, I too would like to see idempotency as
> a
> > more basic feature, on top which txns can be built.
> >
> > 3 - The KIP seems focused on a use case where consumption from a topic
> and
> > subsequent production are part of the producer transaction.
> >
> > It'd be great to see a way to extend the producer transaction to include
> > additional transactional resources,
> > so that the consumption from another topic just becomes a special case of
> > a more general "distributed" txn.
> >
> > Edo
> > --------------------------------------------------
> > Edoardo Comar
> > IBM MessageHub
> > eco...@uk.ibm.com
> > IBM UK Ltd, Hursley Park, SO21 2JN
> >
> > IBM United Kingdom Limited Registered in England and Wales with number
> > 741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants.
> PO6
> > 3AU
> >
> >
> >
> > From:   Guozhang Wang <wangg...@gmail.com>
> > To:     "dev@kafka.apache.org" <dev@kafka.apache.org>
> > Date:   30/11/2016 22:20
> > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and Transactional
> > Messaging
> >
> >
> >
> > Hi all,
> >
> > I have just created KIP-98 to enhance Kafka with exactly once delivery
> > semantics:
> >
> > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > >*
> >
> > This KIP adds a transactional messaging mechanism along with an
> idempotent
> > producer implementation to make sure that 1) duplicated messages sent
> from
> > the same identified producer can be detected on the broker side, and 2) a
> > group of messages sent within a transaction will atomically be either
> > reflected and fetchable to consumers or not as a whole.
> >
> > The above wiki page provides a high-level view of the proposed changes as
> > well as summarized guarantees. Initial draft of the detailed
> > implementation
> > design is described in this Google doc:
> >
> > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > 0wSw9ra8
> >
> >
> > We would love to hear your comments and suggestions.
> >
> > Thanks,
> >
> > -- Guozhang
> >
> >
> >
> > Unless stated otherwise above:
> > IBM United Kingdom Limited - Registered in England and Wales with number
> > 741598.
> > Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6
> 3AU
> >
> 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