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 2021/01/23 17:12:47 UTC

[iotdb] branch master updated: fix the bug that when introduce the virtual sg, the path not consist with the tsfile's in snapshot (#2533)

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

hxd 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 2023c20  fix the bug that when introduce the virtual sg, the path not consist with the tsfile's in snapshot (#2533)
2023c20 is described below

commit 2023c20f499f8416586e3def4a47bd7c94504d89
Author: HouliangQi <ne...@163.com>
AuthorDate: Sun Jan 24 01:12:22 2021 +0800

    fix the bug that when introduce the virtual sg, the path not consist with the tsfile's in snapshot (#2533)
    
    * change method name of getVirtualStorageGroupName to getVirtualStorageGroupId in FilePathUtils
---
 .../iotdb/cluster/log/snapshot/FileSnapshot.java   |  38 +++----
 .../cluster/query/filter/SlotTsFileFilter.java     |  16 +--
 .../org/apache/iotdb/cluster/common/TestUtils.java |   6 +-
 .../cluster/server/member/DataGroupMemberTest.java |  24 +++--
 .../org/apache/iotdb/db/utils/FilePathUtils.java   |  77 +++++++++++---
 .../iotdb/db/integration/IoTDBFilePathUtilsIT.java | 109 +++++++++++++++++++
 .../apache/iotdb/db/utils/FilePathUtilsTest.java   | 117 +++++++++++++++++++++
 7 files changed, 332 insertions(+), 55 deletions(-)

diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
index fe3a7a0..d837441 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/snapshot/FileSnapshot.java
@@ -60,6 +60,7 @@ import org.apache.iotdb.db.exception.metadata.IllegalPathException;
 import org.apache.iotdb.db.metadata.PartialPath;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.db.utils.SchemaUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.TimeseriesSchema;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
