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 events in Kafka
   2. You can create queryable materialized views/indexes derived off of
   these events in Kafka Streams, which I believe would include what in event
   sourcing is called aggregates.

If you change the logic by which aggregates are computed off the raw events
you would rerun the streams app that derived it to recompute the derived
state from the event log. Since this is Kafka this can be done in a
blue/green fashion where you keep the old version of the app running and
start a new version in parallel which recomputes the state from scratch,
then cut over to the newly derived version of your app. In other words the
state is part of the app (which may be split over many instances) not part
of some remote db shared by many versions of the app.

The two things that I have heard are missing:

   1. Your query indexes in the streams app are only eventually consistent
   with writes to the Kafka topic. There is no read-after-write consistency.
   2. You can't compute individual aggregates on their own. That is if you
   have an Order aggregate you can recompute the set of orders from scratch
   but you can't recompute just Order 12453.
   3. The query functionality in streams is quite limited.

Of these two I think (1) and (3) are real limitations for many use cases.

I am actually not sure if (2) is a problem. In general, I do think that if
you change your logic for deriving aggregates from events, the only way to
correctly regenerate your state is to recompute off the event log, right?
Doing this in a one-off way for just some entities may result in derived
state that doesn't match the code and input events you have in odd ways.
Anyhow not sure if that is what you are saying is missing but other people
have said that.

Does that match what you are saying? I actually am too ignorant of this
area and its terminology to fully understand what you mean by the three
examples you give.

-Jay


On Fri, Jul 21, 2017 at 6:51 AM, Chris Richardson <ch...@chrisrichardson.net
> wrote:

> Hi,
>
> I like Kafka but I don't understand the claim that it can be used for Event
> Sourcing (here <http://microservices.io/patterns/data/event-sourcing.html>
> and here <https://martinfowler.com/eaaDev/EventSourcing.html>)
>
> One part of the event sourcing is the ability to subscribe to events
> published by aggregates and clearly Kafka works well there.
>
> But the other part of Event Sourcing is "database" like functionality,
> which includes
>
>    - findEventsByPrimaryKey() - needed to be able to reconstruct an
>    aggregate from its events - the essence of event sourcing
>    - Atomic updates -  for updating aggregates  - findEventsByPrimaryKey()
>    - business logic - insertNewEvents()) in order to handle this kind of
>    scenario.
>
> The approach we have taken is to implement event sourcing using a database
> and Kafka.
> For instance: see
> https://blog.eventuate.io/2016/10/06/eventuate-local-
> event-sourcing-and-cqrs-with-spring-boot-apache-kafka-and-mysql/
>
> Chris
>
> --
> Learn microservices - http://learnmicroservices.io
> Microservices application platform http://eventuate.io
>
>
> On Fri, Jul 21, 2017 at 12:25 AM, José Antonio Iñigo <
> joseantonio.in...@gmail.com> wrote:
>
> > Hi everybody,
> >
> > I have been struggling with this problem for quite a while now, resorting
> > to stackoverflow
> > <https://stackoverflow.com/questions/45144429/event-
> > sourcing-apache-kafka-kafka-streams-how-to-assure-atomicity-transa>
> > for some help with no success. I am hoping to that here I'll get a more
> > authoritative answer from experienced Kafka users.
> >
> > This is the summary of my problem:
> >
> > - I am developing an application based on Spring Boot Microservices for a
> > shopping domain.
> > - I want to use Event Sourcing, having Kafka to register the events and
> > Kafka Streams API stores to materialize the views.
> > - To simplify the scenario we'll consider only two domains: Orders and
> > Products.
> > - The conflicting part:
> >    1) OrderService publishes an OrderPlaced event indicating a productId
> > and the quantity.
> >    2) ProductService consumes the event and queries (with an interactive
> > query) its Kafka Streams Store (ProductsStore) to check the availability
> of
> > the product. If there is availabilty it publishes a ProductReserved event
> > with productId and quantity:
> >
> > if("OrderPlaced".equals(event.get("eventType"))){
> >
> >     Order order = new Order();
> >     order.setId((String)event.get("orderId"));
> >     order.setProductId((Integer)(event.get("productId")));
> >     order.setUid(event.get("uid").toString());
> >
> >     // QUERY PRODUCTSTOCK TO CHECK AVAILABILITY
> >     Integer productStock = getProductStock(order.getProductId());
> >
> >     if(productStock > 0) {
> >         Map<String, Object> event = new HashMap<>();
> >         event.put("name", "ProductReserved");
> >         event.put("orderId", order.getId());
> >         event.put("productId", order.getProductId());
> >
> >         // WRITES A PRODUCT RESERVED EVENT TO orders topic
> >
> > orderProcessor.output().send(MessageBuilder.withPayload(event).build(),
> > 500);
> >     }else{
> >         //XXX CANCEL ORDER
> >     }
> > }
> >
> >    Then ProductService consumes its own event in a Kafka Streams
> processor
> > to update the stock of the product in the ProductsStore.
> >
> > KStream<Integer, JsonNode> stream = kStreamBuilder.stream(integerSerde,
> > jsonSerde, "orders");
> > stream.filter(...).groupByKey().reduce((...) -> {...}, "ProductsStock");
> >
> >    3.1) Suppose that in 1) two orders were placed simultaneously for the
> > same product and there is only stock for one of them
> >    3.2) ProductService would process the first one, the stock is ok and
> > would publish the ProductReserved event.
> >    3.3) We can't assure that ProductService will always process in the
> > Kafka Streams processor the order1 ProductReserved event to update
> > ProductsStore before the order2 OrderCreated is processed. Then in cases
> > ProductService will generate a ProductReserved for order2 incorrectly,
> > generating an inconsistency.
> >
> > IMPORTANT: You can find the detailed description, with code and the
> events
> > that are published and consumed in the previously referenced
> stackoverflow
> > question.
> >
> > After so much thinking and looking up online I haven't found a single
> place
> > where I can get a clear way to deal with Event Sourcing with Kafka+Kafka
> > Streams solving the problem of atomicity.
> >
> > I'd really appreciate if someone could propose a solution for this.
> >
> > Regards
> > Jose
> >
>

Reply via email to