This is an automated email from the ASF dual-hosted git repository.

jiangtian pushed a commit to branch cluster_new
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/cluster_new by this push:
     new ee697a3  fix takeSnapshot bug
     new 00bd239  Merge pull request #1872 from mychaow/snapshotfix
ee697a3 is described below

commit ee697a30154dad8b3fb698c1ef1d2d1c801e2763
Author: chaow <[email protected]>
AuthorDate: Tue Oct 27 16:59:30 2020 +0800

    fix takeSnapshot bug
---
 .../cluster/log/manage/FilePartitionedSnapshotLogManager.java  | 10 +++++++---
 .../iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java |  2 +-
 .../org/apache/iotdb/cluster/log/manage/RaftLogManager.java    |  6 +++++-
 3 files changed, 13 insertions(+), 5 deletions(-)

diff --git 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
index 7aec32a..074fe41 100644
--- 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
+++ 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/FilePartitionedSnapshotLogManager.java
@@ -25,6 +25,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+
+import org.apache.iotdb.cluster.exception.EntryCompactedException;
 import org.apache.iotdb.cluster.log.LogApplier;
 import org.apache.iotdb.cluster.log.snapshot.FileSnapshot;
 import org.apache.iotdb.cluster.partition.PartitionTable;
@@ -76,14 +78,16 @@ public class FilePartitionedSnapshotLogManager extends 
PartitionedSnapshotLogMan
       syncFlushAllProcessor();
       logger.info("{}: Taking snapshots, IoTDB is flushed", getName());
       // TODO-cluster https://issues.apache.org/jira/browse/IOTDB-820
+      super.takeSnapshot();
       synchronized (this) {
-        super.takeSnapshot();
         collectTimeseriesSchemas();
-        snapshotLastLogIndex = getCommitLogIndex();
-        snapshotLastLogTerm = getCommitLogTerm();
+        snapshotLastLogIndex = getBlockAppliedCommitIndex();
+        snapshotLastLogTerm = getTerm(snapshotLastLogIndex);
         collectTsFilesAndFillTimeseriesSchemas();
         logger.info("{}: Snapshot is taken", getName());
       }
+    } catch (EntryCompactedException e) {
+      logger.error("failed to do snapshot.", e);
     } finally {
       super.resetBlockAppliedCommitIndex();
     }
diff --git 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
index a8ff8a1..7e0fa76 100644
--- 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
+++ 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MetaSingleSnapshotLogManager.java
@@ -58,8 +58,8 @@ public class MetaSingleSnapshotLogManager extends 
RaftLogManager {
   @SuppressWarnings("java:S1135") // ignore todos
   public void takeSnapshot() throws IOException {
     // TODO-cluster https://issues.apache.org/jira/browse/IOTDB-820
+    super.takeSnapshot();
     synchronized (this) {
-      super.takeSnapshot();
       storageGroupTTLMap = IoTDB.metaManager.getStorageGroupsTTL();
       try {
         IAuthorizer authorizer = BasicAuthorizer.getInstance();
diff --git 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
index 99fa253..57c3778 100644
--- 
a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
+++ 
b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/RaftLogManager.java
@@ -852,7 +852,7 @@ public abstract class RaftLogManager {
     try {
       long nextToCheckIndex = maxHaveAppliedCommitIndex + 1;
       if (nextToCheckIndex > commitIndex || nextToCheckIndex > 
getCommittedEntryManager()
-          .getLastIndex()) {
+          .getLastIndex() || (blockAppliedCommitIndex > 0 && 
blockAppliedCommitIndex < nextToCheckIndex)) {
         // avoid spinning
         Thread.sleep(5);
         return;
@@ -913,4 +913,8 @@ public abstract class RaftLogManager {
   public String getName() {
     return name;
   }
+
+  public long getBlockAppliedCommitIndex() {
+    return blockAppliedCommitIndex;
+  }
 }

Reply via email to