You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by he...@apache.org on 2023/05/08 14:37:47 UTC

[iotdb] branch tiered_storage updated: add hybrid mode in tsfile

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

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


The following commit(s) were added to refs/heads/tiered_storage by this push:
     new 0f80c3c850 add hybrid mode in tsfile
0f80c3c850 is described below

commit 0f80c3c850168f41424ebf4a91e25992e2ab5574
Author: HeimingZ <zh...@qq.com>
AuthorDate: Mon May 8 22:37:28 2023 +0800

    add hybrid mode in tsfile
---
 .../iotdb/tsfile/fileSystem/FSFactoryProducer.java |  31 +-----
 .../tsfile/fileSystem/{FSType.java => FSPath.java} |  22 +++-
 .../org/apache/iotdb/tsfile/fileSystem/FSType.java |   3 +-
 .../fileInputFactory/HybridFileInputFactory.java   |  48 ++++++++
 .../OSFileInputFactory.java}                       |  20 +++-
 .../fileOutputFactory/HybridFileOutputFactory.java |  47 ++++++++
 .../OSFileOutputFactory.java}                      |  13 ++-
 .../fileSystem/fsFactory/HybridFSFactory.java      | 123 +++++++++++++++++++++
 .../tsfile/fileSystem/fsFactory/OSFSFactory.java   |  97 ++++++++++++++++
 .../org/apache/iotdb/tsfile/utils/FSUtils.java     |  97 ++++++++++++++++
 10 files changed, 460 insertions(+), 41 deletions(-)

diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSFactoryProducer.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSFactoryProducer.java
index bd497b8a2f..2a8711efe1 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSFactoryProducer.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSFactoryProducer.java
@@ -19,36 +19,17 @@
 
 package org.apache.iotdb.tsfile.fileSystem;
 
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.fileSystem.fileInputFactory.FileInputFactory;
-import org.apache.iotdb.tsfile.fileSystem.fileInputFactory.HDFSInputFactory;
-import org.apache.iotdb.tsfile.fileSystem.fileInputFactory.LocalFSInputFactory;
+import org.apache.iotdb.tsfile.fileSystem.fileInputFactory.HybridFileInputFactory;
 import org.apache.iotdb.tsfile.fileSystem.fileOutputFactory.FileOutputFactory;
-import org.apache.iotdb.tsfile.fileSystem.fileOutputFactory.HDFSOutputFactory;
-import org.apache.iotdb.tsfile.fileSystem.fileOutputFactory.LocalFSOutputFactory;
+import org.apache.iotdb.tsfile.fileSystem.fileOutputFactory.HybridFileOutputFactory;
 import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory;
