alievmirza commented on code in PR #937:
URL: https://github.com/apache/ignite-3/pull/937#discussion_r929948779


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionSnapshotStorageFactory.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.table.distributed.raft.snapshot;
+
+import java.util.List;
+import org.apache.ignite.internal.raft.storage.SnapshotStorageFactory;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.raft.jraft.RaftMessagesFactory;
+import org.apache.ignite.raft.jraft.entity.RaftOutter.SnapshotMeta;
+import org.apache.ignite.raft.jraft.option.RaftOptions;
+import org.apache.ignite.raft.jraft.storage.SnapshotStorage;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotReader;
+import org.apache.ignite.raft.jraft.storage.snapshot.SnapshotWriter;
+
+/**
+ * Snapshot storage factory for {@link MvPartitionStorage}. Utilizes the fact 
that every partition already stores its latest applied index
+ * and thus can inself be used as its own snapshot.
+ *
+ * <p/>Uses {@link MvPartitionStorage#persistedIndex()} and configuration, 
passed into constructor, to create a {@link SnapshotMeta} object
+ * in {@link SnapshotReader#load()}.
+ *
+ * <p/>Snapshot writer doesn't allow explicit save of any actual file. {@link 
SnapshotWriter#saveMeta(SnapshotMeta)} simply returns
+ * {@code true}, and {@link SnapshotWriter#addFile(String)} throws an 
exception.
+ */
+public class PartitionSnapshotStorageFactory implements SnapshotStorageFactory 
{
+    /** Partition storage. */
+    private final MvPartitionStorage partitionStorage;
+
+    /** List of peers. */
+    private final List<String> peers;
+
+    /** List of learners. */
+    private final List<String> learners;
+
+    /** RAFT log index read from {@link MvPartitionStorage#persistedIndex()} 
during factory instantiation. */
+    private final long persistedRaftIndex;
+
+    /**
+     * Constructor.
+     *
+     * @param partitionStorage MV partition storage.
+     * @param peers List of raft group peers to be used in snapshot meta.
+     * @param learners List of raft group learners to be used in snapshot meta.
+     *
+     * @see SnapshotMeta
+     */
+    @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+    public PartitionSnapshotStorageFactory(MvPartitionStorage 
partitionStorage, List<String> peers, List<String> learners) {
+        this.partitionStorage = partitionStorage;
+        this.peers = peers;
+        this.learners = learners;
+
+        persistedRaftIndex = partitionStorage.persistedIndex();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public SnapshotStorage createSnapshotStorage(String uri, RaftOptions 
raftOptions) {
+        SnapshotMeta snapshotMeta = new RaftMessagesFactory().snapshotMeta()
+                .lastIncludedIndex(persistedRaftIndex)
+                // According to the code of 
org.apache.ignite.raft.jraft.core.NodeImpl.bootstrap, it's "dangerous" to init 
term with a value

Review Comment:
   @ascherbakoff This example is artificial, but I'll try to explain motivation 
of setting term to 0 or 1. Imagine that node that was restored from a snapshot 
has term that is higher than current leader's term. On the first 
`NodeImpl#handleAppendEntriesRequest` it will check it's own term with leader's 
term from request and will send unsuccessful response with higher term. Leader 
will receive this in `Replicator#onAppendEntriesReturned` and will step down as 
far as it received term higher than it's own. 
   
   So, it is more safe to set it to value that is definitely less than current 
term in a cluster. 
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to