ID | IEP-59 |
Author | |
Sponsor | Anton Vinogradov |
Created | 14.10.2020 |
Status | DRAFT |
Many use-cases build on observation and processing changed records.
These use-cases include but not limited by
For now, such scenarios are hard to implement in Ignite.
The only solution that can help with it, for now, is a Continuous Query.
Disadvantages of the CQ in described scenarios:
The convenient solution should be:
Ignite CDC is a new utility that should be run on the server node host. CDC utility watches by the appearance of the WAL archive segments.
On the segment archiving, the utility iterates it using the existing WAL Iterator and notifications CDC Consumer of each record from the segment.
Public API:
/** Consumer of data change events. */ @IgniteExperimental public interface ChangeDataCaptureConsumer { /** * Starts the consumer. */ public void start(); /** * Handles entry changes events. * If this method return {@code true} then current offset will be stored * and ongoing notifications after CDC application fail/restart will be started from it. * * @param events Entry change events. * @return {@code True} if current offset should be saved on the disk * to continue from it in case any failures or restart. */ public boolean onEvents(Iterator<ChangeDataCaptureEvent> events); /** * Stops the consumer. * This methods can be invoked only after {@link #start()}. */ public void stop(); } /** * Event of single entry change. * Instance presents new value of modified entry. * * @see IgniteCDC * @see CaptureDataChangeConsumer */ @IgniteExperimental public interface ChangeDataCaptureEvent extends Serializable { /** * @return Key for the changed entry. */ public Object key(); /** * @return Value for the changed entry or {@code null} in case of entry removal. */ @Nullable public Object value(); /** * @return {@code True} if event fired on primary node for partition containing this entry. * @see <a href=" * https://ignite.apache.org/docs/latest/configuring-caches/configuring-backups#configuring-partition-backups"> * Configuring partition backups.</a> */ public boolean primary(); /** * Ignite split dataset into smaller chunks to distribute them across the cluster. * {@link ChangeDataCaptureConsumer} implementations can use {@link #partition()} to split changes processing * in the same way as it done for the cache. * * @return Partition number. * @see Affinity#partition(Object) * @see Affinity#partitions() * @see <a href="https://ignite.apache.org/docs/latest/data-modeling/data-partitioning">Data partitioning</a> * @see <a href="https://ignite.apache.org/docs/latest/data-modeling/affinity-collocation">Affinity collocation</a> */ public int partition(); /** * @return Version of the entry. */ public CacheEntryVersion version(); /** * @return Cache ID. * @see org.apache.ignite.internal.util.typedef.internal.CU#cacheId(String) * @see CacheView#cacheId() */ public int cacheId(); } /** * Entry event order. * Two concurrent updates of the same entry can be ordered based on {@link ChangeEventOrder} comparsion. * Greater value means that event occurs later. */ @IgniteExperimental public interface CacheEntryVersion extends Comparable<CacheEntryVersion>, Serializable { /** * Order of the update. Value is an incremental counter value. Scope of counter is node. * @return Version order. */ public long order(); /** @return Node order on which this version was assigned. */ public int nodeOrder(); /** * Cluster id is a value to distinguish updates in case user wants to aggregate and sort updates from several * Ignite clusters. {@code clusterId} id can be set for the node using * {@link GridCacheVersionManager#dataCenterId(byte)}. * * @return Cluster id. */ public byte clusterId(); /** @return Topology version plus number of seconds from the start time of the first grid node. */ public int topologyVersion(); /** * If source of the update is "local" cluster then {@code null} will be returned. * If updated comes from the other cluster using {@link IgniteInternalCache#putAllConflict(Map)} * then entry version for other cluster. * @return Replication version. * @see IgniteInternalCache#putAllConflict(Map) * @see IgniteInternalCache#removeAllConflict(Map) */ public CacheEntryVersion otherClusterVersion(); }
Risks and Assumptions
In case of using separate process for capturing data changes from WAL archives makes the lag between CDC event happens and consumer notified about it is relatively big. it's proposed to provide opportunity to capture data and notify consumers directly from Ignite process. It helps minimize the lag by cost of additional memory usage.
IgniteConfiguration#cdcConsumer
- implementation of the CdcConsumer interface.IgniteConfiguration#cdcBufSize
- size of the buffer used by CDC to store captured changes. Default is (walSegCount * walSegSize), for the default values it is 640MB.Note, there is a confusion of using “segment” word:
DataStorageConfiguration#walSegmentSize
.DataStorageConfiguration#walBuffSize
.CdcWorker is a thread responsible for collecting WAL records, transforming them into cdc events, submitting them to the CdcConsumer
. The worker has 2 modes:
bufferMode
- consumes WAL records from the CdcBufferQueue
, that is filled directly from the WAL manager.archiveMode
- consumes WAL records from archived WAL segments.CdcBufferQueue
is being filled in background in this mode.Initialization
CdcConsumerState#loadWalState
.archiveMode
. It switches to the CdcBufferQueue
after:Capturing from the buffer (wal-sync-thread)
ReadSegment
and rolling the WAL segment, to guarantee there are no changes in the underlying buffer.ReadSegments
to the CdcWorker
.CdcWorker
checks remaining capacity and the buffer sizeWALPointer
)Queue
while nextHead is not reached.archiveMode
.Body loop (cdc-worker-thread)
bufferMode
:Queue
, transforms ReadSegment data to Iterator<CdcEvent>
, pushes them to CdcConsumer
.CdcConsumer
should be async then?archiveMode
:CdcMain
- await archived segmentsCdcConsumer
.bufferMode
:WALPointer
after initializationCdcConsumerState
. Policy for committing the progress: by WAL segment.https://dev.mysql.com/doc/refman/8.0/en/mysqlbinlog.html
https://debezium.io/documentation/reference/1.2/architecture.html
https://jdbc.postgresql.org/documentation/head/replication.html
https://www.oracle.com/middleware/technologies/goldengate.html