Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

IDIEP-43
Author
Sponsor
Created

  

Status

Status
colourGreen
titleACTIVEDONE


Table of Contents

Motivation

Motivation

The most of open-source distributed systems provide `cluster snapshots` functionality, but the Apache Ignite doesn't have such one. Cluster snapshots will allow users to copy their data from an active cluster and load it later on another, such as copying data from a production system into a smaller QA or development system. 

Description

Management

Configuration

Snapshot storage path allowed to be configured by IgniteConfiguration , by default IGNITE_HOME/work/snapshots  directory used.

Code Block
languagejava
themeConfluence
titleIgniteConfiguration#snapshotPath
collapsetrue
public class IgniteConfiguration {
    /**
     * Directory where will be stored all results of snapshot operations. If {@code null} then
     * relative {@link #DFLT_SNAPSHOT_DIRECTORY} will be used.
     */
    private String snapshotPath;
}

Create snapshot

[public] Java API

Code Block
languagejava
themeConfluence
titleIgniteSnapshot
collapsetrue
public interface IgniteSnapshot {
    /**
     * Create a consistent copy of all persistence cache groups from the whole cluster.
     *
     * @param name Snapshot name.
     * @return Future which will be completed when a process ends.
     */
    public IgniteFuture<Void> createSnapshot(String name);
}

...

Code Block
languagejava
themeConfluence
titleIgniteSnapshotManager#createRemoteSnapshot
collapsetrue
/**
 * @param parts Collection of pairs group and appropriate cache partition to be snapshot.
 * @param rmtNodeId The remote node to connect to.
 * @param partConsumer Received partition handler.
 * @return Future which will be completed when requested snapshot fully received.
 */
public IgniteInternalFuture<Void> createRemoteSnapshot(
    UUID rmtNodeId,
    Map<Integer, Set<Integer>> parts,
    BiConsumer<File, GroupPartitionId> partConsumer);

Snapshot Events

Ignite distributed events functionality allows user applications to receive notifications when some of the events occur in the distributed cluster environment. User must be able to get notified for snapshot operation executions within the cluster:

  • EVT_CLUSTER_SNAPSHOT_STARTED – the snapshot operation started on a server node.
  • EVT_CLUSTER_SNAPSHOT_FINISHED – the snapshot operation finished successfully.
  • EVT_CLUSTER_SNAPSHOT_FAILED – the snapshot operation interrupted due to the reason came in the additional message.

Snapshot Security

Ignite must have a capability to specify permissions to allow/disallow execution of cluster snapshot operation. The following permission must be supported:

  • ADMIN_SNAPSHOT_OPS – permission to control creation and cancellation cluster snapshot operations.

Restore snapshot (manually)

The snapshot procedure stores all internal files (binary meta, marshaller meta, cache group data files, and cache group configuration) the same directory structure way as the Apache Ignite does with preserving configured consistent node id.

...

Code Block
languagebash
themeConfluence
titleSnashot Directory Structure
collapsetrue
maxmuzaf@TYE-SNE-0009931 ignite % tree work
work
└── snapshots
    └── backup23012020
        ├── binary_meta
        │   ├── snapshot_IgniteClusterSnapshotSelfTest0
        │   ├── snapshot_IgniteClusterSnapshotSelfTest1
        │   └── snapshot_IgniteClusterSnapshotSelfTest2
        ├── db
        │   ├── snapshot_IgniteClusterSnapshotSelfTest0
        │   │   ├── cache-default
        │   │   │   ├── cache_data.dat
        │   │   │   ├── part-0.bin
        │   │   │   ├── part-2.bin
        │   │   │   ├── part-3.bin
        │   │   │   ├── part-4.bin
        │   │   │   ├── part-5.bin
        │   │   │   └── part-6.bin
        │   │   └── cache-txCache
        │   │       ├── cache_data.dat
        │   │       ├── part-3.bin
        │   │       ├── part-4.bin
        │   │       └── part-6.bin
        │   ├── snapshot_IgniteClusterSnapshotSelfTest1
        │   │   ├── cache-default
        │   │   │   ├── cache_data.dat
        │   │   │   ├── part-1.bin
        │   │   │   ├── part-3.bin
        │   │   │   ├── part-5.bin
        │   │   │   ├── part-6.bin
        │   │   │   └── part-7.bin
        │   │   └── cache-txCache
        │   │       ├── cache_data.dat
        │   │       ├── part-1.bin
        │   │       ├── part-5.bin
        │   │       └── part-7.bin
        │   └── snapshot_IgniteClusterSnapshotSelfTest2
        │       ├── cache-default
        │       │   ├── cache_data.dat
        │       │   ├── part-0.bin
        │       │   ├── part-1.bin
        │       │   ├── part-2.bin
        │       │   ├── part-4.bin
        │       │   └── part-7.bin
        │       └── cache-txCache
        │           ├── cache_data.dat
        │           ├── part-0.bin
        │           └── part-2.bin
        └── marshaller

17 directories, 30 files

Restore snapshot (automatic)

Restore the cache groups on the active cluster

...

Whole cluster restore

// TBD

Snapshot lifecycle

Sometimes the user is faced with the task of adding post-processing to the snapshot operation, for example, calculating the checksum of files, checking the consistency, etc. The same applies to the automatic restore procedure - the user should be able to check the consistency of files, checksums, etc. before restore them.

...

