You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2023/09/18 10:20:44 UTC

[iotdb] branch fix_datanode_stuck created (now a1abfc91da4)

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

haonan pushed a change to branch fix_datanode_stuck
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at a1abfc91da4 fix progress not exit and add comments

This branch includes the following new commits:

     new b92fdaa8c1d Fix datanode start up stuck when the disk is broken
     new 312c86b5018 Fix datanode start up stuck when the disk is broken
     new f49afa94f0b Merge branch 'master' of github.com:apache/iotdb into fix_datanode_stuck
     new a1abfc91da4 fix progress not exit and add comments

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[iotdb] 02/04: Fix datanode start up stuck when the disk is broken

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 312c86b501809d127f7100e22fe081a4b8cac357
Author: HTHou <hh...@outlook.com>
AuthorDate: Mon Sep 18 16:19:58 2023 +0800

    Fix datanode start up stuck when the disk is broken
---
 .../concurrent/ExceptionalCountDownLatch.java      | 54 ++++++++++++++++++++++
 1 file changed, 54 insertions(+)

diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java
new file mode 100644
index 00000000000..ef933147aa5
--- /dev/null
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java
@@ -0,0 +1,54 @@
+/*
+ * 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.iotdb.commons.concurrent;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
+
+/** This class adds . */
+public class ExceptionalCountDownLatch {
+  private final CountDownLatch latch;
+  private final AtomicReference<String> exceptionMessage = new AtomicReference<>();
+
+  public ExceptionalCountDownLatch(int count) {
+    this.latch = new CountDownLatch(count);
+  }
+
+  public void countDown() {
+    latch.countDown();
+  }
+
+  public void countDownWithException(String message) {
+    exceptionMessage.set(message);
+    countDown();
+  }
+
+  public void await() throws InterruptedException {
+    latch.await();
+  }
+
+  public boolean hasException() {
+    return exceptionMessage.get() != null;
+  }
+
+  public String getExceptionMessage() {
+    return exceptionMessage.get();
+  }
+}


[iotdb] 03/04: Merge branch 'master' of github.com:apache/iotdb into fix_datanode_stuck

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f49afa94f0bdec4d4574bf54fd68dbc4f986be1e
Merge: 312c86b5018 3313d742104
Author: HTHou <hh...@outlook.com>
AuthorDate: Mon Sep 18 16:24:40 2023 +0800

    Merge branch 'master' of github.com:apache/iotdb into fix_datanode_stuck

 example/flink-sql/pom.xml                          | 10 +--
 integration-test/pom.xml                           | 83 +++++++++++++++++++++-
 .../config/constant/PipeConnectorConstant.java     |  4 ++
 ...eResponse.java => AirGapELanguageConstant.java} | 14 +++-
 .../payload/airgap/AirGapOneByteResponse.java      |  4 ++
 .../protocol/airgap/IoTDBAirGapConnector.java      | 21 +++++-
 .../pipe/receiver/airgap/IoTDBAirGapReceiver.java  | 58 ++++++++++-----
 7 files changed, 165 insertions(+), 29 deletions(-)


[iotdb] 01/04: Fix datanode start up stuck when the disk is broken

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit b92fdaa8c1d56771daa030076f97de5eae6dc304
Author: HTHou <hh...@outlook.com>
AuthorDate: Mon Sep 18 16:19:50 2023 +0800

    Fix datanode start up stuck when the disk is broken
---
 .../org/apache/iotdb/db/storageengine/StorageEngine.java     | 10 ++++++----
 .../apache/iotdb/db/storageengine/dataregion/DataRegion.java |  4 ++++
 .../dataregion/wal/recover/WALRecoverManager.java            | 12 +++++++++---
 .../RewriteCrossSpaceCompactionWithFastPerformerTest.java    |  4 ++--
 ...ewriteCrossSpaceCompactionWithReadPointPerformerTest.java |  4 ++--
 .../dataregion/wal/recover/WALRecoverManagerTest.java        |  4 ++--
 6 files changed, 25 insertions(+), 13 deletions(-)

diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java
index 1cbdf6e7397..46e04b4335e 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.storageengine;
 import org.apache.iotdb.common.rpc.thrift.TFlushReq;
 import org.apache.iotdb.common.rpc.thrift.TSStatus;
 import org.apache.iotdb.common.rpc.thrift.TSetTTLReq;
