ignite-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Maxim Muzafarov <maxmu...@gmail.com>
Subject [DISCUSSION] Hot cache backup
Date Wed, 27 Feb 2019 16:53:03 GMT
Igniters,

Some of the stores with which the Apache Ignite is often compared has
a feature called Snapshots [1] [2]. This feature provides an
eventually consistent view on stored data for different purposes (e.g.
moving data between environments, saving a backup of data for the
further restore procedure and so on). The Apache Ignite has all
opportunities and machinery to provide cache and\or data region
snapshots out of the box but still don't have them.

This issue derives from IEP-28 [5] on which I'm currently working on
(partially described in the section [6]). I would like to solve this
issue too and make Apache Ignite more attractive to use on a
production environment. I've haven't investigated in-memory type
caches yet, but for caches with enabled persistence, we can do it
without any performance impact on cache operations (some additional IO
operations are needed to copy cache data to backup store, copy on
write technique is used here). We just need to use our DiscoverySpi,
PME and Checkpointer process the right way.

For the first step, we can store all backup data on each of cache
affinity node locally. For instance, the `backup\snapshotId\cache0`
folder will be created and all `cache0` partitions will be stored
there for each local node for the snapshot process with id
`snapshotId`. In future, we can teach nodes to upload snapshotted
partitions to the one remote node or cloud.

--

High-level process overview

A new snapshot process is managed via DiscoverySpi and
CommunicationSpi messages.

1. The initiator sends a request to the cluster (DiscoveryMessage).
2. When the node receives a message it initiates PME.
3. The node begins checkpoint process (holding write lock a short time)
4. The node starts to track any write attempts to the snapshotting
partition and places the copy of original pages to the temp file.
5. The node performs merge the partition file with the corresponding delta.
6. When the node finishes the backup process it sends ack message with
saved partitions to the initiator (or the error response).
7. When all ack messages received the backup is finished.

The only problem here is that when the request message arrives at the
particular node during running checkpoint PME will be locked until it
ends. This is not good. But hopefully, it will be fixed here [4].

--

Probable API

>From the cache perspective:

IgniteFuture<IgniteSnapshot> snapshotFut =
    ignite.cache("default")
        .shapshotter()
        .create("myShapshotId");

IgniteSnapshot cacheSnapshot = snapshotFut.get();

IgniteCache<K, V> copiedCache =
    ignite.createCache("CopyCache")
        .withConfiguration(defaultCache.getConfiguration())
        .loadFromSnapshot(cacheSnapshot.id());

>From the command line perspective:

control.sh --snapshot take cache0,cache1,cache2

--

WDYT?
Will it be a useful feature for the Apache Ignite?


[1] https://geode.apache.org/docs/guide/10/managing/cache_snapshots/chapter_overview.html
[2] https://docs.datastax.com/en/cassandra/3.0/cassandra/operations/opsBackupTakesSnapshot.html
[3] http://apache-ignite-developers.2346864.n4.nabble.com/Data-Snapshots-in-Ignite-td4183.html
[4] https://issues.apache.org/jira/browse/IGNITE-10508
[5] https://cwiki.apache.org/confluence/display/IGNITE/IEP-28%3A+Cluster+peer-2-peer+balancing
[6] https://cwiki.apache.org/confluence/display/IGNITE/IEP-28%3A+Cluster+peer-2-peer+balancing#IEP-28:Clusterpeer-2-peerbalancing-Checkpointer

Mime
View raw message