Agreed, I think it is better to either support transactions properly and fully 
to the level or expectation normally associated with a transactional system. Or 
don't provide it at all.  

Having a half way house can be more dangerous.

As I said earlier the issue of message duplication can be handled entirely 
without transactions.

Sent from my iPhone

> On 9 Dec 2016, at 10:49, Andrew Schofield <andrew_schofield_j...@outlook.com> 
> wrote:
> 
> I've been pondering this question of coordinating other resource managers with
> Kafka transactions for a while and I'm not convinced it's a good idea. My
> reservations come down to the guarantees that it would provide in failure
> scenarios.
> 
> I don't think KIP-98 gives proper ACID semantics in the presence of all
> failures. For a transaction which contains a mixture of publishes and offset
> updates, a bunch of topics are involved and it appears to me that an
> uncontrolled shutdown could result in some but not all of the lazy writes
> making it to disk.
> 
> Here are some of the failures that I'm worried about:
> 
> * A message is published to a topic which crashes the leader Kafka node, as
>  it's replicated across the cluster, it crashes all of the other Kafka nodes
>  (we've really had this - SEGV, our fault and we've fixed it, but it happened)
>  so this is a kind of rolling node crash in a cluster
> * Out of memory error in one or more Kafka nodes
> * Disk fills in one or more Kafka nodes
> * Uncontrolled power-off to all nodes in the cluster
> 
> Does KIP-98 guarantee atomicity for transactions in all of these cases?
> Unless all of the topics involved in a transaction are recovered to the
> same point in time, you can't consider a transaction to be properly atomic.
> If application code is designed expecting atomicity, there are going to be
> tears. Perhaps only when disaster strikes, but the risk is there.
> 
> I think KIP-98 is interesting, but I wouldn't equate what it provides
> with the transactions provided by resource managers with traditional 
> transaction
> logging. It's not better or worse, just different. If you tried to migrate
> from a previous transactional system to Kafka transactions, I think you'd
> better have procedures for reconciliation with the other resource managers.
> Better still, don't build applications that are so fragile. The principle
> of dumb pipes and smart endpoints is good in my view.
> 
> If you're creating a global transaction containing two or more resource
> managers and using two-phase commit, it's very important that all of the
> resource managers maintain a coherent view of the sequence of events. If any
> part fails due to a software or hardware failure, once the system is
> recovered, nothing must be forgotten. If you look at how presume-abort
> works, you'll see how important this is.
> 
> Kafka doesn't really fit very nicely in this kind of environment because of
> the way that it writes lazily to disk. The theory is that you must avoid at 
> all
> costs having an uncontrolled shutdown of an entire cluster because you'll lose
> a little data at the end of the logs. So, if you are coordinating Kafka and a
> relational database in a global transaction, it's theoretically possible that
> a crashed Kafka would be a little forgetful while a crashed database would 
> not.
> The database would be an order of magnitude or more slower because of the way
> its recovery logs are handled, but it would not be forgetful in the same way.
> 
> You get exactly the same kind of worries when you implement some kind of
> asynchronous replication for disaster recovery, even if all of the resource
> managers force all of their log writes to disk eagerly. The replica at the DR
> site is slightly behind the primary site, so if you have to recover from an
> outage and switch to the DR site, it can be considered to be slightly 
> forgetful
> about the last few moments before the outage. This is why a DR plan usually 
> has
> some concept of compensation or reconciliation to make good any forgotten 
> work.
> 
> In summary, I think Kafka would have to change in ways which would negate many
> of its good points in order to support XA transactions. It would be better to
> design applications to be resilient to message duplication and loss, rather
> than tightly coupling resource managers and ending up with something fragile.
> 
> Don't get me wrong. This is not an anti-Kafka rant. I just work with people
> used to traditional transactional systems, making use of Kafka for business
> applications, and it's important to understand the concepts on both sides
> and make sure your assumptions are valid.
> 
> Andrew Schofield
> IBM Watson and Cloud Platform
> 
> 
>> From: Michael Pearce <michael.pea...@ig.com>
>> Sent: 09 December 2016 06:19
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and Transactional 
>> Messaging
>>     
>> 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.

Reply via email to