Reviving this old thread, and as before I appreciate any help the Infinispan community
might provide. There definitely is interest in Debezium capturing the changes being made
to an Infinispan cluster. This isn’t as important when Infinispan is used as a cache, but
when Infinispan is used as a store then it is important for other apps/services to be able
to accurately keep up with the changes being made in the store.
On Jul 29, 2016, at 8:47 AM, Galder Zamarreño
<galder(a)redhat.com> wrote:
--
Galder Zamarreño
Infinispan, Red Hat
> On 11 Jul 2016, at 16:41, Randall Hauch <rhauch(a)redhat.com> wrote:
>
>>
>> On Jul 11, 2016, at 3:42 AM, Adrian Nistor <anistor(a)redhat.com> wrote:
>>
>> Hi Randall,
>>
>> Infinispan supports both push and pull access models. The push model is supported
by events (and listeners), which are cluster wide and are available in both library and
remote mode (hotrod). The notification system is pretty advanced as there is a filtering
mechanism available that can use a hand coded filter / converter or one specified in jpql
(experimental atm). Getting a snapshot of the initial data is also possible. But
infinispan does not produce a transaction log to be used for determining all changes that
happened since a previous connection time, so you'll always have to get a new full
snapshot when re-connecting.
>>
>> So if Infinispan is the data store I would base the Debezium connector
implementation on Infinispan's event notification system. Not sure about the other use
case though.
>>
>
> Thanks, Adrian, for the feedback. A couple of questions.
>
> You mentioned Infinispan has a pull model — is this just using the normal API to read
the entries?
>
> With event listeners, a single connection will receive all of the events that occur
in the cluster, correct? Is it possible (e.g., a very unfortunately timed crash) for a
change to be made to the cache without an event being produced and sent to listeners?
^ Yeah, that can happen due to async nature of remote events. However, there's the
possibility for clients, upon receiving a new topology, to receive the current state of
the server as events, see [1] and [2]
[1]
http://infinispan.org/docs/dev/user_guide/user_guide.html#client_event_li...
<
http://infinispan.org/docs/dev/user_guide/user_guide.html#client_event_li...
[2]
http://infinispan.org/docs/dev/user_guide/user_guide.html#client_event_li...
<
http://infinispan.org/docs/dev/user_guide/user_guide.html#client_event_li...
It is critical that any change event stream is consistent with the store, and the change
event stream is worthless without it. Only when the change event stream is an accurate
representation of what changed can downstream consumers use the stream to rebuild their
own perfect copy of the upstream store and to keep those copies consistent with the
upstream store.
So, given that the events are handled asynchronously, in a cluster how are multiple
changes to a single entry handled. For example, if a client sets entry <A,Foo>, then
a short time after that (or another) client sets entry <A,Bar>, is it guaranteed
that a client listening to events will see <A,Foo> first and <A,Bar> some time
later? Or is it possible that a client listening might first see <A,Bar> and then
<A,Foo>?
> What happens if the network fails or partitions? How does cross site replication
address this?
In terms of cross-site, depends what the client is connected to. Clients can now failover
between sites, so they should be able to deal with events too in the same as explained
above.
>
> Has there been any thought about adding to Infinispan a write ahead log or
transaction log to each node or, better yet, for the whole cluster?
Not that I'm aware of but we've recently added security audit log, so a
transaction log might make sense too.
Without a transaction log, Debezium would have to use a client listener with
includeCurrentState=true to obtain the state every time it reconnects. If Debezium just
included all of this state in the event stream, then the stream might contain lots of
superfluous or unnecessary events, then this impacts all downstream consumers by forcing
them to spend a lot of time processing changes that never really happened. So the only way
to avoid that would be for Debezium to use an external store to track the changes it has
seen so far so that it doesn’t include unnecessary events in the change event stream. It’d
be a shame to have to require this much infrastructure.
A transaction log would really be a great way to solve this problem. Has there been any
more thought about Infinispan using and exposing a transaction log? Or perhaps Infinispan
could record the changes in a Kafka topic directly?
(I guess if the Infinispan cache used relational database(s) as a cache store(s), then
Debezium could just capture the changes from there. That seems like a big constraint,
though.)
Thoughts?
Cheers,
>
> Thanks again!
>
>> Adrian
>>
>> On 07/09/2016 04:38 PM, Randall Hauch wrote:
>>> The Debezium project [1] is working on building change data capture
connectors for a variety of databases. MySQL is available now, MongoDB will be soon, and
PostgreSQL and Oracle are next on our roadmap.
>>>
>>> One way in which Debezium and Infinispan can be used together is when
Infinispan is being used as a cache for data stored in a database. In this case, Debezium
can capture the changes to the database and produce a stream of events; a separate process
can consume these change and evict entries from an Infinispan cache.
>>>
>>> If Infinispan is to be used as a data store, then it would be useful for
Debezium to be able to capture those changes so other apps/services can consume the
changes. First of all, does this make sense? Secondly, if it does, then Debezium would
need an Infinispan connector, and it’s not clear to me how that connector might capture
the changes from Infinispan.
>>>
>>> Debezium typically monitors the log of transactions/changes that are
committed to a database. Of course how this works varies for each type of database. For
example, MySQL internally produces a transaction log that contains information about every
committed row change, and MySQL ensures that every committed change is included and that
non-committed changes are excluded. The MySQL mechanism is actually part of the
replication mechanism, so slaves update their internal state by reading the master’s log.
The Debezium MySQL connector [2] simply reads the same log.
>>>
>>> Infinispan has several mechanisms that may be useful:
>>>
>>> • Interceptors - See [3]. This seems pretty straightforward and IIUC
provides access to all internal operations. However, it’s not clear to me whether a single
interceptor will see all the changes in a cluster (perhaps in local and replicated modes)
or only those changes that happen on that particular node (in distributed mode). It’s also
not clear whether this interceptor is called within the context of the cache’s
transaction, so if a failure happens just at the wrong time whether a change might be made
to the cache but is not seen by the interceptor (or vice versa).
>>> • Cross-site replication - See [4][5]. A potential advantage of this
mechanism appears to be that it is defined (more) globally, and it appears to function if
the remote backup comes back online after being offline for a period of time.
>>> • State transfer - is it possible to participate as a non-active member of
the cluster, and to effectively read all state transfer activities that occur within the
cluster?
>>> • Cache store - tie into the cache store mechanism, perhaps by wrapping an
existing cache store and sitting between the cache and the cache store
>>> • Monitor the cache store - don’t monitor Infinispan at all, and instead
monitor the store in which Infinispan is storing entries. (This is probably the least
attractive, since some stores can’t be monitored, or because the store is persisting an
opaque binary value.)
>>>
>>> Are there other mechanism that might be used?
>>>
>>> There are a couple of important requirements for change data capture to be
able to work correctly:
>>>
>>> • Upon initial connection, the CDC connector must be able to obtain a
snapshot of all existing data, followed by seeing all changes to data that may have
occurred since the snapshot was started. If the connector is stopped/fails, upon restart
it needs to be able to reconnect and either see all changes that occurred since it last
was capturing changes, or perform a snapshot. (Performing a snapshot upon
restart is very inefficient and undesirable.) This works as follows: the CDC connector
only records the “offset” in the source’s sequence of events; what this “offset” entails
depends on the source. Upon restart, the connector can use this offset information to
coordinate with the source where it wants to start reading. (In MySQL and PostgreSQL,
every event includes the filename of the log and position in that file. MongoDB includes
in each event the monotonically increasing timestamp of the transaction.
>>> • No change can be missed, even when things go wrong and components crash.
>>> • When a new entry is added, the “after” state of the entity will be
included. When an entry is updated, the “after” state will be included in the event; if
possible, the event should also include the “before” state. When an entry is removed, the
“before” state should be included in the event.
>>>
>>> Any thoughts or advice would be greatly appreciated.
>>>
>>> Best regards,
>>>
>>> Randall
>>>
>>>
>>> [1]
http://debezium.io
>>> [2]
http://debezium.io/docs/connectors/mysql/
>>> [3]
http://infinispan.org/docs/stable/user_guide/user_guide.html#_custom_inte...
>>> [4]
http://infinispan.org/docs/stable/user_guide/user_guide.html#CrossSiteRep...
>>> [5]
https://github.com/infinispan/infinispan/wiki/Design-For-Cross-Site-Repli...
>>>
>>>
>>> _______________________________________________
>>> infinispan-dev mailing list
>>>
>>> infinispan-dev(a)lists.jboss.org
>>>
https://lists.jboss.org/mailman/listinfo/infinispan-dev
>>
>
> _______________________________________________
> infinispan-dev mailing list
> infinispan-dev(a)lists.jboss.org
>
https://lists.jboss.org/mailman/listinfo/infinispan-dev
_______________________________________________
infinispan-dev mailing list
infinispan-dev(a)lists.jboss.org <mailto:infinispan-dev@lists.jboss.org>
https://lists.jboss.org/mailman/listinfo/infinispan-dev
<
https://lists.jboss.org/mailman/listinfo/infinispan-dev>