+import org.apache.iotdb.commons.concurrent.ExceptionalCountDownLatch;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
@@ -28,6 +29,7 @@ import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.consensus.DataRegionId;
 import org.apache.iotdb.commons.exception.ShutdownException;
+import org.apache.iotdb.commons.exception.StartupException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.service.IService;
 import org.apache.iotdb.commons.service.ServiceType;
@@ -82,7 +84,6 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -209,7 +210,7 @@ public class StorageEngine implements IService {
     isAllSgReady.set(allSgReady);
   }
 
-  public void recover() {
+  public void recover() throws StartupException {
     setAllSgReady(false);
     cachedThreadPool =
         IoTDBThreadPoolFactory.newCachedThreadPool(ThreadName.STORAGE_ENGINE_CACHED_POOL.getName());
@@ -245,7 +246,7 @@ public class StorageEngine implements IService {
     readyDataRegionNum = new AtomicInteger(0);
     // init wal recover manager
     WALRecoverManager.getInstance()
-        .setAllDataRegionScannedLatch(new CountDownLatch(recoverDataRegionNum));
+        .setAllDataRegionScannedLatch(new ExceptionalCountDownLatch(recoverDataRegionNum));
     for (Map.Entry<String, List<DataRegionId>> entry : localDataRegionInfo.entrySet()) {
       String sgName = entry.getKey();
       for (DataRegionId dataRegionId : entry.getValue()) {
@@ -261,6 +262,7 @@ public class StorageEngine implements IService {
               } catch (DataRegionException e) {
                 logger.error(
                     "Failed to recover data region {}[{}]", sgName, dataRegionId.getId(), e);
+                return null;
               }
               dataRegionMap.put(dataRegionId, dataRegion);
               logger.info(
@@ -300,7 +302,7 @@ public class StorageEngine implements IService {
   }
 
   @Override
-  public void start() {
+  public void start() throws StartupException {
     // build time Interval to divide time partition
     initTimePartition();
     // create systemDir
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
index 0b8ad3e5a3f..ae6282eaaa2 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java
@@ -541,6 +541,10 @@ public class DataRegion implements IDataRegionForQuery {
         updatePartitionFileVersion(partitionNum, resource.getVersion());
       }
     } catch (IOException e) {
+      // signal wal recover manager to recover this region's files
+      WALRecoverManager.getInstance()
+          .getAllDataRegionScannedLatch()
+          .countDownWithException(e.getMessage());
       throw new DataRegionException(e);
     }
 
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
index 44c80cad828..c8f08090a06 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.storageengine.dataregion.wal.recover;
 
+import org.apache.iotdb.commons.concurrent.ExceptionalCountDownLatch;
 import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.conf.CommonConfig;
@@ -55,7 +56,7 @@ public class WALRecoverManager {
   private volatile boolean hasStarted = false;
   // start recovery after all data regions have submitted unsealed zero-level TsFiles
   @SuppressWarnings("squid:S3077")
-  private volatile CountDownLatch allDataRegionScannedLatch;
+  private volatile ExceptionalCountDownLatch allDataRegionScannedLatch;
   // threads to recover wal nodes
   private ExecutorService recoverThreadPool;
   // stores all UnsealedTsFileRecoverPerformer submitted by data region processors
@@ -90,6 +91,9 @@ public class WALRecoverManager {
         Thread.currentThread().interrupt();
         throw new WALRecoverException("Fail to recover wal.", e);
       }
+      if (allDataRegionScannedLatch.hasException()) {
+        throw new DataRegionException(allDataRegionScannedLatch.getExceptionMessage());
+      }
       logger.info(
           "Data regions have submitted all unsealed TsFiles, start recovering TsFiles in each wal node.");
       // recover each wal node's TsFiles
@@ -110,6 +114,8 @@ public class WALRecoverManager {
       }
       // deal with remaining TsFiles which don't have wal
       asyncRecoverLeftTsFiles();
+    } catch (DataRegionException e) {
+      throw new RuntimeException(e.getMessage());
     } catch (Exception e) {
       for (UnsealedTsFileRecoverPerformer recoverPerformer :
           absolutePath2RecoverPerformer.values()) {
@@ -202,11 +208,11 @@ public class WALRecoverManager {
     return null;
   }
 
-  public CountDownLatch getAllDataRegionScannedLatch() {
+  public ExceptionalCountDownLatch getAllDataRegionScannedLatch() {
     return allDataRegionScannedLatch;
   }
 
-  public void setAllDataRegionScannedLatch(CountDownLatch allDataRegionScannedLatch) {
+  public void setAllDataRegionScannedLatch(ExceptionalCountDownLatch allDataRegionScannedLatch) {
     this.allDataRegionScannedLatch = allDataRegionScannedLatch;
   }
 
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java
index 8a22594dc8f..7ba150e3b32 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.storageengine.dataregion.compaction.cross;
 
+import org.apache.iotdb.commons.concurrent.ExceptionalCountDownLatch;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.AlignedPath;
@@ -61,7 +62,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.CROSS_COMPACTION_TMP_FILE_SUFFIX;
@@ -76,7 +76,7 @@ public class RewriteCrossSpaceCompactionWithFastPerformerTest extends AbstractCo
   public void setUp()
       throws IOException, WriteProcessException, MetadataException, InterruptedException {
     super.setUp();
-    WALRecoverManager.getInstance().setAllDataRegionScannedLatch(new CountDownLatch(1));
+    WALRecoverManager.getInstance().setAllDataRegionScannedLatch(new ExceptionalCountDownLatch(1));
     IoTDBDescriptor.getInstance().getConfig().setTargetChunkSize(1024);
     Thread.currentThread().setName("pool-1-IoTDB-Compaction-Worker-1");
   }
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java
index d7a7942acfe..05d5d77d89a 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.storageengine.dataregion.compaction.cross;
 
+import org.apache.iotdb.commons.concurrent.ExceptionalCountDownLatch;
 import org.apache.iotdb.commons.exception.IllegalPathException;
 import org.apache.iotdb.commons.exception.MetadataException;
 import org.apache.iotdb.commons.path.AlignedPath;
@@ -61,7 +62,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.iotdb.commons.conf.IoTDBConstant.CROSS_COMPACTION_TMP_FILE_SUFFIX;
@@ -76,7 +76,7 @@ public class RewriteCrossSpaceCompactionWithReadPointPerformerTest extends Abstr
   public void setUp()
       throws IOException, WriteProcessException, MetadataException, InterruptedException {
     super.setUp();
-    WALRecoverManager.getInstance().setAllDataRegionScannedLatch(new CountDownLatch(1));
+    WALRecoverManager.getInstance().setAllDataRegionScannedLatch(new ExceptionalCountDownLatch(1));
     IoTDBDescriptor.getInstance().getConfig().setTargetChunkSize(1024);
     Thread.currentThread().setName("pool-1-IoTDB-Compaction-Worker-1");
   }
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManagerTest.java
index 8777464abf4..73cb2936ade 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManagerTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManagerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.storageengine.dataregion.wal.recover;
 
+import org.apache.iotdb.commons.concurrent.ExceptionalCountDownLatch;
 import org.apache.iotdb.commons.conf.CommonConfig;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
 import org.apache.iotdb.commons.exception.IllegalPathException;
@@ -72,7 +73,6 @@ import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -265,7 +265,7 @@ public class WALRecoverManagerTest {
     // prepare tsFiles
     List<WALRecoverListener> recoverListeners = prepareCrashedTsFile();
     // recover
-    recoverManager.setAllDataRegionScannedLatch(new CountDownLatch(0));
+    recoverManager.setAllDataRegionScannedLatch(new ExceptionalCountDownLatch(0));
     recoverManager.recover();
     // check recover listeners
     try {


[iotdb] 04/04: fix progress not exit and add comments

Posted by ha...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a1abfc91da45e11e65809e563e724d763931f1ae
Author: HTHou <hh...@outlook.com>
AuthorDate: Mon Sep 18 18:20:25 2023 +0800

    fix progress not exit and add comments
---
 .../main/java/org/apache/iotdb/db/service/DataNode.java    | 14 +++++++++++---
 .../dataregion/wal/recover/WALRecoverManager.java          |  5 +++--
 .../recover/SizeTieredCompactionRecoverTest.java           |  3 ++-
 .../java/org/apache/iotdb/db/utils/EnvironmentUtils.java   |  9 ++++-----
 .../commons/concurrent/ExceptionalCountDownLatch.java      |  2 +-
 5 files changed, 21 insertions(+), 12 deletions(-)

diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java
index e238bc4dc74..08080d28162 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/service/DataNode.java
@@ -140,6 +140,9 @@ public class DataNode implements DataNodeMBean {
   private static final String REGISTER_INTERRUPTION =
       "Unexpected interruption when waiting to register to the cluster";
 
+  private boolean schemaRegionConsensusStarted = false;
+  private boolean dataRegionConsensusStarted = false;
+
   private DataNode() {
     // We do not init anything here, so that we can re-initialize the instance in IT.
   }
@@ -487,7 +490,9 @@ public class DataNode implements DataNodeMBean {
 
     try {
       SchemaRegionConsensusImpl.getInstance().start();
+      schemaRegionConsensusStarted = true;
       DataRegionConsensusImpl.getInstance().start();
+      dataRegionConsensusStarted = true;
     } catch (IOException e) {
       throw new StartupException(e);
     }
@@ -871,11 +876,14 @@ public class DataNode implements DataNodeMBean {
 
   public void stop() {
     deactivate();
-
     try {
       MetricService.getInstance().stop();
-      SchemaRegionConsensusImpl.getInstance().stop();
-      DataRegionConsensusImpl.getInstance().stop();
+      if (schemaRegionConsensusStarted) {
+        SchemaRegionConsensusImpl.getInstance().stop();
+      }
+      if (dataRegionConsensusStarted) {
+        DataRegionConsensusImpl.getInstance().stop();
+      }
     } catch (Exception e) {
       logger.error("Stop data node error", e);
     }
diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
index c8f08090a06..42ccc9d3dac 100644
--- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
+++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/recover/WALRecoverManager.java
@@ -24,6 +24,7 @@ import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.commons.concurrent.ThreadName;
 import org.apache.iotdb.commons.conf.CommonConfig;
 import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.exception.StartupException;
 import org.apache.iotdb.commons.file.SystemFileFactory;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.db.exception.DataRegionException;
@@ -65,7 +66,7 @@ public class WALRecoverManager {
 
   private WALRecoverManager() {}
 
-  public void recover() throws WALRecoverException {
+  public void recover() throws WALRecoverException, StartupException {
     logger.info("Start recovering wal.");
     try {
       // collect wal nodes' information
@@ -115,7 +116,7 @@ public class WALRecoverManager {
       // deal with remaining TsFiles which don't have wal
       asyncRecoverLeftTsFiles();
     } catch (DataRegionException e) {
-      throw new RuntimeException(e.getMessage());
+      throw new StartupException(e.getMessage());
     } catch (Exception e) {
       for (UnsealedTsFileRecoverPerformer recoverPerformer :
           absolutePath2RecoverPerformer.values()) {
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/recover/SizeTieredCompactionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/recover/SizeTieredCompactionRecoverTest.java
index b0ba4ae37db..3d23d55a511 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/recover/SizeTieredCompactionRecoverTest.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/recover/SizeTieredCompactionRecoverTest.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.db.storageengine.dataregion.compaction.recover;
 
 import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.exception.StartupException;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -175,7 +176,7 @@ public class SizeTieredCompactionRecoverTest {
 
   /** Test when a file that is not a directory exists under virtual storageGroup dir. */
   @Test
-  public void testRecoverWithUncorrectTimePartionDir() {
+  public void testRecoverWithUncorrectTimePartionDir() throws StartupException {
     StorageEngine.getInstance().start();
     try {
       File timePartitionDir = new File(SEQ_FILE_DIR);
diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
index 7506dc14337..9766269e420 100644
--- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
+++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java
@@ -269,13 +269,12 @@ public class EnvironmentUtils {
 
     createAllDir();
 
-    StorageEngine.getInstance().start();
-
-    SchemaEngine.getInstance().init();
+    try {
+      StorageEngine.getInstance().start();
 
-    CompactionTaskManager.getInstance().start();
+      SchemaEngine.getInstance().init();
 
-    try {
+      CompactionTaskManager.getInstance().start();
       WALManager.getInstance().start();
       FlushManager.getInstance().start();
     } catch (StartupException e) {
diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java
index ef933147aa5..a28665fb562 100644
--- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java
+++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/concurrent/ExceptionalCountDownLatch.java
@@ -22,7 +22,7 @@ package org.apache.iotdb.commons.concurrent;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReference;
 
-/** This class adds . */
+/** This class supports passing exception message when using CountDownLatch. */
 public class ExceptionalCountDownLatch {
   private final CountDownLatch latch;
   private final AtomicReference<String> exceptionMessage = new AtomicReference<>();