You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 32 Next »

IDIEP-59
Author
SponsorAnton Vinogradov
Created14.10.2020 
StatusDRAFT


Motivation

Many use-cases build on observation and processing changed records.

These use-cases include but not limited by

  • Export data into some warehouse, full-text search system, or distributed log system.
  • Online statistics and analytics
  • Wait and respond to some specific events or data changes.

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:

  • CQ requires data to be sent over the network
  • CQ parts (filter, transformer) live inside server node JVM so issues in it may affect server node stability.
  • Slow CQ listener leads to increasing of the memory consumption of the server node.
  • Fails of the CQ listener lead to the loss of the events.

The convenient solution should be:

  • Independence from the server node process (JVM) - issues and failures of the consumer shouldn't lead to server node instability.
  • Notification guarantees and failover - i.e. track and save a pointer to the last consumed record. Continue notification from this pointer in case of restart.
  • Resilience for the consumer - it's not an issue when a consumer temporarily consumes slower than data appear.

Description:

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.


Design choices:

  • CDC application works as a separate process.
  • CDC relies on the existing Ignite mechanism - WAL.
  • IEP Scope - deliver local data change events to a local consumer.
  • CDC keeps consumer offset in a special file.
    WAL process will start from this offset on restart.
  • To prevent interference between the WAL archive process and CDC Ignite will create a hard link to the newly created segment in a special folder.
    After success processing, CDC will delete this link.
    Note, data will be removed from the disk only after CDC and Ignite will remove the link to a segment from both corresponding folders.
  • To manage minimal event gap new configuration timeout introduced - WalForceArchiveTimeout.
  • Flag to distinguish DataEntry on primary and backup nodes introduced.
  • All public APIs market with @IgniteExperimental to be able to improve it based on real-world usage feedback.
  • CDC consumer will be notified about binary metadata changes (Phase 2).
  • Configuration parameter "Maximum CDC folder size" will be implemented to prevent disk volume exceed.
  • CDC folder resolved using the logic as Ignite node does.
  • CDC application should be restarted by the OS mechanism in case of any error (destination unavailability, for example)
  • Initially, single CDC consumer supported. Support of several concurrently running consumers will be implemented in Phase2.


Public API:


Java 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

  • CDC utility will be started and automatically restarted in the case of failure by the OS or some external tools to provide stable change event processing.
  • CDC feature may be used for the deployment that has WAL only.
  • At the start of the CDC first consumed event will be the first event available in the WAL archive.
  • The lag between the record change and CDC consumer notification will depend on segment archiving timeout and requires additional configuration from the user.
  • CDC failover depends on the WAL archive segment count. If the CDC application will be down a relatively long time it possible that Ignite deletes certain archive segments,
    therefore consumer can't continue to receive changed records and must restart from the existing segments.

Online (real-time) CDC

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.

User interface:

  1. IgniteConfiguration#cdcConsumer - implementation of the CdcConsumer interface.
  2. IgniteConfiguration#cdcBufSize - size of the buffer used by CDC to store captured changes. Default is (walSegCount * walSegSize), for the default values it is 640MB.
  3. Logs: 
    1. Initialization info.
    2. Switch between working modes.
  4. metrics: 
    1. Ordinary CDC metrics (count of captured WAL segments and entries).
    2. Current working mode.
    3. Used buffer space.
    4. Lag between buffer and WAL archive (segments).
    5. Lag between writing to WAL and capturing by CDC (milliseconds).
    6. Last captured WALPointer.

Segments:

Note, there is a confusion of using “segment” word:

  1. WAL segments are represented as numerated files. Size of WAL segments is configured with DataStorageConfiguration#walSegmentSize.
  2. ReadSegment is a slice of the mmap WAL segment. It contains WAL records to sync with the actual file. Size of the segment differs from time to time and its maximum can be configured with DataStorageConfiguration#walBuffSize.

CdcWorker:

CdcWorker is a thread responsible for collecting WAL records, transforming them into cdc events, submitting them to the CdcConsumer. The worker has 2 modes:

  1. BUFFER_MODE - consumes WAL records from the CdcBufferQueue, that is filled directly from the WAL manager.
  2. ARCHIVE_MODE - consumes WAL records from archived WAL segments.
    1. Note, that the CdcBufferQueue is being filled in background in this mode.

Initialization:

  1. CdcWorker initialized with CdcConsumerState#loadWalState.
  2. Initial mode is ARCHIVE_MODE. It switches to the CdcBufferQueue after:
    1. The loaded pointer is not reached in the archive.
    2. OR the head of the buffer queue is less than the loaded pointer. 

Capturing from the buffer (wal-sync-thread):

  1. In wal-sync-thread (the only reader of mmap WAL), under the lock that synchronizes preparing ReadSegment and rolling the WAL segment, to guarantee there are no changes in the underlying buffer.
  2. Offers a deep copy of flushing ReadSegments to the CdcWorker.
  3. CdcWorker checks remaining capacity and the buffer size.
  4. If the size fits the capacity then store the offered buffer data into the Queue. 
  5. Otherwise: 
    1. Remove from the queue tail segments to free space for the offered buffer.
    2. Store the head of the offered buffer as nextHead (WALPointer).
    3. It captures data from the Queue while nextHead is not reached.
    4. Switch to the ARCHIVE_MODE.

Body loop (cdc-worker-thread):

  1. BUFFER_MODE:
    1. Polls the Queue, transforms ReadSegment data to Iterator<CdcEvent>, pushes them to CdcConsumer.
    2. Optimization: transform segment buffers to CDC events in background (to reduce the buffer usage). CdcConsumer should be async then?
  2. ARCHIVE_MODE:
    1. Similar to CdcMain - await archived segments.
    2. Submits the read WAL records to the CdcConsumer.
    3. For every segment/record checks a condition to switch to the bufferMode:
      1. Check the loaded WALPointer after initialization.
      2. OR while nextHead is not reached.
  3. In both modes it persists CdcConsumerState. Policy for committing the progress: by WAL segment.

Discussion Links

http://apache-ignite-developers.2346864.n4.nabble.com/DISCUSSION-IEP-59-CDC-Capture-Data-Change-tc49677.html

Reference Links

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

https://docs.microsoft.com/ru-ru/sql/relational-databases/track-changes/track-data-changes-sql-server?view=sql-server-ver15

Tickets

key summary type created updated due assignee reporter priority status resolution

JQL and issue key arguments for this macro require at least one Jira application link to be configured

  • No labels