DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
DUE TO SPAM, SIGN-UP IS DISABLED. Goto Selfserve wiki signup and request an account.
...
| ID | IEP-43 | ||||||||
| Author | |||||||||
| Sponsor | |||||||||
| Created |
| ||||||||
| Status |
|
| Table of Contents |
|---|
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.
Snapshot storage path allowed to be configured by IgniteConfiguration , by default IGNITE_HOME/work/snapshots directory used.
| Code Block | ||||||||
|---|---|---|---|---|---|---|---|---|
| ||||||||
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;
} |
| Code Block | ||||||||
|---|---|---|---|---|---|---|---|---|
| ||||||||
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 | ||||||||
|---|---|---|---|---|---|---|---|---|
| ||||||||
/**
* @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); |
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:
Ignite must have a capability to specify permissions to allow/disallow execution of cluster snapshot operation. The following permission must be supported:
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 | ||||||||
|---|---|---|---|---|---|---|---|---|
| ||||||||
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
|
...
// TBD
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 | ||||||||
|---|---|---|---|---|---|---|---|---|
| ||||||||
/** 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;
}
|
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:
...
0 - to length ).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.
...
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.
...
TransmissionHandler#chunkHandler).During the cluster snapshot operation, a node may crash for some reason.
...