You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hx...@apache.org on 2019/04/18 15:57:41 UTC

[incubator-iotdb] branch refactor_overflow updated: allow save overflow files into several folders

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

hxd pushed a commit to branch refactor_overflow
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/refactor_overflow by this push:
     new b03060d  allow save overflow files into several folders
b03060d is described below

commit b03060d339083f02996b4fd947757fed09908505
Author: xiangdong huang <sa...@gmail.com>
AuthorDate: Thu Apr 18 23:57:29 2019 +0800

    allow save overflow files into several folders
---
 iotdb/iotdb/conf/iotdb-engine.properties           | 15 +++++-
 iotdb/iotdb/conf/logback.xml                       |  2 -
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 23 +++++---
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  2 +
 .../iotdb/db/conf/directories/Directories.java     | 61 ++++++++++++++++++----
 .../db/engine/overflow/io/OverflowProcessor.java   |  3 +-
 .../db/engine/overflowdata/OverflowProcessor.java  | 51 ++++++++++++++++++
 .../db/engine/tsfiledata/TsFileProcessor.java      |  6 ++-
 .../apache/iotdb/db/monitor/MonitorConstants.java  |  3 +-
 .../org/apache/iotdb/db/utils/OpenFileNumUtil.java |  2 +-
 .../java/org/apache/iotdb/db/engine/PathUtils.java |  8 +--
 .../bufferwrite/BufferWriteProcessorNewTest.java   |  4 +-
 .../bufferwrite/BufferWriteProcessorTest.java      | 10 ++--
 .../memcontrol/BufferwriteFileSizeControlTest.java |  2 +-
 .../memcontrol/BufferwriteMetaSizeControlTest.java |  2 +-
 15 files changed, 157 insertions(+), 37 deletions(-)

diff --git a/iotdb/iotdb/conf/iotdb-engine.properties b/iotdb/iotdb/conf/iotdb-engine.properties
index 5c7046f..c22dba4 100644
--- a/iotdb/iotdb/conf/iotdb-engine.properties
+++ b/iotdb/iotdb/conf/iotdb-engine.properties
@@ -52,7 +52,7 @@ force_wal_period_in_ms=10
 # data_dir=/path/iotdb/data/data
 
 # tsfile dir
-# For this property, multiple directories should be set, and all directories should be separated by ",". All TsFiles will be allocated separately in all these directories. Moreover, setting absolute directories is suggested.
+# For this property, multiple directories can be set, and all directories should be separated by ",". All TsFiles will be allocated separately in all these directories. Moreover, setting absolute directories is suggested.
 # If this property is unset, system will save the TsFiles in the default relative path directory under the data_dir folder(i.e., %IOTDB_HOME%/data/data/settled).
 # If some are absolute, system will save the data in exact location they point to.
 # If some are relative, system will save the data in the relative path directory they indicate under the data_dir folder.
@@ -64,6 +64,19 @@ force_wal_period_in_ms=10
 # If its prefix is "/", then the path is absolute. Otherwise, it is relative.
 # tsfile_dir=/path/iotdb/data/data,data/data
 
+# overflow dir
+# For this property, multiple directories can be set, and all directories should be separated by ",". All Overflow Files will be allocated separately in all these directories. Moreover, setting absolute directories is suggested.
+# If this property is unset, system will save the Overflow Files in the default relative path directory under the data_dir folder(i.e., %IOTDB_HOME%/data/data/overflow).
+# If some are absolute, system will save the data in exact location they point to.
+# If some are relative, system will save the data in the relative path directory they indicate under the data_dir folder.
+# Note: If some are assigned an empty string(i.e.,zero-length), they will be handled as a relative path.
+# For windows platform
+# If its prefix is a drive specifier followed by "\\", or if its prefix is "\\\\", then the path is absolute. Otherwise, it is relative.
+# overflow_dir=D:\\iotdb\\data\\overflow,overflow2
+# For Linux platform
+# If its prefix is "/", then the path is absolute. Otherwise, it is relative.
+# overflow_dir=/path/iotdb/data/overflow1,overflow2
+
 # mult_dir_strategy
 # The strategy is used to choose a directory from tsfile_dir for the system to store a new tsfile.
 # System provides three strategies to choose from, or user can create his own strategy by extending org.apache.iotdb.db.conf.directories.strategy.DirectoryStrategy.
