You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2019/06/22 11:37:38 UTC

[incubator-iotdb] 01/03: add overflow directory control to DirectoryManager

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

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

commit 6cb5502627152fcfa176835f8282c4a2386ef8bf
Author: qiaojialin <64...@qq.com>
AuthorDate: Sat Jun 22 19:25:47 2019 +0800

    add overflow directory control to DirectoryManager
---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 23 +++++---
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  2 +
 .../db/conf/directories/DirectoryManager.java      | 59 ++++++++++++++----
 .../apache/iotdb/db/monitor/MonitorConstants.java  |  3 +-
 .../java/org/apache/iotdb/db/service/IoTDB.java    |  5 +-
 .../org/apache/iotdb/db/service/JDBCService.java   |  4 +-
 ...cServiceManager.java => SyncServerManager.java} | 69 ++++------------------
 .../db/sync/receiver/SyncServiceEventHandler.java  | 55 -----------------
 .../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 +-
 .../iotdb/tsfile/write/TsFileReadWriteTest.java    |  4 +-
 15 files changed, 101 insertions(+), 151 deletions(-)

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 991b5ae..454a8f0 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
@@ -34,7 +34,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 = "unorder";
+  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";
@@ -86,7 +86,7 @@ public class IoTDBConfig {
   /**
    * Data directory of Overflow data.
    */
-  private String overflowDataDir = "overflow";
+  private String[] overflowDataDirs = {DEFAULT_OVERFLOW_DIR};
 
   /**
    * Data directory of fileNode data.
@@ -309,7 +309,16 @@ 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});
@@ -500,12 +509,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 e62b8ea..f6ecec3 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
@@ -131,6 +131,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/DirectoryManager.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
index 1c17c79..3a575f1 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
@@ -37,21 +37,28 @@ public class DirectoryManager {
   private static final Logger LOGGER = LoggerFactory.getLogger(DirectoryManager.class);
 
   private List<String> tsfileFolders;
-  private DirectoryStrategy strategy;
+  private List<String> overflowFolders;
+  private DirectoryStrategy tsfileStrategy;
+  private DirectoryStrategy overflowStrategy;
 
   private DirectoryManager() {
     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 +66,22 @@ public class DirectoryManager {
     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 +95,7 @@ public class DirectoryManager {
    * @return next folder index
    */
   public int getNextFolderIndexForTsFile() {
-    return strategy.nextFolderIndex();
+    return tsfileStrategy.nextFolderIndex();
   }
 
   public String getTsFileFolder(int index) {
@@ -110,4 +117,34 @@ public class DirectoryManager {
   public String getWALFolder() {
     return IoTDBDescriptor.getInstance().getConfig().getWalFolder();
   }
+
+  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);
+  }
 }
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 c7ed8c1..129aa91 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/service/IoTDB.java b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index 3e70542..ed1fce8 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -27,7 +27,8 @@ import org.apache.iotdb.db.exception.FileNodeManagerException;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.builder.ExceptionBuilder;
 import org.apache.iotdb.db.monitor.StatMonitor;
