Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-25 Thread José Antonio Iñigo
I didn't say anything... my producer wasn't working properly, messages are arriving now. Finally I got it working :-) Thanks so much!! 2017-07-25 12:23 GMT+02:00 José Antonio Iñigo : > Hi Ben, > > now I can see what you meant previously about using a Transformer. I was > following a wrong approa

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-25 Thread José Antonio Iñigo
Hi Ben, now I can see what you meant previously about using a Transformer. I was following a wrong approach dividing the processing between a Listener and a Stream processor. There's only one thing left that I don't know how to work out, this a draft of my code based on yours: @Bean @SuppressWar

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-24 Thread Ben Stopford
No worries Jose ;-) So there are a few ways you could do this, but I think it’s important that you manage a single “stock level” state store, backed by a changelog. Use this for validation, and keep it up to date at the same time. You should also ensure the input topic(s) are partitioned by produc

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-22 Thread José Antonio Iñigo
Hi Garret, At the moment, to simplify the problem I only have one topic, orders, where I add products and decrement them based on ProductAdded and ProductReserved events. Yeaterday I was reading about EoS but I don't know if it'll solve the problem. Dividing the query-update in two steps means th

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-22 Thread José Antonio Iñigo
My previous mail was in fact addressed to Ben, not Chris, sorry for the mistake. Regards On Sat, 22 Jul 2017 at 00:15, José Antonio Iñigo < joseantonio.in...@gmail.com> wrote: > Hi Chris, > > *"if I understand your problem correctly, the issue is that you need to* > > > *decrement the stock coun

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Garrett Barton
Could you take in both topics via the same stream? Meaning don't do a kafka streams join, literally just read both streams. If KStream cant do this, dunno haven't tried, then simple upstream merge job to throw them into 1 topic with same partitioning scheme. I'd assume you would have the products

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread José Antonio Iñigo
Hi Chris, *"if I understand your problem correctly, the issue is that you need todecrement the stock count when you reserve it, rather than splitting it* *into a second phase."* That's exactly the problem, I would need to: 1) Read the OrderPlaced event from Kafka in ProductService... 2) ...que

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Jay Kreps
Hey Chris, I heard a similar complaint from a few people. I am quite ignorant about event sourcing and don't feel I understand the relationship fully but I am interested in understanding a little better what you are saying. I think we see the world this way: 1. You store the log of primary ev

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Ben Stopford
Hi Jose If I understand your problem correctly, the issue is that you need to decrement the stock count when you reserve it, rather than splitting it into a second phase. You can do this via the DSL with a Transfomer. There's a related example below. Alternatively you could do it with the processor

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Debasish Ghosh
Kafka has quite a few tricks up its sleeve that can help implementing event sourced systems .. - application state - one of the things that u may want to do in an event sourced system is manage and query the state of the application. If you use Kafka Streams, you get the full functionalit

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Michal Borowiecki
With Kafka Streams you get those and atomicity via Exactly-once-Semantics. Michał On 21/07/17 14:51, Chris Richardson wrote: Hi, I like Kafka but I don't understand the claim that it can be used for Event Sourcing (here and here

Re: Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread Chris Richardson
Hi, I like Kafka but I don't understand the claim that it can be used for Event Sourcing (here and here ) One part of the event sourcing is the ability to subscribe to events published

Event sourcing with Kafka and Kafka Streams. How to deal with atomicity

2017-07-21 Thread José Antonio Iñigo
Hi everybody, I have been struggling with this problem for quite a while now, resorting to stackoverflow for some help with no success. I am hoping to that here I'll get a more a