Andrey,

> What about primary/backup node data consistency.

Primary and backup partitions must be fully consistent in a snapshot,
additional recovery procedures not required. So, when we restore a
snapshot on the same topology everything will work right out of the
box - no WAL needed.

This is achieved by triggering PME [1]. Doing this we will get a point
in time when all started transactions are finished (on backups too)
and new ones are blocked on a new topology version. That's the point
in time when snapshot operation starts. And also this is a weak point
of the current solution since the process blocks all cluster
transactions for a while. See [2].

> I cant quite picture how persistence rebalancing works

The WAL-rebalance will not happen. The full rebalance will be used in
case of restoring a snapshot on different topology. For now, only
restoring on the same cluster topology (same baseline) will work fine,
other cases must be explicitly tested but in a theory, it will work
too.

> You analyze alternative snapshot solutions based on WAL?

Do you mean taking snapshots from the cluster without blocking
transactions (without PME)? It's not a trivial task from my point of
view. Currently, I have no design for it which can cover all corner
cases.


[1] 
https://cwiki.apache.org/confluence/display/IGNITE/%28Partition+Map%29+Exchange+-+under+the+hood
[2] 
https://github.com/apache/ignite/blob/master/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java#L1524

On Thu, 9 Apr 2020 at 00:52, Andrey Dolmatov <itdolma...@gmail.com> wrote:
>
> I would like to understand your solution deeper. Hope, that my questions
> are interesting not only for me:
>
>    - What about primary/backup node data consistency. I found, that [1]
>    Cassandra uses eventually consistent backups, so some backup data could
>    miss from snapshot. If I apply snapshot, would Ignite detect and rebalance
>    data to backup nodes?
>    - I cant quite picture how persistence rebalancing works, but according
>    to [2] it uses WAL logs. Snapshot doesn't contain WAL data, correct? Did
>    You analyze alternative snapshot solutions based on WAL?
>
> [1]
> https://docs.datastax.com/en/cassandra-oss/3.0/cassandra/operations/opsAboutSnapshots.html
> [2]
> https://cwiki.apache.org/confluence/display/IGNITE/Persistent+Store+Architecture#PersistentStoreArchitecture-Rebalancing
>
> ср, 8 апр. 2020 г. в 18:22, Maxim Muzafarov <mmu...@apache.org>:
>
> > Andrey,
> >
> >
> > Thanks for your questions, I've also clarified some details on the
> > IEP-43 [1] page according to them.
> >
> > > Does snapshot contain only primary data or backup partitions or both?
> >
> > A snapshot contains a full copy of persistence data on each local
> > node. This means all primary, backup partitions and the SQL index file
> > available on the local node are copied to snapshot.
> >
> > > Could I create snapshot from m-node cluster and apply it to n-node
> > cluster (n<>m)?
> >
> > Currently, the restore procedure is fully manual, but it is possible
> > to restore on different topology in general. There are a few options
> > here:
> > - m == n, the easiest and fastest way
> > - m < n, cluster will start and the rebalance will happen (see
> > testClusterSnapshotWithRebalancing in PR). If some SQL indexes exist
> > it may take a quite a long time to complete.
> > - m > n, the hardest case. For instance, if backups > 1 you can start
> > a cluster and remove node one by one from baseline. I think this case
> > should be covered by additional recovery scripts which will be
> > developed further.
> >
> > > - Should data node has extra space on persistent store to create
> > snapshot? Or, from another point of view, woild size of temporary file be
> > equal to size of all data on cluster node?
> >
> > If a cluster has no load you will need only a free space to store
> > snapshot which is almost equal to the node `db` directory size.
> >
> > If a cluster is under the load it needs some extra space to store
> > intermediate snapshot results. The amount of such space depends on how
> > fast cache partition files are copied to snapshot directory (if disks
> > are slow). The maximum size of the temporary file per each partition
> > is equal to the size of the appropriate partition file. So, the worst
> > case you need x3 extra disk size. But according to my measurements
> > assume SSD is used and size of each partition is 300MB it will require
> > no more than 1-3% to a cluster under high load.
> >
> > - What resulted snapshot is, single file or collection of files (one
> > for every data node)?
> >
> > Check the example of the snapshot directory structure on the IEP-43
> > page [1], this is how a completed snapshot will look like.
> >
> > [1]
> > https://cwiki.apache.org/confluence/display/IGNITE/IEP-43%3A+Cluster+snapshots#IEP-43:Clustersnapshots-Restoresnapshot(manually)
> >
> > On Wed, 8 Apr 2020 at 17:18, Andrey Dolmatov <itdolma...@gmail.com> wrote:
> > >
> > > Hi, Maxim!
> > > It is very useful feature, great job!
> > >
> > > But could you explain me some aspects?
> > >
> > >    - Does snapshot contain only primary data or backup partitions or
> > both?
> > >    - Could I create snapshot from m-node cluster and apply it to n-node
> > >    cluster (n<>m)?
> > >    - Should data node has extra space on persistent store to create
> > >    snapshot? Or, from another point of view, woild size of temporary
> > file be
> > >    equal to size of all data on cluster node?
> > >    - What resulted snapshot is, single file or collection of files (one
> > for
> > >    every data node)?
> > >
> > > I apologize for my questions, but i really interested in such feature.
> > >
> > >
> > > вт, 7 апр. 2020 г. в 22:10, Maxim Muzafarov <maxmu...@gmail.com>:
> > >
> > > > Igniters,
> > > >
> > > >
> > > > I'd like to back to the discussion of a snapshot operation for Apache
> > > > Ignite for persistence cache groups and I propose my changes below. I
> > > > have prepared everything so that the discussion is as meaningful and
> > > > specific as much as possible:
> > > >
> > > > - IEP-43: Cluster snapshot [1]
> > > > - The Jira task IGNITE-11073 [2]
> > > > - PR with described changes, Patch Available [4]
> > > >
> > > > Changes are ready for review.
> > > >
> > > >
> > > > Here are a few implementation details and my thoughts:
> > > >
> > > > 1. Snapshot restore assumed to be manual at the first step. The
> > > > process will be described on our documentation pages, but it is
> > > > possible to start node right from the snapshot directory since the
> > > > directory structure is preserved (check
> > > > `testConsistentClusterSnapshotUnderLoad` in the PR). We also have some
> > > > options here about how the restore process must look like:
> > > > - fully manual snapshot restore (will be documented)
> > > > - ansible or shell scripts for restore
> > > > - Java API for restore (I doubt we should go this way).
> > > >
> > > > 3. The snapshot `create` procedure creates a snapshot of all
> > > > persistent caches available on the cluster (see limitations [1]).
> > > >
> > > > 2. The snapshot `create` procedure is available through Java API and
> > > > JMX (control.sh may be implemented further).
> > > >
> > > > Java API:
> > > > IgniteFuture<Void> fut = ignite.snapshot()
> > > > .createSnapshot(name);
> > > >
> > > > JMX:
> > > > SnapshotMXBean mxBean = getMBean(ignite.name());
> > > > mxBean.createSnapshot(name);
> > > >
> > > > 3. The Distribute Process [3] is used to perform a cluster-wide
> > > > snapshot procedure, so we've avoided a lot of boilerplate code here.
> > > >
> > > > 4. The design document [1] contains also an internal API for creating
> > > > a consistent local snapshot of requested cache groups and transfer it
> > > > to another node using the FileTransmission protocol [6]. This is one
> > > > of the parts of IEP-28 [5] for cluster rebalancing via partition files
> > > > and an important part for understanding the whole design.
> > > >
> > > > Java API:
> > > > public IgniteInternalFuture<Void> createRemoteSnapshot(
> > > >     UUID rmtNodeId,
> > > >     Map<Integer, Set<Integer>> parts,
> > > >     BiConsumer<File, GroupPartitionId> partConsumer);
> > > >
> > > >
> > > > Please, share your thoughts and take a loot at my changes [4].
> > > >
> > > >
> > > > [1]
> > > >
> > https://cwiki.apache.org/confluence/display/IGNITE/IEP-43%3A+Cluster+snapshots
> > > > [2] https://issues.apache.org/jira/browse/IGNITE-11073
> > > > [3]
> > > >
> > https://github.com/apache/ignite/blob/master/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java#L49
> > > > [4] https://github.com/apache/ignite/pull/7607
> > > > [5]
> > > >
> > https://cwiki.apache.org/confluence/display/IGNITE/IEP-28%3A+Cluster+peer-2-peer+balancing#IEP-28:Clusterpeer-2-peerbalancing-Filetransferbetweennodes
> > > > [6]
> > > >
> > https://github.com/apache/ignite/blob/master/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/TransmissionHandler.java#L42
> > > >
> > > >
> > > > On Thu, 28 Feb 2019 at 14:43, Dmitriy Pavlov <dpav...@apache.org>
> > wrote:
> > > > >
> > > > > Hi Maxim,
> > > > >
> > > > > I agree with Denis and I have just one concern here.
> > > > >
> > > > > Apache Ignite has quite a long story (started even before Apache),
> > and
> > > > now
> > > > > it has a way too huge number of features. Some of these features
> > > > > - are developed and well known by community members,
> > > > > - some of them were contributed a long time ago and nobody develops
> > it,
> > > > > - and, actually, in some rare cases, nobody in the community knows
> > how it
> > > > > works and how to change it.
> > > > >
> > > > > Such features may attract users, but a bug in it may ruin impression
> > > > about
> > > > > the product. Even worse, nobody can help to solve it, and only user
> > > > himself
> > > > > or herself may be encouraged to contribute a fix.
> > > > >
> > > > > And my concern here, such a big feature should have a number of
> > > > interested
> > > > > contributors, who can support it in case if others lost interest. I
> > will
> > > > be
> > > > > happy if 3-5 members will come and say, yes, I will do a review/I
> > will
> > > > help
> > > > > with further changes.
> > > > >
> > > > > Just to be clear, I'm not against it, and I'll never cast -1 for it,
> > but
> > > > it
> > > > > would be more comfortable to develop this feature with understanding
> > that
> > > > > this work will not be useless.
> > > > >
> > > > > Sincerely,
> > > > > Dmitriy Pavlov
> > > > >
> > > > > ср, 27 февр. 2019 г. в 23:36, Denis Magda <dma...@apache.org>:
> > > > >
> > > > > > Maxim,
> > > > > >
> > > > > > GridGain has this exact feature available for Ignite native
> > persistence
> > > > > > deployments. It's not as easy as it might have been seen from the
> > > > > > enablement perspective. Took us many years to make it production
> > ready,
> > > > > > involving many engineers. If the rest of the community wants to
> > create
> > > > > > something similar and available in open source then please take
> > this
> > > > > > estimate into consideration.
> > > > > >
> > > > > > -
> > > > > > Denis
> > > > > >
> > > > > >
> > > > > > On Wed, Feb 27, 2019 at 8:53 AM Maxim Muzafarov <
> > maxmu...@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > 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
> > > > > > >
> > > > > >
> > > >
> >

Reply via email to