Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-24 Thread Matthias J. Sax
Michael, I don't think we "hand waved" this question, but as you mentioned anyway, without an implementation it's hard (or even impossible) to quantify the probability. Nevertheless, I'll try: First, I want to refer to the "Error Handling" section in the KIP: https://docs.google.com/document/d/1p

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-24 Thread Michael Noll
Ah, I think the use of the word "regression" was misleading. Let me try to rephrase: At the moment it is unclear to me whether the scenario where we need to rebuild a state store from scratch will be as likely, a little bit more likely, much more likely, or very, very much more likely to happen t

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-22 Thread Sriram Subramanian
This is a completely new feature which is controlled by a config. It would be a regression if you upgraded streams and saw a different behavior. That would not happen in this case. This is similar to how we are introducing idempotent producer in core kafka with a config to turn it on. There is no g

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-22 Thread Michael Noll
Separate reply because I'm switching topics (no pun intended). :-) One impact of the Streams EOS design is how we handle failures in Streams. In the EOS design we have effectively three main failure categories, as far as I understand: 1. Transient failures (which we now e.g. handle with infinite

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-22 Thread Michael Noll
I second Eno's concern regarding the impact of Streams EOS on state stores. > We do a full recovery today and the EOS proposal will not make this any worse. Yes, today we do a full state store recovery under certain failures. However, I think the point (or perhaps: open question) is that, with t

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-21 Thread Sriram Subramanian
To add to this discussion, I do think we should think about this in increments. We do a full recovery today and the EOS proposal will not make this any worse. Using store snapshot is a good option to avoid store recovery in the future but as Eno points out, all pluggable stores would need to have t

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-21 Thread Matthias J. Sax
Thanks for your feedback Eno. For now, I still think that the KIP itself does not need to talk about this in more detail, because we apply the same strategy for EoS as for non-EoS (as of 0.10.2). Thus, in case of a clean shutdown, we write the checkpoint file for a store and thus know we can reus

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-20 Thread Eno Thereska
Hi Matthias, I'd like to see some more info on how you propose to handle transactions that involve state stores in the KIP itself. The design doc has info about various optimisations like RocksDb snapshots and transactions and such, but will there be a user-visible interface that indicates whet

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-15 Thread Matthias J. Sax
Just a quick follow up: Our overall proposal is, to implement KIP-129 as is as a “Stream EoS 1.0” version. The raised concerns are all valid, but hard to quantify at the moment. Implementing KIP-129, that provides a clean design, allows us to gain more insight in the performance implications. This

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-15 Thread Matthias J. Sax
Hi, I want to pick up this thread again. As there are some concerns about the "producer per task" design, we did write up an alternative "producer per thread" design and discuss pros/cons of both approaches: https://docs.google.com/document/d/1CfOJaa6mdg5o7pLf_zXISV4oE0ZeMZwT_sG1QWgL4EE Looking

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-10 Thread Damian Guy
Hi Matthias, Thanks for the response. I agree with you regarding the use of PartitionGrouper to reduce the number of tasks. It would be good to have an idea of any additional load on the brokers as we increase the number of tasks and therefore producers. Thanks, Damian On Wed, 8 Mar 2017 at 01:4

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-07 Thread Matthias J. Sax
Damian, Jun, Thanks for your input. About Performance test: I can follow up with more performance tests using more partitions and also collecting broker metrics. However, I want to highlight again, that even if 1000+ partitions would be problematic, one can simply implement PartitionGrouper in

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-07 Thread Jun Rao
Hi, Guozhang, Thanks for the KIP. A couple of comments. 1. About the impact on producer batching. My understanding is that typically different sub-topologies in the same task are publishing to different topics. Since the producer batching happens at the topic/partition level, using a producer per

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-07 Thread Damian Guy
Hi Matthias, Thanks. The perf test is a good start but I don't think it goes far enough. 100 partitions is not a lot. What happens when there are thousands of partitions? What is the load on the brokers? How much more memory is used by the Streams App etc? Thanks, Damian On Tue, 7 Mar 2017 at 03

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-06 Thread Matthias J. Sax
Hi, I want to give a first respond: 1. Producer per task: First, we did some performance tests, indicating that the performance penalty is small. Please have a look here: https://docs.google.com/spreadsheets/d/18aGOB13-ibwHJl5VE27HnHBlNkZAIeLF9yQ99FDxkuM/edit?usp=sharing For the test, we ran

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-03 Thread Eno Thereska
Another question: The KIP doesn’t exactly spell out how it uses the idempotence guarantee from KIP-98. It seems that only the transactional part is needed. Or is the idempotence guarantee working behind the scenes and helping for some scenarios for which it is not worthwhile aborting a transac

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-02 Thread Jay Kreps
I second the concern on with the one producer per task approach. At a high-level it seems to make sense but I think Damian is exactly right that that cuts against the general design of the producer. Many people have high input partition counts and will have high task counts as a result. I think pro

Re: [DISCUSS] KIP-129: Kafka Streams Exactly-Once Semantics

2017-03-01 Thread Damian Guy
Hi Guozhang, Thanks for the KIP! This is an important feature for Kafka Streams and will help to unlock a bunch of use cases. I have some concerns/questions: 1. Producer per task: I'm worried about the overhead this is going to put on both the streams app and the Kafka Brokers. You can eas