diff --git a/iotdb/iotdb/conf/logback.xml b/iotdb/iotdb/conf/logback.xml
index 1f8a915..2629723 100644
--- a/iotdb/iotdb/conf/logback.xml
+++ b/iotdb/iotdb/conf/logback.xml
@@ -111,7 +111,6 @@
     </appender>
     <logger level="info" name="org.apache.iotdb.db.service"/>
     <logger level="info" name="org.apache.iotdb.db.conf"/>
-
     <!-- a log appender that collect all log records whose level is greather than debug-->
     <appender class="ch.qos.logback.core.rolling.RollingFileAppender" name="FILEALL">
         <file>${IOTDB_HOME}/logs/log_all.log</file>
@@ -130,7 +129,6 @@
             <level>INFO</level>
         </filter>
     </appender>
-
     <root level="info">
         <appender-ref ref="FILEDEBUG"/>
         <appender-ref ref="FILEWARN"/>
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index 9790f37..f588430 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -33,6 +33,7 @@ public class IoTDBConfig {
   public static final String DEFAULT_DATA_DIR = "data";
   public static final String DEFAULT_SYS_DIR = "system";
   public static final String DEFAULT_TSFILE_DIR = "settled";
+  public static final String DEFAULT_OVERFLOW_DIR = "overflow";
   public static final String MULT_DIR_STRATEGY_PREFIX =
       "org.apache.iotdb.db.conf.directories.strategy.";
   public static final String DEFAULT_MULT_DIR_STRATEGY = "MaxDiskUsableSpaceFirstStrategy";
@@ -84,7 +85,7 @@ public class IoTDBConfig {
   /**
    * Data directory of Overflow data.
    */
-  private String overflowDataDir = "overflow";
+  private String[] overflowDataDirs = {DEFAULT_OVERFLOW_DIR};
 
   /**
    * Data directory of fileNode data.
@@ -297,7 +298,17 @@ public class IoTDBConfig {
     if (getDataDir().length() > 0 && !getDataDir().endsWith(File.separator)) {
       setDataDir(getDataDir() + File.separatorChar);
     }
-    setOverflowDataDir(getDataDir() + getOverflowDataDir());
+
+    if (getOverflowDataDirs() == null || getOverflowDataDirs().length == 0) {
+      setOverflowDataDirs(new String[]{DEFAULT_OVERFLOW_DIR});
+    }
+    for (int i = 0; i < getOverflowDataDirs().length; i++) {
+      if (new File(getOverflowDataDirs()[i]).isAbsolute()) {
+        continue;
+      }
+
+      getOverflowDataDirs()[i] = getDataDir() + getOverflowDataDirs()[i];
+    }
 
     if (getBufferWriteDirs() == null || getBufferWriteDirs().length == 0) {
       setBufferWriteDirs(new String[]{DEFAULT_TSFILE_DIR});
@@ -488,12 +499,12 @@ public class IoTDBConfig {
     this.walDir = walDir;
   }
 
-  public String getOverflowDataDir() {
-    return overflowDataDir;
+  public String[] getOverflowDataDirs() {
+    return overflowDataDirs;
   }
 
-  public void setOverflowDataDir(String overflowDataDir) {
-    this.overflowDataDir = overflowDataDir;
+  public void setOverflowDataDirs(String[] overflowDataDirs) {
+    this.overflowDataDirs = overflowDataDirs;
   }
 
   public String getFileNodeDir() {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 7ab5215..a665849 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -130,6 +130,8 @@ public class IoTDBDescriptor {
       conf.setDataDir(properties.getProperty("data_dir", conf.getDataDir()));
       conf.setBufferWriteDirs(properties.getProperty("tsfile_dir", conf.DEFAULT_TSFILE_DIR)
           .split(","));
+      conf.setOverflowDataDirs(properties.getProperty("overflow_dir", conf.DEFAULT_OVERFLOW_DIR)
+          .split(","));
       conf.setSysDir(properties.getProperty("sys_dir", conf.getSysDir()));
       conf.setWalDir(properties.getProperty("wal_dir", conf.getWalDir()));
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java
index ad9c847..2acaaf0 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/Directories.java
@@ -37,21 +37,29 @@ public class Directories {
   private static final Logger LOGGER = LoggerFactory.getLogger(Directories.class);
 
   private List<String> tsfileFolders;
-  private DirectoryStrategy strategy;
+  private List<String> overflowFolders;
+
+  private DirectoryStrategy tsfileStrategy;
+  private DirectoryStrategy overflowStrategy;
 
   private Directories() {
     tsfileFolders = new ArrayList<>(
         Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getBufferWriteDirs()));
-    initFolders();
+    initFolders(tsfileFolders);
+    overflowFolders = new ArrayList<>(
+        Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getOverflowDataDirs()));
+    initFolders(overflowFolders);
 
     String strategyName = "";
     try {
       strategyName = IoTDBDescriptor.getInstance().getConfig().getMultDirStrategyClassName();
       Class<?> clazz = Class.forName(strategyName);
-      strategy = (DirectoryStrategy) clazz.newInstance();
-      strategy.init(tsfileFolders);
+      tsfileStrategy = (DirectoryStrategy) clazz.newInstance();
+      tsfileStrategy.init(tsfileFolders);
+      overflowStrategy = (DirectoryStrategy) clazz.newInstance();
+      overflowStrategy.init(overflowFolders);
     } catch (Exception e) {
-      LOGGER.error("can't find strategy {} for mult-directories.", strategyName, e);
+      LOGGER.error("can't find tsfileStrategy {} for mult-directories.", strategyName, e);
     }
   }
 
@@ -59,22 +67,22 @@ public class Directories {
     return DirectoriesHolder.INSTANCE;
   }
 
-  private void initFolders() {
-    for (String folder : tsfileFolders) {
+  private void initFolders(List<String> folders) {
+    for (String folder : folders) {
       File file = new File(folder);
       if (file.mkdirs()) {
-        LOGGER.info("folder {} in tsfileFolders doesn't exist, create it", file.getPath());
+        LOGGER.info("folder {} doesn't exist, create it", file.getPath());
       }
     }
   }
 
   // only used by test
-  public String getFolderForTest() {
+  public String getTsFolderForTest() {
     return tsfileFolders.get(0);
   }
 
   // only used by test
-  public void setFolderForTest(String path) {
+  public void setTsFolderForTest(String path) {
     tsfileFolders.set(0, path);
   }
 
@@ -88,7 +96,7 @@ public class Directories {
    * @return next folder index
    */
   public int getNextFolderIndexForTsFile() {
-    return strategy.nextFolderIndex();
+    return tsfileStrategy.nextFolderIndex();
   }
 
   public String getTsFileFolder(int index) {
@@ -103,6 +111,37 @@ public class Directories {
     return tsfileFolders;
   }
 
+
+  public String getNextFolderForOverflowFile() {
+    return getOverflowFileFolder(getNextFolderIndexForOverflowFile());
+  }
+
+  /**
+   * get next folder index for OverflowFile.
+   *
+   * @return next folder index
+   */
+  public int getNextFolderIndexForOverflowFile() {
+    return overflowStrategy.nextFolderIndex();
+  }
+
+  public String getOverflowFileFolder(int index) {
+    return overflowFolders.get(index);
+  }
+
+  public int getOverflowFileFolderIndex(String folder) {
+    return overflowFolders.indexOf(folder);
+  }
+
+  public List<String> getAllOverflowFileFolders() {
+    return overflowFolders;
+  }
+
+  // only used by test
+  public String getOverflowFolderForTest() {
+    return overflowFolders.get(0);
+  }
+
   private static class DirectoriesHolder {
     private static final Directories INSTANCE = new Directories();
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/io/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/io/OverflowProcessor.java
index 2e7302f..2af7b33 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/io/OverflowProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/io/OverflowProcessor.java
@@ -102,7 +102,8 @@ public class OverflowProcessor extends Processor {
     super(processorName);
     this.fileSchema = fileSchema;
     this.versionController = versionController;
-    String overflowDirPath = TsFileDBConf.getOverflowDataDir();
+    //for this old version, we only support one Overflow file
+    String overflowDirPath = TsFileDBConf.getOverflowDataDirs()[0];
     if (overflowDirPath.length() > 0
         && overflowDirPath.charAt(overflowDirPath.length() - 1) != File.separatorChar) {
       overflowDirPath = overflowDirPath + File.separatorChar;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflowdata/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflowdata/OverflowProcessor.java
new file mode 100644
index 0000000..f5419c7
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflowdata/OverflowProcessor.java
@@ -0,0 +1,51 @@
+/**
+ * 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.db.engine.overflowdata;
+
+import java.io.IOException;
+import org.apache.iotdb.db.engine.bufferwrite.Action;
+import org.apache.iotdb.db.engine.tsfiledata.TsFileProcessor;
+import org.apache.iotdb.db.engine.version.VersionController;
+import org.apache.iotdb.db.exception.BufferWriteProcessorException;
+import org.apache.iotdb.tsfile.write.schema.FileSchema;
+
+public class OverflowProcessor extends TsFileProcessor {
+
+  /**
+   * constructor of BufferWriteProcessor. data will be stored in baseDir/processorName/ folder.
+   *
+   * @param processorName processor name
+   * @param fileSchemaRef file schema
+   * @throws BufferWriteProcessorException BufferWriteProcessorException
+   */
+  public OverflowProcessor(String processorName,
+      Action beforeFlushAction,
+      Action afterFlushAction,
+      Action afterCloseAction,
+      VersionController versionController,
+      FileSchema fileSchemaRef)
+      throws BufferWriteProcessorException, IOException {
+    super(processorName, beforeFlushAction, afterFlushAction, afterCloseAction, versionController,
+        fileSchemaRef);
+  }
+
+
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/tsfiledata/TsFileProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/tsfiledata/TsFileProcessor.java
index 7ef1001..488e100 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/tsfiledata/TsFileProcessor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/tsfiledata/TsFileProcessor.java
@@ -122,12 +122,16 @@ public class TsFileProcessor extends Processor {
   private TsFileResource currentResource;
 
   private List<TsFileResource> tsFileResources;
+  // device -> datafiles
   private Map<String, List<TsFileResource>> inverseIndexOfResource;
 
+  // device -> min timestamp in current data file
   private Map<String, Long> minWrittenTimeForEachDeviceInCurrentFile;
+  // device -> max timestamp in current data file (maybe in memory)
   private Map<String, Long> maxWrittenTimeForEachDeviceInCurrentFile;
+  // device -> the max timestamp in current data file on disk
   private Map<String, Long> lastFlushedTimeForEachDevice;
-
+  //wal
   private WriteLogNode logNode;
   private VersionController versionController;
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/monitor/MonitorConstants.java b/iotdb/src/main/java/org/apache/iotdb/db/monitor/MonitorConstants.java
index ec899c8..ecbb27e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/monitor/MonitorConstants.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/monitor/MonitorConstants.java
@@ -94,7 +94,8 @@ public class MonitorConstants {
 
   public enum FileSizeConstants {
     DATA(Monitor.INSTANCE.getBaseDirectory()),
-    OVERFLOW(new File(config.getOverflowDataDir()).getAbsolutePath()),
+    //TODO FIXME this is absolutely wrong!
+    OVERFLOW(Monitor.INSTANCE.getBaseDirectory() + File.separatorChar + "overflow"),
     SETTLED(Monitor.INSTANCE.getBaseDirectory() + File.separatorChar + "settled"),
     WAL(new File(config.getWalFolder()).getAbsolutePath()),
     INFO(new File(config.getFileNodeDir()).getAbsolutePath()),
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
index 32d9b47..8709b0a 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java
@@ -257,7 +257,7 @@ public class OpenFileNumUtil {
     TOTAL_OPEN_FILE_NUM(null),
     DATA_OPEN_FILE_NUM(Collections.singletonList(config.getDataDir())),
     DELTA_OPEN_FILE_NUM(directories.getAllTsFileFolders()),
-    OVERFLOW_OPEN_FILE_NUM(Collections.singletonList(config.getOverflowDataDir())),
+    OVERFLOW_OPEN_FILE_NUM(directories.getAllOverflowFileFolders()),
     WAL_OPEN_FILE_NUM(Collections.singletonList(config.getWalFolder())),
     METADATA_OPEN_FILE_NUM(Collections.singletonList(config.getMetadataDir())),
     DIGEST_OPEN_FILE_NUM(Collections.singletonList(config.getFileNodeDir())),
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/PathUtils.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/PathUtils.java
index 315586b..dcf9b52 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/PathUtils.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/PathUtils.java
@@ -29,7 +29,7 @@ public class PathUtils {
   private static Directories directories = Directories.getInstance();
 
   public static File getBufferWriteDir(String nameSpacePath) {
-    String bufferwriteDirPath = directories.getFolderForTest();
+    String bufferwriteDirPath = directories.getTsFolderForTest();
     if (bufferwriteDirPath.length() > 0
         && bufferwriteDirPath.charAt(bufferwriteDirPath.length() - 1) != File.separatorChar) {
       bufferwriteDirPath = bufferwriteDirPath + File.separatorChar;
@@ -39,13 +39,13 @@ public class PathUtils {
     return dataDir;
   }
 
-  public static File getOverflowWriteDir(String nameSpacePath) {
-    String overflowWriteDir = config.getOverflowDataDir();
+  public static File getOverflowWriteDir(String processName) {
+    String overflowWriteDir = Directories.getInstance().getOverflowFolderForTest();
     if (overflowWriteDir.length() > 0
         && overflowWriteDir.charAt(overflowWriteDir.length() - 1) != File.separatorChar) {
       overflowWriteDir = overflowWriteDir + File.separatorChar;
     }
-    String dataDirPath = overflowWriteDir + nameSpacePath;
+    String dataDirPath = overflowWriteDir + processName;
     File dataDir = new File(dataDirPath);
     return dataDir;
   }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java
index 8cbf640..6299215 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java
@@ -101,7 +101,7 @@ public class BufferWriteProcessorNewTest {
   @Test
   public void testWriteAndFlush()
       throws BufferWriteProcessorException, WriteProcessException, IOException, InterruptedException {
-    bufferwrite = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(),
+    bufferwrite = new BufferWriteProcessor(Directories.getInstance().getTsFolderForTest(),
         processorName, filename,
         parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(processorName));
@@ -157,7 +157,7 @@ public class BufferWriteProcessorNewTest {
 
     // test recovery
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        Directories.getInstance().getFolderForTest(), processorName, filename, parameters,
+        Directories.getInstance().getTsFolderForTest(), processorName, filename, parameters,
         SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(processorName));
     pair = bufferWriteProcessor.queryBufferWriteData(processorName, measurementId, dataType, props);
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java
index f058562..3956736 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java
@@ -120,7 +120,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndAbnormalRecover()
       throws WriteProcessException, InterruptedException, IOException, ProcessorException {
-    bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directories.getTsFolderForTest(), deviceId, insertPath,
         parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     for (int i = 1; i < 100; i++) {
@@ -145,7 +145,7 @@ public class BufferWriteProcessorTest {
     bufferwrite.close();
     file.renameTo(restoreFile);
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        directories.getFolderForTest(), deviceId,
+        directories.getTsFolderForTest(), deviceId,
         insertPath, parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     assertTrue(insertFile.exists());
@@ -164,7 +164,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndNormalRecover()
       throws WriteProcessException, ProcessorException, InterruptedException {
-    bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directories.getTsFolderForTest(), deviceId, insertPath,
         parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     for (int i = 1; i < 100; i++) {
@@ -178,7 +178,7 @@ public class BufferWriteProcessorTest {
     File restoreFile = new File(dataFile, restoreFilePath);
     assertTrue(restoreFile.exists());
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        directories.getFolderForTest(), deviceId,
+        directories.getTsFolderForTest(), deviceId,
         insertPath, parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferWriteProcessor
@@ -196,7 +196,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndQuery()
       throws WriteProcessException, InterruptedException, ProcessorException {
-    bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directories.getTsFolderForTest(), deviceId, insertPath,
         parameters, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     assertFalse(bufferwrite.isFlush());
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java
index 9a2393c..ca9e59d 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java
@@ -135,7 +135,7 @@ public class BufferwriteFileSizeControlTest {
     parameters.put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, fnflushaction);
 
     try {
-      processor = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(), nsp,
+      processor = new BufferWriteProcessor(Directories.getInstance().getTsFolderForTest(), nsp,
           filename,
           parameters, SysTimeVersionController.INSTANCE, FileSchemaUtils.constructFileSchema(nsp));
     } catch (BufferWriteProcessorException e) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java
index 926f2cb..a800a80 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java
@@ -135,7 +135,7 @@ public class BufferwriteMetaSizeControlTest {
     parameters.put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, fnflushaction);
 
     try {
-      processor = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(), nsp,
+      processor = new BufferWriteProcessor(Directories.getInstance().getTsFolderForTest(), nsp,
           filename,
           parameters, SysTimeVersionController.INSTANCE, FileSchemaUtils.constructFileSchema(nsp));
     } catch (BufferWriteProcessorException e) {