-import org.apache.iotdb.db.sync.receiver.SyncServiceManager;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.sync.receiver.SyncServerManager;
 import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -95,7 +96,7 @@ public class IoTDB implements IoTDBMBean {
     registerManager.register(CloseMergeService.getInstance());
     registerManager.register(StatMonitor.getInstance());
     registerManager.register(BasicMemController.getInstance());
-    registerManager.register(SyncServiceManager.getInstance());
+    registerManager.register(SyncServerManager.getInstance());
 
     JMXService.registerMBean(getInstance(), mbeanName);
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java b/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java
index f75c350..315510a 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java
@@ -219,9 +219,9 @@ public class JDBCService implements JDBCServiceMBean, IService {
         close();
         // TODO debug log, will be deleted in production env
         if(threadStopLatch == null) {
-          LOGGER.info("JDBC Service Stop Count Down latch is null");
+          LOGGER.info("Stop Count Down latch is null");
         } else {
-          LOGGER.info("JDBC Service Stop Count Down latch is {}", threadStopLatch.getCount());
+          LOGGER.info("Stop Count Down latch is {}", threadStopLatch.getCount());
         }
         // debug log, will be deleted in production env
 
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceManager.java b/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
similarity index 63%
rename from iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceManager.java
rename to iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
index 15a500d..11292ea 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
@@ -19,7 +19,6 @@
 package org.apache.iotdb.db.sync.receiver;
 
 import java.net.InetSocketAddress;
-import java.util.concurrent.CountDownLatch;
 import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.concurrent.ThreadName;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -42,18 +41,16 @@ import org.slf4j.LoggerFactory;
 /**
  * sync receiver server.
  */
-public class SyncServiceManager implements IService {
+public class SyncServerManager implements IService {
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(SyncServiceManager.class);
+  private static final Logger LOGGER = LoggerFactory.getLogger(SyncServerManager.class);
   private Thread syncServerThread;
   private IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
-  private CountDownLatch startLatch;
-  private CountDownLatch stopLatch;
 
-  private SyncServiceManager() {
+  private SyncServerManager() {
   }
 
-  public static final SyncServiceManager getInstance() {
+  public static final SyncServerManager getInstance() {
     return ServerManagerHolder.INSTANCE;
   }
 
@@ -71,27 +68,10 @@ public class SyncServiceManager implements IService {
       return;
     }
     conf.setIpWhiteList(conf.getIpWhiteList().replaceAll(" ", ""));
-    resetLatch();
-    try {
-      syncServerThread = new SyncServiceThread(startLatch, stopLatch);
-      syncServerThread.setName(ThreadName.SYNC_SERVER.getName());
-      syncServerThread.start();
-      startLatch.await();
-    } catch (InterruptedException e) {
-      String errorMessage = String
-          .format("Failed to start %s because of %s", this.getID().getName(),
-              e.getMessage());
-      LOGGER.error(errorMessage);
-      throw new StartupException(errorMessage);
-    }
-    LOGGER
-        .info("{}: start {} successfully, listening on ip {} port {}", IoTDBConstant.GLOBAL_DB_NAME,
-            this.getID().getName(), conf.getRpcAddress(), conf.getSyncServerPort());
-  }
-
-  private void resetLatch(){
-    startLatch = new CountDownLatch(1);
-    stopLatch = new CountDownLatch(1);
+    syncServerThread = new SyncServiceThread();
+    syncServerThread.setName(ThreadName.SYNC_SERVER.getName());
+    syncServerThread.start();
+    LOGGER.info("Sync server has started.");
   }
 
   /**
@@ -99,21 +79,9 @@ public class SyncServiceManager implements IService {
    */
   @Override
   public void stop() {
-    if (!conf.isSyncEnable()) {
-      return;
-    }
-    ((SyncServiceThread) syncServerThread).close();
-    LOGGER.info("{}: closing {}...", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName());
-    if (syncServerThread != null) {
+    if (conf.isSyncEnable()) {
       ((SyncServiceThread) syncServerThread).close();
     }
-    try {
-      stopLatch.await();
-      resetLatch();
-      LOGGER.info("{}: setCloseMark {} successfully", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName());
-    } catch (InterruptedException e) {
-      LOGGER.error("{}: setCloseMark {} failed because {}", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName(), e);
-    }
   }
 
   @Override
@@ -123,7 +91,7 @@ public class SyncServiceManager implements IService {
 
   private static class ServerManagerHolder {
 
-    private static final SyncServiceManager INSTANCE = new SyncServiceManager();
+    private static final SyncServerManager INSTANCE = new SyncServerManager();
   }
 
   private class SyncServiceThread extends Thread {
@@ -133,13 +101,9 @@ public class SyncServiceManager implements IService {
     private Factory protocolFactory;
     private Processor<SyncService.Iface> processor;
     private TThreadPoolServer.Args poolArgs;
-    private CountDownLatch threadStartLatch;
-    private CountDownLatch threadStopLatch;
 
-    public SyncServiceThread(CountDownLatch threadStartLatch, CountDownLatch threadStopLatch) {
-      this.processor = new SyncService.Processor<>(new SyncServiceImpl());
-      this.threadStartLatch = threadStartLatch;
-      this.threadStopLatch = threadStopLatch;
+    public SyncServiceThread() {
+      processor = new SyncService.Processor<>(new SyncServiceImpl());
     }
 
     @Override
@@ -155,7 +119,6 @@ public class SyncServiceManager implements IService {
         poolArgs.protocolFactory(protocolFactory);
         poolArgs.processor(processor);
         poolServer = new TThreadPoolServer(poolArgs);
-        poolServer.setServerEventHandler(new SyncServiceEventHandler(threadStartLatch));
         poolServer.serve();
       } catch (TTransportException e) {
         LOGGER.error("{}: failed to start {}, because ", IoTDBConstant.GLOBAL_DB_NAME,
@@ -164,14 +127,6 @@ public class SyncServiceManager implements IService {
         LOGGER.error("{}: {} exit, because ", IoTDBConstant.GLOBAL_DB_NAME, getID().getName(), e);
       } finally {
         close();
-        if(threadStopLatch == null) {
-          LOGGER.info("Sync Service Stop Count Down latch is null");
-        } else {
-          LOGGER.info("Sync Service Stop Count Down latch is {}", threadStopLatch.getCount());
-        }
-        if (threadStopLatch != null && threadStopLatch.getCount() == 1) {
-          threadStopLatch.countDown();
-        }
         LOGGER.info("{}: setCloseMark TThreadPoolServer and TServerSocket for {}",
             IoTDBConstant.GLOBAL_DB_NAME, getID().getName());
       }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceEventHandler.java b/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceEventHandler.java
deleted file mode 100644
index cb77a20..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceEventHandler.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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.sync.receiver;
-
-import java.util.concurrent.CountDownLatch;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.server.ServerContext;
-import org.apache.thrift.server.TServerEventHandler;
-import org.apache.thrift.transport.TTransport;
-
-public class SyncServiceEventHandler implements TServerEventHandler {
-
-  private CountDownLatch startLatch;
-
-  public SyncServiceEventHandler(CountDownLatch startLatch) {
-    this.startLatch = startLatch;
-  }
-
-  @Override
-  public void preServe() {
-    startLatch.countDown();
-  }
-
-  @Override
-  public ServerContext createContext(TProtocol input, TProtocol output) {
-    return null;
-  }
-
-  @Override
-  public void deleteContext(ServerContext serverContext, TProtocol input, TProtocol output) {
-    // Do nothing.
-  }
-
-  @Override
-  public void processContext(ServerContext serverContext, TTransport inputTransport,
-      TTransport outputTransport) {
-    // Do nothing.
-  }
-}
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 438226e..577e6d8 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(directoryManager.getAllTsFileFolders()),
-    OVERFLOW_OPEN_FILE_NUM(Collections.singletonList(config.getOverflowDataDir())),
+    OVERFLOW_OPEN_FILE_NUM(directoryManager.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 3fb8826..08fafc7 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 DirectoryManager directoryManager = DirectoryManager.getInstance();
 
   public static File getBufferWriteDir(String nameSpacePath) {
-    String bufferwriteDirPath = directoryManager.getFolderForTest();
+    String bufferwriteDirPath = directoryManager.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 = DirectoryManager.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 43930b2..e3e3815 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
@@ -93,7 +93,7 @@ public class BufferWriteProcessorNewTest {
   @Test
   public void testWriteAndFlush()
       throws BufferWriteProcessorException, WriteProcessException, IOException, InterruptedException {
-    bufferwrite = new BufferWriteProcessor(DirectoryManager.getInstance().getFolderForTest(),
+    bufferwrite = new BufferWriteProcessor(DirectoryManager.getInstance().getTsFolderForTest(),
         processorName, filename,
         parameters, x->{},  SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(processorName));
@@ -148,7 +148,7 @@ public class BufferWriteProcessorNewTest {
 
     // test recovery
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        DirectoryManager.getInstance().getFolderForTest(), processorName, filename, parameters, x->{},
+        DirectoryManager.getInstance().getTsFolderForTest(), processorName, filename, parameters, x->{},
         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 3fd05a4..b9d50fb 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
@@ -124,7 +124,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndAbnormalRecover()
       throws WriteProcessException, InterruptedException, IOException, ProcessorException {
-    bufferwrite = new BufferWriteProcessor(directoryManager.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directoryManager.getTsFolderForTest(), deviceId, insertPath,
         parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     for (int i = 1; i < 100; i++) {
@@ -154,7 +154,7 @@ public class BufferWriteProcessorTest {
     }
     file.renameTo(restoreFile);
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        directoryManager.getFolderForTest(), deviceId,
+        directoryManager.getTsFolderForTest(), deviceId,
         insertPath, parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     assertTrue(insertFile.exists());
@@ -178,7 +178,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndNormalRecover()
       throws WriteProcessException, ProcessorException, InterruptedException {
-    bufferwrite = new BufferWriteProcessor(directoryManager.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directoryManager.getTsFolderForTest(), deviceId, insertPath,
         parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     for (int i = 1; i < 100; i++) {
@@ -192,7 +192,7 @@ public class BufferWriteProcessorTest {
     File restoreFile = new File(dataFile, restoreFilePath);
     assertTrue(restoreFile.exists());
     BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor(
-        directoryManager.getFolderForTest(), deviceId,
+        directoryManager.getTsFolderForTest(), deviceId,
         insertPath, parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     Pair<ReadOnlyMemChunk, List<ChunkMetaData>> pair = bufferWriteProcessor
@@ -215,7 +215,7 @@ public class BufferWriteProcessorTest {
   @Test
   public void testWriteAndQuery()
       throws WriteProcessException, InterruptedException, ProcessorException {
-    bufferwrite = new BufferWriteProcessor(directoryManager.getFolderForTest(), deviceId, insertPath,
+    bufferwrite = new BufferWriteProcessor(directoryManager.getTsFolderForTest(), deviceId, insertPath,
         parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
         FileSchemaUtils.constructFileSchema(deviceId));
     assertTrue(bufferwrite.canBeClosed());
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 89fa997..34fa60c 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
@@ -137,7 +137,7 @@ public class BufferwriteFileSizeControlTest {
     parameters.put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, fnflushaction);
 
     try {
-      processor = new BufferWriteProcessor(DirectoryManager.getInstance().getFolderForTest(), nsp,
+      processor = new BufferWriteProcessor(DirectoryManager.getInstance().getTsFolderForTest(), nsp,
           filename,
           parameters, bfcloseConsumer, SysTimeVersionController.INSTANCE,
           FileSchemaUtils.constructFileSchema(nsp));
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 35c466f..84da4a6 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(DirectoryManager.getInstance().getFolderForTest(), nsp,
+      processor = new BufferWriteProcessor(DirectoryManager.getInstance().getTsFolderForTest(), nsp,
           filename,
           parameters, x->{}, SysTimeVersionController.INSTANCE, FileSchemaUtils.constructFileSchema(nsp));
     } catch (BufferWriteProcessorException e) {
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java
index 416f2bc..8220c2a 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java
@@ -49,8 +49,8 @@ import org.junit.Test;
 public class TsFileReadWriteTest {
 
   private final double delta = 0.0000001;
-  String path = "read_write_rle.tsfile";
-  File f;
+  private String path = "read_write_rle.tsfile";
+  private File f;
 
   @Before
   public void setUp() throws Exception {