[infinispan-dev] Infinispan and change data capture

Gustavo Fernandes gustavo at infinispan.org
Thu Dec 15 05:18:37 EST 2016


On Thu, Dec 15, 2016 at 9:54 AM, Emmanuel Bernard <emmanuel at hibernate.org>
wrote:

> The goal is as followed: allow to collect all changes to push them to
> Debezium and thus Kafka.
>
> This need does not require to remember all changes since the beginning of
> time in Infinispan. Just enough to:
> - let Kafka catchup assuming it is the bottleneck
> - let us not lose a change in Kafka when it happened in Infinispan
> (coordinator, owner, replicas dying)
>
> The ability to read back history would then be handled by the Debezium /
> Kafka tail, not infinispan itself.
>
>
Having an embedded Debezium connector pushing everything to Kafka sounds
cool, but what impact would it bring to the other stream consumers:

* Remote listeners, which is supported in several clients apart from Java
* Continuous Queries (the same)
* Spark Stream
* Other eventual 3rd party stream processors: Apache Flick, Storm, etc.



> Check my email on this tread from Dec 9th.
>
> > On 12 Dec 2016, at 16:13, Sanne Grinovero <sanne at infinispan.org> wrote:
> >
> > I'm reading many clever suggestions for various aspects of such a
> > system, but I fail to see a clear definition of the goal.
> >
> >> From Randall's opening email I understand how MySQL does this, but
> > it's an example and I'm not sure which aspects are implementation
> > details of how MySQL happens to accomplish this, and which aspects are
> > requirements for the Infinispan enhancement proposals.
> >
> > I remember a meeting with Manik Surtani, Jonathan Halliday and Mark
> > Little, whose outcome was a general agreement that Infinispan would
> > eventually need both tombstones and versioned entries, not just for
> > change data capture but to improve several other aspects;
> > unfortunately that was in December 2010 and never became a priority,
> > but the benefits are clear.
> > The complexities which have put off such plans lie in the "garbage
> > collection", aka the need to not grow the history without bounds, and
> > have to drop or compact history.
> >
> > So I'm definitely sold on the need to add a certain amount of history,
> > but we need to define how much of this history is expected to be held.
> >
> > In short, what's the ultimate goal? I see two main but different
> > options intertwined:
> > - allow to synchronize the *final state* of a replica
> > - inspect specific changes
> >
> > For the first case, it would be enough for us to be able to provide a
> > "squashed history" (as in Git squash), but we'd need to keep versioned
> > shapshots around and someone needs to tell you which ones can be
> > garbage collected.
> > For example when a key is: written, updated, updated, deleted since
> > the snapshot, we'll send only "deleted" as the intermediary states are
> > irrelevant.
> > For the second case, say the goal is to inspect fluctuations of price
> > variations of some item, then the intermediary states are not
> > irrelevant.
> >
> > Which one will we want to solve? Both?
> > Personally the attempt of solving the second one seems like a huge
> > pivot of the project, the current data-structures and storage are not
> > designed for this. I see the value of such benefits, but maybe
> > Infinispan is not the right tool for such a problem.
> >
> > I'd prefer to focus on the benefits of the squashed history, and have
> > versioned entries soon, but even in that case we need to define which
> > versions need to be kept around, and how garbage collection /
> > vacuuming is handled.
> > This can be designed to be transparent to the client: handled as an
> > internal implementation detail which we use to improve performance of
> > Infinispan itself, or it can be exposed to clients to implement change
> > data capture, but in this case we need to track which clients are
> > still going to need an older snapshot; this has an impact as clients
> > would need to be registered, and has a significant impact on the
> > storage strategies.
> >
> > Within Kafka the log compaction strategies are configurable; I have no
> > experience with Kafka but the docs seem to suggest that it's most
> > often used to provide the last known value of each key. That would be
> > doable for us, but Kafka also does allow optionally for wider scope
> > retention strategies: can we agree that that would not be an option
> > with Infinispan? If not, these goals need to be clarified.
> >
> > My main concern is that if we don't limit the scope of which
> > capabilities we want Infinispan to provide, it risks to become the
> > same thing as Kafka, rather than integrating with it. I don't think we
> > want to pivot all our storage design into efficiently treating large
> > scale logs.
> >
> > In short, I'd like to see an agreement that analyzing e.g.
> > fluctuations in stock prices would be a non-goal, if these are stored
> > as {"stock name", value} key/value pairs. One could still implement
> > such a thing by using a more sophisticated model, just don't expect to
> > be able to see all intermediary values each entry has ever had since
> > the key was first used.
> >
> > # Commenting on specific proposals
> >
> > On ID generation: I'd definitely go with IDs per segment rather than
> > IDs per key for the purpose of change data capture. If you go with
> > independent IDs per key, the client would need to keep track of each
> > version of each entry, which has an high overhead and degree of
> > complexity for the clients.
> > On the other hand, we already guarantee that each segment is managed
> > by a single primary owner, so attaching the "segment transaction id"
> > to each internal entry being changed can be implemented efficiently by
> > Infinispan.
> > Segment ownership handoff needs to be highly consistent during cluster
> > topology changes, but that requirement already exists; we'd just need
> > to make sure that this monotonic counter is included during the
> > handoff of the responsibility as primary owner of a segment.
> >
> > Thanks,
> >
> > Sanne
> >
> >
> >
> >
> >
> > On 12 December 2016 at 13:58, Gustavo Fernandes <gustavo at infinispan.org>
> wrote:
> >>
> >>
> >> On Fri, Dec 9, 2016 at 9:13 AM, Radim Vansa <rvansa at redhat.com> wrote:
> >>>
> >>> But introducing globally monotonous counter means that
> >>> there will be a single contention point.
> >>
> >>
> >> I wonder if the trade-off of Flake Ids [1] could be acceptable for this
> use
> >> case.
> >>
> >> [1] http://yellerapp.com/posts/2015-02-09-flake-ids.html
> >>
> >>
> >> _______________________________________________
> >> infinispan-dev mailing list
> >> infinispan-dev at lists.jboss.org
> >> https://lists.jboss.org/mailman/listinfo/infinispan-dev
> > _______________________________________________
> > infinispan-dev mailing list
> > infinispan-dev at lists.jboss.org
> > https://lists.jboss.org/mailman/listinfo/infinispan-dev
>
>
> _______________________________________________
> infinispan-dev mailing list
> infinispan-dev at lists.jboss.org
> https://lists.jboss.org/mailman/listinfo/infinispan-dev
>
-------------- next part --------------
An HTML attachment was scrubbed...
URL: http://lists.jboss.org/pipermail/infinispan-dev/attachments/20161215/710faace/attachment.html 


More information about the infinispan-dev mailing list