-import org.apache.iotdb.tsfile.fileSystem.fsFactory.HDFSFactory;
-import org.apache.iotdb.tsfile.fileSystem.fsFactory.LocalFSFactory;
+import org.apache.iotdb.tsfile.fileSystem.fsFactory.HybridFSFactory;
 
 public class FSFactoryProducer {
-
-  private static FSType fSType = TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs();
-
-  private static FSFactory fsFactory;
-  private static FileInputFactory fileInputFactory;
-  private static FileOutputFactory fileOutputFactory;
-
-  static {
-    if (fSType.equals(FSType.HDFS)) {
-      fsFactory = new HDFSFactory();
-      fileInputFactory = new HDFSInputFactory();
-      fileOutputFactory = new HDFSOutputFactory();
-    } else {
-      fsFactory = new LocalFSFactory();
-      fileInputFactory = new LocalFSInputFactory();
-      fileOutputFactory = new LocalFSOutputFactory();
-    }
-  }
+  private static FSFactory fsFactory = new HybridFSFactory();
+  private static FileInputFactory fileInputFactory = new HybridFileInputFactory();
+  private static FileOutputFactory fileOutputFactory = new HybridFileOutputFactory();
 
   public static FSFactory getFSFactory() {
     return fsFactory;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSPath.java
similarity index 70%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
copy to tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSPath.java
index 764df632d0..cf26786f60 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSPath.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,10 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.iotdb.tsfile.fileSystem;
 
-public enum FSType {
-  LOCAL,
-  HDFS
+public class FSPath {
+  private final FSType fsType;
+  private final String path;
+
+  public FSPath(FSType fsType, String path) {
+    this.fsType = fsType;
+    this.path = path;
+  }
+
+  public FSType getFsType() {
+    return fsType;
+  }
+
+  public String getPath() {
+    return path;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
index 764df632d0..22e275c4c4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
@@ -21,5 +21,6 @@ package org.apache.iotdb.tsfile.fileSystem;
 
 public enum FSType {
   LOCAL,
-  HDFS
+  HDFS,
+  OBJECT_STORAGE,
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/HybridFileInputFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/HybridFileInputFactory.java
new file mode 100644
index 0000000000..f9140af71e
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/HybridFileInputFactory.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tsfile.fileSystem.fileInputFactory;
+
+import org.apache.iotdb.tsfile.fileSystem.FSPath;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
+import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.utils.FSUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class HybridFileInputFactory implements FileInputFactory {
+  private static final Logger logger = LoggerFactory.getLogger(HybridFileInputFactory.class);
+  private static final Map<FSType, FileInputFactory> inputFactories = new HashMap<>();
+
+  static {
+    inputFactories.put(FSType.LOCAL, new LocalFSInputFactory());
+    inputFactories.put(FSType.HDFS, new HDFSInputFactory());
+    inputFactories.put(FSType.OBJECT_STORAGE, new OSFileInputFactory());
+  }
+
+  @Override
+  public TsFileInput getTsFileInput(String filePath) throws IOException {
+    FSPath path = FSUtils.parse(filePath);
+    return inputFactories.get(path.getFsType()).getTsFileInput(path.getPath());
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/OSFileInputFactory.java
similarity index 59%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
copy to tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/OSFileInputFactory.java
index 764df632d0..05793508c7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileInputFactory/OSFileInputFactory.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,10 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.tsfile.fileSystem.fileInputFactory;
 
-package org.apache.iotdb.tsfile.fileSystem;
+import org.apache.iotdb.tsfile.read.reader.TsFileInput;
 
-public enum FSType {
-  LOCAL,
-  HDFS
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+public class OSFileInputFactory implements FileInputFactory {
+  private static final Logger logger = LoggerFactory.getLogger(OSFileInputFactory.class);
+
+  @Override
+  public TsFileInput getTsFileInput(String filePath) throws IOException {
+    return null;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/HybridFileOutputFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/HybridFileOutputFactory.java
new file mode 100644
index 0000000000..80e628bd39
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/HybridFileOutputFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.tsfile.fileSystem.fileOutputFactory;
+
+import org.apache.iotdb.tsfile.fileSystem.FSPath;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
+import org.apache.iotdb.tsfile.utils.FSUtils;
+import org.apache.iotdb.tsfile.write.writer.TsFileOutput;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class HybridFileOutputFactory implements FileOutputFactory {
+  private static final Logger logger = LoggerFactory.getLogger(HybridFileOutputFactory.class);
+  private static final Map<FSType, FileOutputFactory> outputFactories = new ConcurrentHashMap<>();
+
+  static {
+    outputFactories.put(FSType.LOCAL, new LocalFSOutputFactory());
+    outputFactories.put(FSType.HDFS, new HDFSOutputFactory());
+    outputFactories.put(FSType.OBJECT_STORAGE, new OSFileOutputFactory());
+  }
+
+  @Override
+  public TsFileOutput getTsFileOutput(String filePath, boolean append) {
+    FSPath path = FSUtils.parse(filePath);
+    return outputFactories.get(path.getFsType()).getTsFileOutput(path.getPath(), append);
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/OSFileOutputFactory.java
similarity index 69%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
copy to tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/OSFileOutputFactory.java
index 764df632d0..73409ed39a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/FSType.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fileOutputFactory/OSFileOutputFactory.java
@@ -7,7 +7,7 @@
  * "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
+ *     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
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.tsfile.fileSystem.fileOutputFactory;
 
-package org.apache.iotdb.tsfile.fileSystem;
+import org.apache.iotdb.tsfile.write.writer.TsFileOutput;
 
-public enum FSType {
-  LOCAL,
-  HDFS
+public class OSFileOutputFactory implements FileOutputFactory {
+  @Override
+  public TsFileOutput getTsFileOutput(String filePath, boolean append) {
+    return null;
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/HybridFSFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/HybridFSFactory.java
new file mode 100644
index 0000000000..48c338796f
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/HybridFSFactory.java
@@ -0,0 +1,123 @@
+/*
+ * 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.tsfile.fileSystem.fsFactory;
+
+import org.apache.iotdb.tsfile.fileSystem.FSPath;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
+import org.apache.iotdb.tsfile.utils.FSUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class HybridFSFactory implements FSFactory {
+  private static final Logger logger = LoggerFactory.getLogger(HybridFSFactory.class);
+  private static final Map<FSType, FSFactory> fsFactories = new ConcurrentHashMap<>();
+
+  static {
+    fsFactories.put(FSType.LOCAL, new LocalFSFactory());
+    fsFactories.put(FSType.HDFS, new HDFSFactory());
+    fsFactories.put(FSType.OBJECT_STORAGE, new OSFSFactory());
+  }
+
+  @Override
+  public File getFileWithParent(String pathname) {
+    FSPath path = FSUtils.parse(pathname);
+    return fsFactories.get(path.getFsType()).getFileWithParent(path.getPath());
+  }
+
+  @Override
+  public File getFile(String pathname) {
+    FSPath path = FSUtils.parse(pathname);
+    return fsFactories.get(path.getFsType()).getFile(path.getPath());
+  }
+
+  @Override
+  public File getFile(String parent, String child) {
+    FSPath parentPath = FSUtils.parse(parent);
+    return fsFactories.get(parentPath.getFsType()).getFile(parentPath.getPath(), child);
+  }
+
+  @Override
+  public File getFile(File parent, String child) {
+    FSType type = FSUtils.getFSType(parent);
+    return fsFactories.get(type).getFile(parent, child);
+  }
+
+  @Override
+  public File getFile(URI uri) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public BufferedReader getBufferedReader(String filePath) {
+    FSPath path = FSUtils.parse(filePath);
+    return fsFactories.get(path.getFsType()).getBufferedReader(path.getPath());
+  }
+
+  @Override
+  public BufferedWriter getBufferedWriter(String filePath, boolean append) {
+    FSPath path = FSUtils.parse(filePath);
+    return fsFactories.get(path.getFsType()).getBufferedWriter(path.getPath(), append);
+  }
+
+  @Override
+  public BufferedInputStream getBufferedInputStream(String filePath) {
+    FSPath path = FSUtils.parse(filePath);
+    return fsFactories.get(path.getFsType()).getBufferedInputStream(path.getPath());
+  }
+
+  @Override
+  public BufferedOutputStream getBufferedOutputStream(String filePath) {
+    FSPath path = FSUtils.parse(filePath);
+    return fsFactories.get(path.getFsType()).getBufferedOutputStream(path.getPath());
+  }
+
+  @Override
+  public void moveFile(File srcFile, File destFile) {
+    // TODO
+  }
+
+  @Override
+  public File[] listFilesBySuffix(String fileFolder, String suffix) {
+    FSPath folder = FSUtils.parse(fileFolder);
+    return fsFactories.get(folder.getFsType()).listFilesBySuffix(folder.getPath(), suffix);
+  }
+
+  @Override
+  public File[] listFilesByPrefix(String fileFolder, String prefix) {
+    FSPath folder = FSUtils.parse(fileFolder);
+    return fsFactories.get(folder.getFsType()).listFilesByPrefix(folder.getPath(), prefix);
+  }
+
+  @Override
+  public boolean deleteIfExists(File file) throws IOException {
+    FSType type = FSUtils.getFSType(file);
+    return fsFactories.get(type).deleteIfExists(file);
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/OSFSFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/OSFSFactory.java
new file mode 100644
index 0000000000..8114adfb79
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/fileSystem/fsFactory/OSFSFactory.java
@@ -0,0 +1,97 @@
+/*
+ * 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.tsfile.fileSystem.fsFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+
+public class OSFSFactory implements FSFactory {
+  private static final Logger logger = LoggerFactory.getLogger(OSFSFactory.class);
+
+  @Override
+  public File getFileWithParent(String pathname) {
+    return null;
+  }
+
+  @Override
+  public File getFile(String pathname) {
+    return null;
+  }
+
+  @Override
+  public File getFile(String parent, String child) {
+    return null;
+  }
+
+  @Override
+  public File getFile(File parent, String child) {
+    return null;
+  }
+
+  @Override
+  public File getFile(URI uri) {
+    return null;
+  }
+
+  @Override
+  public BufferedReader getBufferedReader(String filePath) {
+    return null;
+  }
+
+  @Override
+  public BufferedWriter getBufferedWriter(String filePath, boolean append) {
+    return null;
+  }
+
+  @Override
+  public BufferedInputStream getBufferedInputStream(String filePath) {
+    return null;
+  }
+
+  @Override
+  public BufferedOutputStream getBufferedOutputStream(String filePath) {
+    return null;
+  }
+
+  @Override
+  public void moveFile(File srcFile, File destFile) {}
+
+  @Override
+  public File[] listFilesBySuffix(String fileFolder, String suffix) {
+    return new File[0];
+  }
+
+  @Override
+  public File[] listFilesByPrefix(String fileFolder, String prefix) {
+    return new File[0];
+  }
+
+  @Override
+  public boolean deleteIfExists(File file) throws IOException {
+    return false;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FSUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FSUtils.java
new file mode 100644
index 0000000000..4513b9f8be
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/FSUtils.java
@@ -0,0 +1,97 @@
+/*
+ * 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.tsfile.utils;
+
+import org.apache.iotdb.tsfile.fileSystem.FSPath;
+import org.apache.iotdb.tsfile.fileSystem.FSType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+
+public class FSUtils {
+  private static final Logger logger = LoggerFactory.getLogger(FSUtils.class);
+
+  private static final FSType[] fsTypes = {FSType.OBJECT_STORAGE, FSType.HDFS};
+  public static final String[] fsPrefix = {"os://", "hdfs://"};
+  private static final String[] fsFileClassName = {
+    "org.apache.iotdb.os.fileSystem.OSFile", "org.apache.iotdb.hadoop.fileSystem.HDFSFile"
+  };
+  private static final Class<?>[] fsFileClass = new Class[fsTypes.length];
+
+  private FSUtils() {}
+
+  static {
+    for (int i = 0; i < fsTypes.length; ++i) {
+      try {
+        fsFileClass[i] = Class.forName(fsFileClassName[i]);
+      } catch (ClassNotFoundException e) {
+        // TODO
+        logger.info(
+            "Failed to get "
+                + fsTypes[i].name()
+                + " file system. Please check your dependency of "
+                + fsTypes[i].name()
+                + " module.",
+            e);
+      }
+    }
+  }
+
+  public static FSType getFSType(File file) {
+    for (int i = 0; i < fsTypes.length; ++i) {
+      if (fsFileClass[i] != null && fsFileClass[i].isInstance(file)) {
+        return fsTypes[i];
+      }
+    }
+    return FSType.LOCAL;
+  }
+
+  public static FSType getFSType(String fsPath) {
+    for (int i = 0; i < fsTypes.length; ++i) {
+      if (fsPath.startsWith(fsPrefix[i])) {
+        return fsTypes[i];
+      }
+    }
+    return FSType.LOCAL;
+  }
+
+  public static String getPath(String fsPath) {
+    for (int i = 0; i < fsTypes.length; ++i) {
+      if (fsPath.startsWith(fsPrefix[i])) {
+        return fsPath.substring(fsPrefix[i].length());
+      }
+    }
+    return fsPath;
+  }
+
+  public static FSPath parse(String fsPath) {
+    FSType type = FSType.LOCAL;
+    String path = fsPath;
+    for (int i = 0; i < fsTypes.length; ++i) {
+      if (fsPath.startsWith(fsPrefix[i])) {
+        type = fsTypes[i];
+        path = fsPath.substring(fsPrefix[i].length());
+        break;
+      }
+    }
+    return new FSPath(type, path);
+  }
+}