@@ -295,13 +296,11 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
      * @return
      */
     private boolean isFileAlreadyPulled(RemoteTsFileResource resource) throws IllegalPathException {
-      String[] pathSegments = FilePathUtils.splitTsFilePath(resource);
-      int segSize = pathSegments.length;
-      // <storageGroupName>/<partitionNum>/<fileName>
-      String storageGroupName = pathSegments[segSize - 3];
-      long partitionNumber = Long.parseLong(pathSegments[segSize - 2]);
+      Pair<String, Long> sgNameAndTimePartitionIdPair = FilePathUtils
+          .getLogicalSgNameAndTimePartitionIdPair(resource);
       return StorageEngine.getInstance()
-          .isFileAlreadyExist(resource, new PartialPath(storageGroupName), partitionNumber);
+          .isFileAlreadyExist(resource, new PartialPath(sgNameAndTimePartitionIdPair.left),
+              sgNameAndTimePartitionIdPair.right);
     }
 
     private void removeRemoteHardLink(RemoteTsFileResource resource) {
@@ -380,11 +379,10 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
      * @param resource
      */
     private void loadRemoteResource(RemoteTsFileResource resource) throws IllegalPathException {
-      // the new file is stored at:
-      // remote/<nodeIdentifier>/<storageGroupName>/<partitionNum>/<fileName>
-      String[] pathSegments = FilePathUtils.splitTsFilePath(resource);
-      int segSize = pathSegments.length;
-      PartialPath storageGroupName = new PartialPath(pathSegments[segSize - 3]);
+      // the new file is stored at: remote/<nodeIdentifier>/<FilePathUtils.getTsFilePrefixPath(resource)>/<tsfile>
+      // you can see FilePathUtils.splitTsFilePath() method for details.
+      PartialPath storageGroupName = new PartialPath(
+          FilePathUtils.getLogicalStorageGroupName(resource));
       File remoteModFile =
           new File(resource.getTsFile().getAbsoluteFile() + ModificationFile.FILE_SUFFIX);
       try {
@@ -430,24 +428,20 @@ public class FileSnapshot extends Snapshot implements TimeseriesSchemaSnapshot {
     private File pullRemoteFile(RemoteTsFileResource resource, Node node) throws IOException {
       logger.info("{}: pulling remote file {} from {}, plan index [{}, {}]", name, resource, node
           , resource.getMinPlanIndex(), resource.getMaxPlanIndex());
-
-      String[] pathSegments = FilePathUtils.splitTsFilePath(resource);
-      int segSize = pathSegments.length;
       // the new file is stored at:
-      // remote/<nodeIdentifier>/<storageGroupName>/<partitionNum>/<fileName>
-      // the file in the snapshot is a hardlink, remove the hardlink suffix
-      String tempFileName = pathSegments[segSize - 1].substring(0,
-          pathSegments[segSize - 1].lastIndexOf('.'));
+      // remote/<nodeIdentifier>/<FilePathUtils.getTsFilePrefixPath(resource)>/<newTsFile>
+      // you can see FilePathUtils.splitTsFilePath() method for details.
+      String tempFileName = FilePathUtils.getTsFileNameWithoutHardLink(resource);
       String tempFilePath =
-          node.getNodeIdentifier() + File.separator + pathSegments[segSize - 3] +
-              File.separator + pathSegments[segSize - 2] + File.separator + tempFileName;
+          node.getNodeIdentifier() + File.separator + FilePathUtils.getTsFilePrefixPath(resource)
+              + File.separator + tempFileName;
       File tempFile = new File(REMOTE_FILE_TEMP_DIR, tempFilePath);
       tempFile.getParentFile().mkdirs();
-      File tempModFile = new File(REMOTE_FILE_TEMP_DIR,
-          tempFilePath + ModificationFile.FILE_SUFFIX);
       if (pullRemoteFile(resource.getTsFile().getAbsolutePath(), node, tempFile)) {
         // TODO-Cluster#353: implement file examination, may be replaced with other algorithm
         if (resource.isWithModification()) {
+          File tempModFile = new File(REMOTE_FILE_TEMP_DIR,
+              tempFilePath + ModificationFile.FILE_SUFFIX);
           pullRemoteFile(resource.getModFile().getFilePath(), node, tempModFile);
         }
         return tempFile;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
index 188fa44..3bae96e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/filter/SlotTsFileFilter.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.cluster.partition.slot.SlotPartitionTable;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.query.filter.TsFileFilter;
 import org.apache.iotdb.db.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,16 +43,15 @@ public class SlotTsFileFilter implements TsFileFilter {
     return fileNotInSlots(resource, slots);
   }
 
-  private static boolean fileNotInSlots(TsFileResource res, List<Integer> nodeSlots) {
-    // <storageGroupName>/<virtualStorageGroupNumber>/<partitionNum>/<fileName>
-    String[] pathSegments = FilePathUtils.splitTsFilePath(res);
-    String storageGroupName = pathSegments[pathSegments.length - 4];
-    int partitionNum = Integer.parseInt(pathSegments[pathSegments.length - 2]);
+  private static boolean fileNotInSlots(TsFileResource resource, List<Integer> nodeSlots) {
+    Pair<String, Long> sgNameAndPartitionIdPair = FilePathUtils
+        .getLogicalSgNameAndTimePartitionIdPair(resource);
     int slot = SlotPartitionTable.getSlotStrategy()
-        .calculateSlotByPartitionNum(storageGroupName, partitionNum,
-        ClusterConstant.SLOT_NUM);
+        .calculateSlotByPartitionNum(sgNameAndPartitionIdPair.left, sgNameAndPartitionIdPair.right,
+            ClusterConstant.SLOT_NUM);
     boolean contained = nodeSlots.contains(slot);
-    logger.debug("The slot of {} is {}, contained: {}", res.getTsFile().getPath(), slot, contained);
+    logger.debug("The slot of {} is {}, contained: {}", resource.getTsFile().getPath(), slot,
+        contained);
     return !contained;
   }
 }
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java b/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
index 771112b..20278ac 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/common/TestUtils.java
@@ -343,12 +343,16 @@ public class TestUtils {
     }
   }
 
+
+  /**
+   * The TsFileResource's path should be consist with the {@link org.apache.iotdb.db.utils.FilePathUtils#splitTsFilePath(TsFileResource)}
+   */
   public static List<TsFileResource> prepareTsFileResources(int sgNum, int fileNum, int seriesNum,
       int ptNum, boolean asHardLink) throws IOException, WriteProcessException {
     List<TsFileResource> ret = new ArrayList<>();
     for (int i = 0; i < fileNum; i++) {
       String fileName = "target" + File.separator + "data" + File.separator + String
-          .format(TestUtils.getTestSg(sgNum) + File.separator + 0 +
+          .format(TestUtils.getTestSg(sgNum) + File.separator + 0 + File.separator + 0 +
               File.separator + "0-%d-0" + TsFileConstant.TSFILE_SUFFIX, i);
       if (asHardLink) {
         fileName = fileName + ".0_0";
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
index 6cc5418..18138d3 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/server/member/DataGroupMemberTest.java
@@ -133,8 +133,9 @@ public class DataGroupMemberTest extends MemberTest {
     snapshotMap = new HashMap<>();
     for (int i = 0; i < ClusterConstant.SLOT_NUM; i++) {
       FileSnapshot fileSnapshot = new FileSnapshot();
-      fileSnapshot.setTimeseriesSchemas(Collections.singletonList(TestUtils.getTestTimeSeriesSchema(0
-          , i)));
+      fileSnapshot
+          .setTimeseriesSchemas(Collections.singletonList(TestUtils.getTestTimeSeriesSchema(0
+              , i)));
       snapshotMap.put(i, fileSnapshot);
     }
     pulledSnapshots = new ConcurrentSkipListSet<>();
@@ -150,10 +151,12 @@ public class DataGroupMemberTest extends MemberTest {
   private PartitionedSnapshotLogManager getLogManager(PartitionGroup partitionGroup,
       DataGroupMember dataGroupMember) {
     return new TestPartitionedLogManager(new DataLogApplier(testMetaMember, dataGroupMember),
-        testMetaMember.getPartitionTable(), partitionGroup.getHeader(), FileSnapshot.Factory.INSTANCE) {
+        testMetaMember.getPartitionTable(), partitionGroup.getHeader(),
+        FileSnapshot.Factory.INSTANCE) {
       @Override
       public Snapshot getSnapshot(long minIndex) {
-        PartitionedSnapshot<FileSnapshot> snapshot = new PartitionedSnapshot<>(FileSnapshot.Factory.INSTANCE);
+        PartitionedSnapshot<FileSnapshot> snapshot = new PartitionedSnapshot<>(
+            FileSnapshot.Factory.INSTANCE);
         if (hasInitialSnapshots) {
           for (int i = 0; i < 100; i++) {
             snapshot.putSnapshot(i, snapshotMap.get(i));
@@ -226,7 +229,8 @@ public class DataGroupMemberTest extends MemberTest {
             }
 
             @Override
-            public void removeHardLink(String hardLinkPath, AsyncMethodCallback<Void> resultHandler) {
+            public void removeHardLink(String hardLinkPath,
+                AsyncMethodCallback<Void> resultHandler) {
               new Thread(() -> {
                 try {
                   Files.deleteIfExists(new File(hardLinkPath).toPath());
@@ -826,7 +830,8 @@ public class DataGroupMemberTest extends MemberTest {
     System.out.println("Start testGetPaths()");
     String path = TestUtils.getTestSg(0);
     AtomicReference<GetAllPathsResult> pathResult = new AtomicReference<>();
-    GenericHandler<GetAllPathsResult> handler = new GenericHandler<>(TestUtils.getNode(0), pathResult);
+    GenericHandler<GetAllPathsResult> handler = new GenericHandler<>(TestUtils.getNode(0),
+        pathResult);
     new DataAsyncService(dataGroupMember)
         .getAllPaths(TestUtils.getNode(0), Collections.singletonList(path), false, handler);
     List<String> result = pathResult.get().paths;
@@ -876,8 +881,8 @@ public class DataGroupMemberTest extends MemberTest {
       throws IOException, IllegalPathException {
     TsFileResource resource = new RemoteTsFileResource();
     String fileName =
-        "target" + File.separator + TestUtils.getTestSg(0) + File.separator + "0" + File.separator +
-    "0-" + serialNum + "-0.tsfile";
+        "target" + File.separator + TestUtils.getTestSg(0) + File.separator + "0" + File.separator
+            + "0" + File.separator + "0-" + serialNum + "-0.tsfile";
     if (asHardLink) {
       fileName = fileName + ".0_0";
     }
@@ -902,7 +907,8 @@ public class DataGroupMemberTest extends MemberTest {
   public void testRemoveLeader() {
     System.out.println("Start testRemoveLeader()");
     Node nodeToRemove = TestUtils.getNode(10);
-    SlotNodeRemovalResult nodeRemovalResult = (SlotNodeRemovalResult) testMetaMember.getPartitionTable()
+    SlotNodeRemovalResult nodeRemovalResult = (SlotNodeRemovalResult) testMetaMember
+        .getPartitionTable()
         .removeNode(nodeToRemove);
     dataGroupMember.setLeader(nodeToRemove);
     dataGroupMember.start();
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
index 64a8e7e..5c4df6f 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FilePathUtils.java
@@ -39,6 +39,7 @@ import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
 import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.utils.Pair;
 
 public class FilePathUtils {
 
@@ -50,10 +51,11 @@ public class FilePathUtils {
 
   /**
    * Format file path to end with File.separator
+   *
    * @param filePath origin file path
    * @return Regularized Path
    */
-  public static String regularizePath(String filePath){
+  public static String regularizePath(String filePath) {
     if (filePath.length() > 0
         && filePath.charAt(filePath.length() - 1) != File.separatorChar) {
       filePath = filePath + File.separatorChar;
@@ -61,25 +63,72 @@ public class FilePathUtils {
     return filePath;
   }
 
+  /**
+   * IMPORTANT, when the path of TsFile changes, the following methods should be changed
+   * accordingly. The sequence TsFile is located at ${IOTDB_DATA_DIR}/data/sequence/. The unsequence
+   * TsFile is located at ${IOTDB_DATA_DIR}/data/unsequence/. Where different storage group's TsFile
+   * is located at <logicalStorageGroupName>/<virtualStorageGroupName>/<timePartitionId>/<fileName>.
+   * For example, one sequence TsFile may locate at /data/data/sequence/root.group_9/0/0/1611199237113-4-0.tsfile
+   *
+   * @param resource the tsFileResource
+   */
   public static String[] splitTsFilePath(TsFileResource resource) {
     return resource.getTsFile().getAbsolutePath().split(PATH_SPLIT_STRING);
   }
 
+  public static String getLogicalStorageGroupName(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    return pathSegments[pathSegments.length - 4];
+  }
+
+  public static String getVirtualStorageGroupId(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    return pathSegments[pathSegments.length - 3];
+  }
+
+  public static long getTimePartitionId(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    return Long.parseLong(pathSegments[pathSegments.length - 2]);
+  }
+
+  /**
+   * @param resource the RemoteTsFileResource
+   * @return the file in the snapshot is a hardlink, remove the hardlink suffix
+   */
+  public static String getTsFileNameWithoutHardLink(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    return pathSegments[pathSegments.length - 1].substring(0,
+        pathSegments[pathSegments.length - 1].lastIndexOf(TsFileConstant.PATH_SEPARATOR));
+  }
+
+  public static String getTsFilePrefixPath(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    int pathLength = pathSegments.length;
+    return pathSegments[pathLength - 4] + File.separator + pathSegments[pathLength - 3]
+        + File.separator + pathSegments[pathLength - 2];
+  }
+
+  public static Pair<String, Long> getLogicalSgNameAndTimePartitionIdPair(TsFileResource resource) {
+    String[] pathSegments = splitTsFilePath(resource);
+    return new Pair<>(pathSegments[pathSegments.length - 4],
+        Long.parseLong(pathSegments[pathSegments.length - 2]));
+  }
+
   public static String[] splitTsFilePath(String tsFileAbsolutePath) {
     return tsFileAbsolutePath.split(PATH_SPLIT_STRING);
   }
 
   /**
-   * get paths from group by level, like root.sg1.d2.s0, root.sg1.d1.s0
-   * level=1, return [root.sg1.*.s0, 0] and pathIndex turns to be [[0, root.sg1.*.s0],
-   * [1, root.sg1.*.s0]]
-   * @param plan the original Aggregation Plan
+   * get paths from group by level, like root.sg1.d2.s0, root.sg1.d1.s0 level=1, return
+   * [root.sg1.*.s0, 0] and pathIndex turns to be [[0, root.sg1.*.s0], [1, root.sg1.*.s0]]
+   *
+   * @param plan      the original Aggregation Plan
    * @param pathIndex the mapping from index of aggregations to the result path name
    * @return
    */
   @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning
   public static Map<String, AggregateResult> getPathByLevel(AggregationPlan plan,
-          Map<Integer, String> pathIndex) throws QueryProcessException {
+      Map<Integer, String> pathIndex) throws QueryProcessException {
     // pathGroupByLevel -> count
     Map<String, AggregateResult> finalPaths = new TreeMap<>();
 
@@ -110,7 +159,7 @@ public class FilePathUtils {
         key = path.toString();
       }
       AggregateResult aggRet = AggregateResultFactory
-              .getAggrResultByName(plan.getAggregations().get(i), dataTypes.get(i));
+          .getAggrResultByName(plan.getAggregations().get(i), dataTypes.get(i));
       finalPaths.putIfAbsent(key, aggRet);
       pathIndex.put(i, key);
     }
@@ -119,10 +168,8 @@ public class FilePathUtils {
   }
 
   /**
-   * merge the raw record by level, for example
-   * raw record [timestamp, root.sg1.d1.s0, root.sg1.d1.s1, root.sg1.d2.s2], level=1
-   * and newRecord data is [100, 1, 1, 1]
-   * return [100, 3]
+   * merge the raw record by level, for example raw record [timestamp, root.sg1.d1.s0,
+   * root.sg1.d1.s1, root.sg1.d2.s2], level=1 and newRecord data is [100, 1, 1, 1] return [100, 3]
    *
    * @param newRecord
    * @param finalPaths
@@ -130,8 +177,8 @@ public class FilePathUtils {
    * @return
    */
   public static List<AggregateResult> mergeRecordByPath(
-          AggregationPlan plan, RowRecord newRecord, Map<String, AggregateResult> finalPaths,
-          Map<Integer, String> pathIndex) {
+      AggregationPlan plan, RowRecord newRecord, Map<String, AggregateResult> finalPaths,
+      Map<Integer, String> pathIndex) {
     if (newRecord.getFields().size() < finalPaths.size()) {
       return Collections.emptyList();
     }
@@ -173,8 +220,8 @@ public class FilePathUtils {
   }
 
   public static List<AggregateResult> mergeRecordByPath(
-          List<AggregateResult> aggResults, Map<String, AggregateResult> finalPaths,
-          Map<Integer, String> pathIndex) {
+      List<AggregateResult> aggResults, Map<String, AggregateResult> finalPaths,
+      Map<Integer, String> pathIndex) {
     if (aggResults.size() < finalPaths.size()) {
       return Collections.emptyList();
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFilePathUtilsIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFilePathUtilsIT.java
new file mode 100644
index 0000000..2dd0968
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFilePathUtilsIT.java
@@ -0,0 +1,109 @@
+/*
+ * 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.integration;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Objects;
+import org.apache.iotdb.db.engine.StorageEngine;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.db.utils.FilePathUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IoTDBFilePathUtilsIT {
+
+  private static Connection connection;
+  private static final Logger logger = LoggerFactory.getLogger(IoTDBFilePathUtilsIT.class);
+
+  @BeforeClass
+  public static void setUp() {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+  }
+
+  private void insertData() throws ClassNotFoundException, SQLException {
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+    Statement statement = connection.createStatement();
+
+    statement.execute("insert into root.sg1.wf01.wt01(timestamp, status) values (1000, true)");
+    statement.execute("insert into root.sg1.wf01.wt01(timestamp, status) values (2000, true)");
+    statement.execute("insert into root.sg1.wf01.wt01(timestamp, status) values (3000, true)");
+    statement.execute("flush");
+    statement.close();
+  }
+
+
+  @Test
+  public void splitTsFilePathTest()
+      throws StorageEngineException, SQLException, ClassNotFoundException {
+    insertData();
+    String storageGroupName = "root.sg1";
+    PartialPath sgPath = null;
+    try {
+      sgPath = new PartialPath(storageGroupName);
+    } catch (IllegalPathException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertNotNull(sgPath);
+    List<TsFileResource> tsFileResources = StorageEngine.getInstance().getProcessor(sgPath)
+        .getSequenceFileTreeSet();
+    Assert.assertNotNull(tsFileResources);
+
+    for (TsFileResource tsFileResource : tsFileResources) {
+      String sgName = FilePathUtils.getLogicalStorageGroupName(tsFileResource);
+      Assert.assertEquals(storageGroupName, sgName);
+
+      Pair<String, Long> logicalSgNameAndTimePartitionIdPair = FilePathUtils
+          .getLogicalSgNameAndTimePartitionIdPair(tsFileResource);
+      Assert.assertEquals(storageGroupName, logicalSgNameAndTimePartitionIdPair.left);
+    }
+  }
+
+  private static void close() {
+    if (Objects.nonNull(connection)) {
+      try {
+        connection.close();
+      } catch (Exception e) {
+        logger.error("close the connection failed,", e);
+      }
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    close();
+    EnvironmentUtils.cleanEnv();
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/utils/FilePathUtilsTest.java b/server/src/test/java/org/apache/iotdb/db/utils/FilePathUtilsTest.java
new file mode 100644
index 0000000..52c8383
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/utils/FilePathUtilsTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.utils;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.commons.io.FileUtils;
+import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class FilePathUtilsTest {
+
+  private static final String storageGroupName = "root.group_9";
+  private static final String virtualSgName = "1";
+  private static final long partitionId = 0;
+  private static final String tsFileName = "1611199237113-4-0.tsfile";
+  private static final String fullPath =
+      "target" + File.separator + storageGroupName + File.separator + virtualSgName + File.separator
+          + partitionId + File.separator + tsFileName;
+
+  private File tsFile;
+
+  @Before
+  public void setUp() {
+    tsFile = new File(fullPath);
+    boolean success = false;
+    try {
+      FileUtils.forceMkdirParent(tsFile);
+      success = tsFile.createNewFile();
+    } catch (IOException e) {
+      Assert.fail(e.getMessage());
+    }
+    Assert.assertTrue(success);
+  }
+
+  @Test
+  public void getLogicalSgNameAndTimePartitionIdPairTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    Pair<String, Long> sgNameAndTimePartitionIdPair = FilePathUtils
+        .getLogicalSgNameAndTimePartitionIdPair(tsFileResource);
+    Assert.assertEquals(storageGroupName, sgNameAndTimePartitionIdPair.left);
+    Assert.assertEquals(partitionId, sgNameAndTimePartitionIdPair.right.longValue());
+  }
+
+  @Test
+  public void getLogicalStorageGroupNameTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    String tmpSgName = FilePathUtils.getLogicalStorageGroupName(tsFileResource);
+    Assert.assertEquals(storageGroupName, tmpSgName);
+  }
+
+  @Test
+  public void getVirtualStorageGroupNameTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    String tmpVirtualSgName = FilePathUtils.getVirtualStorageGroupId(tsFileResource);
+    Assert.assertEquals(virtualSgName, tmpVirtualSgName);
+  }
+
+  @Test
+  public void getTimePartitionIdTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    long tmpTimePartitionId = FilePathUtils.getTimePartitionId(tsFileResource);
+    Assert.assertEquals(partitionId, tmpTimePartitionId);
+  }
+
+  @Test
+  public void getTsFileNameWithoutHardLinkTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    TsFileResource newTsFileResource = tsFileResource.createHardlink();
+    String tsFileNameWithoutHardLink = FilePathUtils
+        .getTsFileNameWithoutHardLink(newTsFileResource);
+    Assert.assertEquals(tsFileName, tsFileNameWithoutHardLink);
+    Assert.assertTrue(newTsFileResource.getTsFile().delete());
+  }
+
+  @Test
+  public void getTsFilePrefixPathTest() {
+    TsFileResource tsFileResource = new TsFileResource();
+    tsFileResource.setFile(tsFile);
+    String tsFilePrefixPath = FilePathUtils
+        .getTsFilePrefixPath(tsFileResource);
+    String exceptPrefixPath =
+        storageGroupName + File.separator + virtualSgName + File.separator + partitionId;
+    Assert.assertEquals(exceptPrefixPath, tsFilePrefixPath);
+  }
+
+  @After
+  public void tearDown() throws IOException, StorageEngineException {
+    Assert.assertTrue(tsFile.delete());
+  }
+}