Code Block
languagejava
themeMidnight
titleSnapshot handlers API
collapsetrue
/** handler */
public interface SnapshotHandler<T> extends Extension {
    /** Snapshot handler type. */
    public SnapshotHandlerType type();

    /** Local processing of a snapshot operation. */
    public @Nullable T invoke(SnapshotHandlerContext ctx) throws Exception;

    /** Processing of results from all nodes. */
    public default void complete(String name, Collection<SnapshotHandlerResult<T>> results) throws Exception {
        for (SnapshotHandlerResult<T> res : results) {
            if (res.error() == null)
                continue;;

            throw new IgniteCheckedException("Snapshot handler has failed " +
                "[snapshot=" + name +
                ", handler=" + getClass().getName() +
                ", nodeId=" + res.node().id() + "].", res.error());
        }
    }
}

/** type */
public enum SnapshotHandlerType {
    /** Handler is called immediately after the snapshot is taken. */
    CREATE,

    /** Handler is called just before restore operation is started. */
    RESTORE
}

/** context */
public class SnapshotHandlerContext {
    SnapshotMetadata metadata;

    Collection<String> grps;

    ClusterNode locNode;
}

/** Result of local processing on the node. In addition to the result received from the handler, it also includes information about the error (if any) and the node on which this result was received. */
public class SnapshotHandlerResult<T> implements Serializable {
    T data;

    Exception err;

    ClusterNode node;
}

Snapshot requirements

  1. Users must have the ability to create a snapshot of persisted user data (in-memory is out of the scope).
  2. Users must have the ability to create a snapshot from the cluster under the load without cluster deactivation.
  3. The snapshot process must not block for a long time any of the user transactions (short-time blocks are acceptable).
  4. The snapshot process must allow creating a data snapshot on each node and transfer it to any of the remote nodes for internal cluster needs.
  5. The created snapshot at the cluster-level must be fully consistent from cluster-wide terms, there should not be any incomplete transactions inside.
  6. The snapshot of each node must be consistent – cache partitions, binary meta, etc. must not have unnecessary changes.

Snapshot overview

With respect to the cluster-wide snapshot operation, the process of creating a copy of user data can be split into the following high-level steps:

...

  1. A new checkpoint starts (forced by node or a regular one).
  2. Under the checkpoint write lock – fix cache partition length for each partition (copy from 0  - to length ).
  3. The task creates new on-heap collections with marshaller meta, binary meta to copy.
  4. The task starts copying partition files.
  5. The checkpoint thread:
    1. If the associated with task checkpoint is not finished - write a dirty page to the original partition file and to delta file.
    2. If the associated with task checkpoint is finished and partition file still copying – read an original page from the original partition file and copy it to the delta file prior to the dirty page write.
  6. If partition file is copied – start merging copied partition with its delta file.
  7. The task ends then all data successfully copied to the target directory and all cache partition files merged with its deltas.

Cluster-wide snapshot

The cluster-wide snapshot is an operation which executes local snapshot task on each node. To achieve cluster-wide snapshot consistency the Partition-Map-Exchange will be reused to block for a while all user transactions.

...

  1. The snapshot distributed process starts a new snapshot operation by sending an initial discovery message.
  2. The distributed process configured action initiates a new local snapshot task on each cluster node.
  3. The discovery event from the distributed process pushes a new exchange task to the exchange worker to start PME.
  4. While transactions are blocked (see onDoneBeforeTopologyUnlock) each local node forces the checkpoint thread and waits while an appropriate local snapshot task starts.
  5. The distributed process collects completion results from each cluster node:
    1. If there are no execution errors and no baseline nodes left the cluster – the snapshot created successfully.
    2. If there are some errors or some of the cluster nodes fails prior to complete local results – all local snapshots will be reverted from each node, the snapshot fails.

Remote snapshot

The internal components must have the ability to request a consistent (not cluster-wide, but locally) snapshot from remote nodes. This machinery is reused for IEP-28: Rebalance via cache partition files to transmit cache partition files. The process requests for execution of a local snapshot task on the remote node and collects back the execution results. The File Transmission protocol is used to transfer files from the remote node to the local node. 

...

  1. The node sends a request via CommunicaionSPI to the remote node with the map of cache group ids and required cache partitions.
  2. The remote node accepts the request and registers locally a new SnapshotFutureTask for execution.
  3. The task is started and sends independently cache partition files and cache delta files when they read.
  4. The node accepts cache partition files and accepts cache partition delta files by applying them on the fly to an appropriate partition (see TransmissionHandler#chunkHandler).
  5. When all requested cache groups and partitions received – the process completes successfully.
    If an error occurred during transmission the snapshot task stops and all temporary files deleted.

Crash recovery

During the cluster snapshot operation, a node may crash for some reason.

  • The crashed node must revert the local uncompleted snapshot at its startup.
  • If the crashed node hasn't complete local snapshot prior to the crash then all nodes affected by this cluster snapshot operation must delete their local snapshot results.

Limitations

  1. The whole cluster allowed only one cluster-wide snapshot operation per time.
  2. Encrypted caches currently not allowed due to required additional changes to merge cache partition with its delta file.
  3. The cache stop operation is not allowed during the ongoing cluster snapshot. An exception will be thrown to users on the cache stop attempt.
  4. The cluster snapshot operation will be stopped if some of the baseline nodes left or fail prior to reporting about their local snapshot completion. Partial local snapshots will be reverted.
  5. Datastructure and system caches are not included in the snapshot.

...