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

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


The following commit(s) were added to refs/heads/master by this push:
     new 0a722ba  fix the test in RaftLogManagerTest
     new ee293e6  Merge pull request #2461 from 
neuyilan/apache_master_0110_applier_bug_fix
0a722ba is described below

commit 0a722baf0cec7683570a8c7e6aec965fbf359ff0
Author: HouliangQi <[email protected]>
AuthorDate: Mon Jan 11 11:12:55 2021 +0800

    fix the test in RaftLogManagerTest
---
 .../apache/iotdb/cluster/log/manage/RaftLogManager.java |  5 ++++-
 .../iotdb/cluster/log/manage/RaftLogManagerTest.java    | 17 ++++++++---------
 2 files changed, 12 insertions(+), 10 deletions(-)

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 61020b8..a5901ed 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
@@ -605,7 +605,8 @@ public abstract class RaftLogManager {
       if (unappliedLogSize > 
ClusterDescriptor.getInstance().getConfig().getMaxNumOfLogsInMem()) {
         logger.debug("There are too many unapplied logs [{}], wait for a while 
to avoid memory "
             + "overflow", unappliedLogSize);
-        Thread.sleep(unappliedLogSize - 
ClusterDescriptor.getInstance().getConfig().getMaxNumOfLogsInMem());
+        Thread.sleep(
+            unappliedLogSize - 
ClusterDescriptor.getInstance().getConfig().getMaxNumOfLogsInMem());
       }
     } catch (TruncateCommittedEntryException e) {
       logger.error("{}: Unexpected error:", name, e);
@@ -648,6 +649,8 @@ public abstract class RaftLogManager {
         logApplier.apply(entry);
       } catch (Exception e) {
         entry.setException(e);
+      } finally {
+        entry.setApplied(true);
       }
     }
   }
diff --git 
a/cluster/src/test/java/org/apache/iotdb/cluster/log/manage/RaftLogManagerTest.java
 
b/cluster/src/test/java/org/apache/iotdb/cluster/log/manage/RaftLogManagerTest.java
index 6ac72fe..410da64 100644
--- 
a/cluster/src/test/java/org/apache/iotdb/cluster/log/manage/RaftLogManagerTest.java
+++ 
b/cluster/src/test/java/org/apache/iotdb/cluster/log/manage/RaftLogManagerTest.java
@@ -86,13 +86,12 @@ public class RaftLogManagerTest {
   private LogApplier logApplier = new TestLogApplier() {
     @Override
     public void apply(Log log) {
-      new Thread(() -> {
-        while (blocked) {
-          // stuck
-        }
-        appliedLogs.put(log.getCurrLogIndex(), log);
-        log.setApplied(true);
-      }).start();
+      if (blocked) {
+        return;
+      }
+      // make sure the log is applied when not blocked
+      appliedLogs.put(log.getCurrLogIndex(), log);
+      log.setApplied(true);
     }
   };
   private int testIdentifier = 1;
@@ -157,7 +156,7 @@ public class RaftLogManagerTest {
       blocked = true;
       instance.setBlockAppliedCommitIndex(99);
       instance.append(logs.subList(50, 100));
-      instance.commitTo(99);
+      instance.commitTo(98);
 
       try {
         // applier is blocked, so this should time out
@@ -167,6 +166,7 @@ public class RaftLogManagerTest {
         assertEquals("wait all log applied time out", e.getMessage());
       }
       blocked = false;
+      instance.commitTo(99);
       // applier is unblocked, BlockAppliedCommitIndex should be soon reached
       ClusterDescriptor.getInstance().getConfig().setCatchUpTimeoutMS(60_000);
       instance.takeSnapshot();
@@ -1342,7 +1342,6 @@ public class RaftLogManagerTest {
     } finally {
       raftLogManager.close();
     }
-
   }
 
   @Test

Reply via email to