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 2021/11/04 10:46:25 UTC

[iotdb] branch master updated: Add compaction version in cache key (#4298)

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

qiaojialin 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 d35eab4  Add compaction version in cache key (#4298)
d35eab4 is described below

commit d35eab40c123c3e628475d0d65416e6930b44810
Author: Jackie Tien <Ja...@foxmail.com>
AuthorDate: Thu Nov 4 18:45:59 2021 +0800

    Add compaction version in cache key (#4298)
---
 .../apache/iotdb/hadoop/fileSystem/HDFSInput.java  |  5 +++
 .../apache/iotdb/hadoop/tsfile/TSFHadoopTest.java  | 49 +++++++++++++++++++---
 .../apache/iotdb/hive/TSFHiveInputFormatTest.java  | 27 +++++++++++-
 .../apache/iotdb/hive/TSFHiveRecordReaderTest.java | 28 ++++++++++++-
 .../db/engine/cache/TimeSeriesMetadataCache.java   | 10 +++--
 .../iotdb/db/engine/cache/ChunkCacheTest.java      |  6 ++-
 .../compaction/inner/InnerCompactionCacheTest.java |  2 +-
 .../iotdb/db/tools/TsFileSketchToolTest.java       | 15 ++++++-
 .../iotdb/tsfile/file/metadata/ChunkMetadata.java  | 12 +++++-
 .../fileOutputFactory/LocalFSOutputFactory.java    |  4 +-
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |  1 +
 .../iotdb/tsfile/read/reader/LocalTsFileInput.java |  5 +++
 .../iotdb/tsfile/read/reader/TsFileInput.java      |  2 +
 .../apache/iotdb/tsfile/utils/FilePathUtils.java   | 37 ++++++++++++++--
 14 files changed, 181 insertions(+), 22 deletions(-)

diff --git a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
index a0480b1..242087b 100644
--- a/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
+++ b/hadoop/src/main/java/org/apache/iotdb/hadoop/fileSystem/HDFSInput.java
@@ -139,4 +139,9 @@ public class HDFSInput implements TsFileInput {
     fsDataInputStream.seek(srcPosition);
     return res;
   }
+
+  @Override
+  public String getFilePath() {
+    return path.toString();
+  }
 }
diff --git a/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TSFHadoopTest.java b/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TSFHadoopTest.java
index 0a6c35c..b86e812 100644
--- a/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TSFHadoopTest.java
+++ b/hadoop/src/test/java/org/apache/iotdb/hadoop/tsfile/TSFHadoopTest.java
@@ -20,9 +20,17 @@ package org.apache.iotdb.hadoop.tsfile;
 
 import org.apache.iotdb.hadoop.fileSystem.HDFSInput;
 import org.apache.iotdb.hadoop.tsfile.constant.TestConstant;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.BooleanWritable;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -31,27 +39,50 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class TSFHadoopTest {
 
   private TSFInputFormat inputFormat = null;
 
-  private String tsfilePath = TestConstant.BASE_OUTPUT_PATH.concat("example_mr.tsfile");
+  private final String tsfilePath =
+      TestConstant.BASE_OUTPUT_PATH
+          .concat("data")
+          .concat(File.separator)
+          .concat("data")
+          .concat(File.separator)
+          .concat("sequence")
+          .concat(File.separator)
+          .concat("root.sg1")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("1-0-0-0.tsfile");
+
+  private FSType beforeFSType;
 
   @Before
   public void setUp() {
 
+    System.out.println("tsfilePath: " + tsfilePath);
     TsFileTestHelper.deleteTsFile(tsfilePath);
     inputFormat = new TSFInputFormat();
   }
 
   @After
   public void tearDown() {
-
     TsFileTestHelper.deleteTsFile(tsfilePath);
   }
 
@@ -120,6 +151,8 @@ public class TSFHadoopTest {
       // set input path to the job
       TSFInputFormat.setInputPaths(job, tsfilePath);
       List<InputSplit> inputSplits = inputFormat.getSplits(job);
+      beforeFSType = TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs();
+      TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(FSType.HDFS);
       TsFileSequenceReader reader =
           new TsFileSequenceReader(new HDFSInput(tsfilePath, job.getConfiguration()));
       System.out.println(reader.readFileMetadata());
@@ -131,6 +164,8 @@ public class TSFHadoopTest {
     } catch (IOException e) {
       e.printStackTrace();
       fail(e.getMessage());
+    } finally {
+      TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(beforeFSType);
     }
   }
 
@@ -148,6 +183,8 @@ public class TSFHadoopTest {
       TSFInputFormat.setReadDeviceId(job, false);
       TSFInputFormat.setReadTime(job, false);
       List<InputSplit> inputSplits = inputFormat.getSplits(job);
+      beforeFSType = TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs();
+      TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(FSType.HDFS);
       TsFileSequenceReader reader =
           new TsFileSequenceReader(new HDFSInput(tsfilePath, job.getConfiguration()));
 
@@ -185,6 +222,8 @@ public class TSFHadoopTest {
     } catch (IOException | TSFHadoopException | InterruptedException e) {
       e.printStackTrace();
       fail(e.getMessage());
+    } finally {
+      TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(beforeFSType);
     }
   }
 }
diff --git a/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveInputFormatTest.java b/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveInputFormatTest.java
index a77e208..57d1c4f 100644
--- a/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveInputFormatTest.java
+++ b/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveInputFormatTest.java
@@ -20,6 +20,8 @@ package org.apache.iotdb.hive;
 
 import org.apache.iotdb.hadoop.tsfile.TSFInputSplit;
 import org.apache.iotdb.hive.constant.TestConstant;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.MapWritable;
@@ -32,16 +34,34 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
 import java.io.IOException;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class TSFHiveInputFormatTest {
 
   private TSFInputSplit inputSplit;
   private TSFHiveInputFormat inputFormat;
   private JobConf job;
-  private String filePath = TestConstant.BASE_OUTPUT_PATH.concat("test.tsfile");
+  private FSType beforeFSType;
+  private final String filePath =
+      TestConstant.BASE_OUTPUT_PATH
+          .concat("data")
+          .concat(File.separator)
+          .concat("data")
+          .concat(File.separator)
+          .concat("sequence")
+          .concat(File.separator)
+          .concat("root.sg1")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("1-0-0-0.tsfile");
 
   @Before
   public void setUp() {
@@ -54,11 +74,14 @@ public class TSFHiveInputFormatTest {
     Path path = new Path(jobPath);
     String[] hosts = {"127.0.0.1"};
     inputSplit = new TSFInputSplit(path, hosts, 0, 3727688L);
+    beforeFSType = TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs();
+    TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(FSType.HDFS);
   }
 
   @After
   public void tearDown() {
     TsFileTestHelper.deleteTsFile(filePath);
+    TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(beforeFSType);
   }
 
   @Test
diff --git a/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveRecordReaderTest.java b/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveRecordReaderTest.java
index aaef034..8480dcd 100644
--- a/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveRecordReaderTest.java
+++ b/hive-connector/src/test/java/org/apache/iotdb/hive/TSFHiveRecordReaderTest.java
@@ -20,6 +20,8 @@ package org.apache.iotdb.hive;
 
 import org.apache.iotdb.hadoop.tsfile.TSFInputSplit;
 import org.apache.iotdb.hive.constant.TestConstant;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
@@ -31,16 +33,35 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
 import java.io.IOException;
 
 import static org.apache.iotdb.hadoop.tsfile.TSFInputFormat.READ_DELTAOBJECTS;
 import static org.apache.iotdb.hadoop.tsfile.TSFInputFormat.READ_MEASUREMENTID;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class TSFHiveRecordReaderTest {
 
   private TSFHiveRecordReader tsfHiveRecordReader;
-  private String filePath = TestConstant.BASE_OUTPUT_PATH.concat("test.tsfile");
+  private final String filePath =
+      TestConstant.BASE_OUTPUT_PATH
+          .concat("data")
+          .concat(File.separator)
+          .concat("data")
+          .concat(File.separator)
+          .concat("sequence")
+          .concat(File.separator)
+          .concat("root.sg1")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("1-0-0-0.tsfile");
+  private FSType beforeFSType;
 
   @Before
   public void setUp() throws IOException {
@@ -48,6 +69,8 @@ public class TSFHiveRecordReaderTest {
     JobConf job = new JobConf();
     Path path = new Path(filePath);
     String[] hosts = {"127.0.0.1"};
+    beforeFSType = TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs();
+    TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(FSType.HDFS);
     TSFInputSplit inputSplit = new TSFInputSplit(path, hosts, 0, 3727528L);
     String[] deviceIds = {"device_1"}; // configure reading which deviceIds
     job.set(READ_DELTAOBJECTS, String.join(",", deviceIds));
@@ -70,6 +93,7 @@ public class TSFHiveRecordReaderTest {
   @After
   public void tearDown() {
     TsFileTestHelper.deleteTsFile(filePath);
+    TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(beforeFSType);
   }
 
   @Test
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
index cd5497e..247474b 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
@@ -402,15 +402,18 @@ public class TimeSeriesMetadataCache {
     private final String filePath;
     private final String tsFilePrefixPath;
     private final long tsFileVersion;
+    // high 32 bit is compaction level, low 32 bit is merge count
+    private final long compactionVersion;
     private final String device;
     private final String measurement;
 
     public TimeSeriesMetadataCacheKey(String filePath, String device, String measurement) {
       this.filePath = filePath;
-      Pair<String, Long> tsFilePrefixPathAndTsFileVersionPair =
+      Pair<String, long[]> tsFilePrefixPathAndTsFileVersionPair =
           FilePathUtils.getTsFilePrefixPathAndTsFileVersionPair(filePath);
       this.tsFilePrefixPath = tsFilePrefixPathAndTsFileVersionPair.left;
-      this.tsFileVersion = tsFilePrefixPathAndTsFileVersionPair.right;
+      this.tsFileVersion = tsFilePrefixPathAndTsFileVersionPair.right[0];
+      this.compactionVersion = tsFilePrefixPathAndTsFileVersionPair.right[1];
       this.device = device;
       this.measurement = measurement;
     }
@@ -427,12 +430,13 @@ public class TimeSeriesMetadataCache {
       return Objects.equals(measurement, that.measurement)
           && Objects.equals(device, that.device)
           && tsFileVersion == that.tsFileVersion
+          && compactionVersion == that.compactionVersion
           && tsFilePrefixPath.equals(that.tsFilePrefixPath);
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(tsFilePrefixPath, tsFileVersion, device, measurement);
+      return Objects.hash(tsFilePrefixPath, tsFileVersion, compactionVersion, device, measurement);
     }
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkCacheTest.java b/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkCacheTest.java
index 2b13dd7..3db886c 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkCacheTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/cache/ChunkCacheTest.java
@@ -49,7 +49,9 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
 import static org.junit.Assert.assertTrue;
@@ -122,8 +124,8 @@ public class ChunkCacheTest {
     Assert.assertEquals(chunk1.getHeader(), chunk2.getHeader());
     Assert.assertEquals(chunk1.getData(), chunk2.getData());
 
-    chunkMetadataKey.setFilePath(null);
     try {
+      chunkMetadataKey.setFilePath(null);
       chunkCache.get(chunkMetadataKey);
       fail();
     } catch (NullPointerException e) {
diff --git a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionCacheTest.java b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionCacheTest.java
index a1c1e39..3e051cb 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionCacheTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/InnerCompactionCacheTest.java
@@ -128,8 +128,8 @@ public class InnerCompactionCacheTest extends InnerCompactionTest {
             CompactionTaskManager.currentTaskNum);
     sizeTieredCompactionTask.call();
 
-    firstChunkMetadata.setFilePath(null);
     try {
+      firstChunkMetadata.setFilePath(null);
       ChunkCache.getInstance().get(firstChunkMetadata);
       fail();
     } catch (NullPointerException e) {
diff --git a/server/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java b/server/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
index ecd0de1..92eb368 100644
--- a/server/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/tools/TsFileSketchToolTest.java
@@ -41,7 +41,20 @@ import java.util.ArrayList;
 import java.util.List;
 
 public class TsFileSketchToolTest {
-  String path = "test.tsfile";
+  String path =
+      "data"
+          .concat(File.separator)
+          .concat("data")
+          .concat(File.separator)
+          .concat("sequence")
+          .concat(File.separator)
+          .concat("root.sg1")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("0")
+          .concat(File.separator)
+          .concat("1-0-0-0.tsfile");;
   String sketchOut = "sketch.out";
   String device = "root.device_0";
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
index 33908b6..1683c8d 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetadata.java
@@ -25,6 +25,7 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
 import org.apache.iotdb.tsfile.read.controller.IChunkLoader;
 import org.apache.iotdb.tsfile.utils.FilePathUtils;
+import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.RamUsageEstimator;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
@@ -78,6 +79,8 @@ public class ChunkMetadata implements Accountable, IChunkMetadata {
 
   // used for ChunkCache, Eg:"root.sg1/0/0"
   private String tsFilePrefixPath;
+  // high 32 bit is compaction level, low 32 bit is merge count
+  private long compactionVersion;
 
   public ChunkMetadata() {}
 
@@ -246,12 +249,13 @@ public class ChunkMetadata implements Accountable, IChunkMetadata {
     ChunkMetadata that = (ChunkMetadata) o;
     return offsetOfChunkHeader == that.offsetOfChunkHeader
         && version == that.version
+        && compactionVersion == that.compactionVersion
         && tsFilePrefixPath.equals(that.tsFilePrefixPath);
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(tsFilePrefixPath, version, offsetOfChunkHeader);
+    return Objects.hash(tsFilePrefixPath, version, compactionVersion, offsetOfChunkHeader);
   }
 
   @Override
@@ -326,8 +330,12 @@ public class ChunkMetadata implements Accountable, IChunkMetadata {
   public void setFilePath(String filePath) {
     this.filePath = filePath;
 
+    Pair<String, long[]> tsFilePrefixPathAndTsFileVersionPair =
+        FilePathUtils.getTsFilePrefixPathAndTsFileVersionPair(filePath);
     // set tsFilePrefixPath
-    tsFilePrefixPath = FilePathUtils.getTsFilePrefixPath(filePath);
+    tsFilePrefixPath = tsFilePrefixPathAndTsFileVersionPair.left;
+    this.version = tsFilePrefixPathAndTsFileVersionPair.right[0];
+    this.compactionVersion = tsFilePrefixPathAndTsFileVersionPair.right[1];
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/LocalFSOutputFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/LocalFSOutputFactory.java
index b8b7c1a..492c003 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/LocalFSOutputFactory.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/LocalFSOutputFactory.java
@@ -36,7 +36,9 @@ public class LocalFSOutputFactory implements FileOutputFactory {
   @Override
   public TsFileOutput getTsFileOutput(String filePath, boolean append) {
     try {
-      return new LocalTsFileOutput(new FileOutputStream(new File(filePath), append));
+      File file = new File(filePath);
+      file.getParentFile().mkdirs();
+      return new LocalTsFileOutput(new FileOutputStream(file, append));
     } catch (IOException e) {
       logger.error("Failed to get TsFile output of file: {}, ", filePath, e);
       return null;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index 7588b84..2a836b4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -162,6 +162,7 @@ public class TsFileSequenceReader implements AutoCloseable {
    */
   public TsFileSequenceReader(TsFileInput input, boolean loadMetadataSize) throws IOException {
     this.tsFileInput = input;
+    this.file = input.getFilePath();
     try {
       if (loadMetadataSize) { // NOTE no autoRepair here
         loadMetadataSize();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
index eb68160..3771e86 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
@@ -148,4 +148,9 @@ public class LocalTsFileInput implements TsFileInput {
     strBuffer.get(bytes, 0, strLength);
     return new String(bytes, 0, strLength);
   }
+
+  @Override
+  public String getFilePath() {
+    return filePath;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
index 6877c4f..8572f20 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/TsFileInput.java
@@ -127,4 +127,6 @@ public interface TsFileInput {
 
   /** read a string from the Input at the given position */
   String readVarIntString(long offset) throws IOException;
+
+  String getFilePath();
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FilePathUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FilePathUtils.java
index 87e8e03..e4a3ccf 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FilePathUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FilePathUtils.java
@@ -19,13 +19,21 @@
 
 package org.apache.iotdb.tsfile.utils;
 
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
 
 import java.io.File;
 
+import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX;
+
 public class FilePathUtils {
 
-  private static final String PATH_SPLIT_STRING = File.separator.equals("\\") ? "\\\\" : "/";
+  private static final String PATH_SPLIT_STRING =
+      TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs() == FSType.LOCAL
+              && File.separator.equals("\\")
+          ? "\\\\"
+          : "/";
   public static final String FILE_NAME_SEPARATOR = "-";
 
   private FilePathUtils() {
@@ -103,6 +111,24 @@ public class FilePathUtils {
     return Long.parseLong(names[1]);
   }
 
+  /**
+   * @return a long array whose length is 2, the first long value is tsfile version, second long
+   *     value is compaction version, high 32 bit is in-space compaction count, low 32 bit is
+   *     cross-space compaction count
+   */
+  private static long[] splitAndGetVersionArray(String tsFileName) {
+    String[] names = tsFileName.split(FILE_NAME_SEPARATOR);
+    long[] versionArray = new long[2];
+    if (names.length != 4) {
+      return versionArray;
+    }
+    versionArray[0] = Long.parseLong(names[1]);
+    versionArray[1] =
+        (Long.parseLong(names[2]) << 32)
+            | Long.parseLong(names[3].substring(0, names[3].length() - TSFILE_SUFFIX.length()));
+    return versionArray;
+  }
+
   public static Pair<String, Long> getLogicalSgNameAndTimePartitionIdPair(
       String tsFileAbsolutePath) {
     String[] pathSegments = splitTsFilePath(tsFileAbsolutePath);
@@ -111,7 +137,12 @@ public class FilePathUtils {
         Long.parseLong(pathSegments[pathSegments.length - 2]));
   }
 
-  public static Pair<String, Long> getTsFilePrefixPathAndTsFileVersionPair(
+  /**
+   * pair.left tsFilePrefixPath, like data/data/sequence/root.sg1/0/0 pair.right is a long array
+   * whose length is 2 pair.right[0] is tsfile version pair.right[1] is compaction version, high 32
+   * bit is compaction level, low 32 bit is merge count
+   */
+  public static Pair<String, long[]> getTsFilePrefixPathAndTsFileVersionPair(
       String tsFileAbsolutePath) {
     String[] pathSegments = splitTsFilePath(tsFileAbsolutePath);
     int pathLength = pathSegments.length;
@@ -121,6 +152,6 @@ public class FilePathUtils {
             + pathSegments[pathLength - 3]
             + File.separator
             + pathSegments[pathLength - 2],
-        splitAndGetTsFileVersion(pathSegments[pathLength - 1]));
+        splitAndGetVersionArray(pathSegments[pathLength - 1]));
   }
 }