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

fanng pushed a commit to branch branch-gvfs-fuse-dev
in repository https://gitbox.apache.org/repos/asf/gravitino.git


The following commit(s) were added to refs/heads/branch-gvfs-fuse-dev by this 
push:
     new 827546566 [#5877] feat (gvfs-fuse): Implement a common filesystem 
layer  (#5878)
827546566 is described below

commit 82754656609b041af2747821c1e49b2bea57124d
Author: Yuhui <h...@datastrato.com>
AuthorDate: Tue Dec 24 11:21:51 2024 +0800

    [#5877] feat (gvfs-fuse): Implement a common filesystem layer  (#5878)
    
    ### What changes were proposed in this pull request?
    
    Implement a common filesystem layer to handle manage file ids, file name
    mappings, and file relationships. and delegate filesystem APIs to
    PathFilesystem.
    
    ### Why are the changes needed?
    
    Fix: #5877
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    Uts
---
 clients/filesystem-fuse/.cargo/config.toml         |   1 -
 clients/filesystem-fuse/Cargo.toml                 |   6 +-
 clients/filesystem-fuse/Makefile                   |  69 ++++
 clients/filesystem-fuse/build.gradle.kts           |  36 +-
 .../filesystem-fuse/src/default_raw_filesystem.rs  | 394 +++++++++++++++++++++
 clients/filesystem-fuse/src/filesystem.rs          | 134 +++++--
 clients/filesystem-fuse/src/fuse_api_handle.rs     |  17 +-
 clients/filesystem-fuse/src/lib.rs                 |   4 +
 clients/filesystem-fuse/src/main.rs                |  52 +++
 clients/filesystem-fuse/src/opened_file.rs         | 141 ++++++++
 clients/filesystem-fuse/src/opened_file_manager.rs | 111 ++++++
 clients/filesystem-fuse/src/utils.rs               |  67 ++++
 12 files changed, 969 insertions(+), 63 deletions(-)

diff --git a/clients/filesystem-fuse/.cargo/config.toml 
b/clients/filesystem-fuse/.cargo/config.toml
index 78bc9f7fe..9d5bb048e 100644
--- a/clients/filesystem-fuse/.cargo/config.toml
+++ b/clients/filesystem-fuse/.cargo/config.toml
@@ -16,5 +16,4 @@
 # under the License.
 
 [build]
-target-dir = "build"
 rustflags = ["-Adead_code", "-Aclippy::redundant-field-names"]
diff --git a/clients/filesystem-fuse/Cargo.toml 
b/clients/filesystem-fuse/Cargo.toml
index 2883cecc6..3bcf20f37 100644
--- a/clients/filesystem-fuse/Cargo.toml
+++ b/clients/filesystem-fuse/Cargo.toml
@@ -30,13 +30,15 @@ name = "gvfs-fuse"
 path = "src/main.rs"
 
 [lib]
-name="gvfs_fuse"
+name = "gvfs_fuse"
 
 [dependencies]
 async-trait = "0.1"
 bytes = "1.6.0"
-futures-util = "0.3.30"
+dashmap = "6.1.0"
 fuse3 = { version = "0.8.1", "features" = ["tokio-runtime", "unprivileged"] }
+futures-util = "0.3.30"
+libc = "0.2.168"
 log = "0.4.22"
 tokio = { version = "1.38.0", features = ["full"] }
 tracing-subscriber = { version = "0.3.18", features = ["env-filter"] }
diff --git a/clients/filesystem-fuse/Makefile b/clients/filesystem-fuse/Makefile
new file mode 100644
index 000000000..f4a4cef20
--- /dev/null
+++ b/clients/filesystem-fuse/Makefile
@@ -0,0 +1,69 @@
+# 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.
+
+.EXPORT_ALL_VARIABLES:
+
+.PHONY: build
+build:
+       cargo build --all-features --workspace
+
+fmt:
+       cargo fmt --all
+
+cargo-sort: install-cargo-sort
+       cargo sort -w
+
+fix-toml: install-taplo-cli
+       taplo fmt
+
+check-fmt:
+       cargo  fmt --all -- --check
+
+check-clippy:
+       cargo  clippy --all-targets --all-features --workspace -- -D warnings
+
+install-cargo-sort:
+       cargo install cargo-sort@1.0.9
+
+check-cargo-sort: install-cargo-sort
+       cargo sort -c
+
+install-cargo-machete:
+       cargo install cargo-machete
+
+cargo-machete: install-cargo-machete
+       cargo machete
+
+install-taplo-cli:
+       cargo install taplo-cli@0.9.0
+
+check-toml: install-taplo-cli
+       taplo check
+
+check: check-fmt check-clippy check-cargo-sort check-toml cargo-machete
+
+doc-test:
+       cargo test --no-fail-fast --doc --all-features --workspace
+
+unit-test: doc-test
+       cargo test --no-fail-fast --lib --all-features --workspace
+
+test: doc-test
+       cargo test --no-fail-fast --all-targets --all-features --workspace
+
+clean:
+       cargo clean
diff --git a/clients/filesystem-fuse/build.gradle.kts 
b/clients/filesystem-fuse/build.gradle.kts
index 08693ddc5..7d24c86a5 100644
--- a/clients/filesystem-fuse/build.gradle.kts
+++ b/clients/filesystem-fuse/build.gradle.kts
@@ -20,8 +20,6 @@
 import org.gradle.api.tasks.Exec
 
 val checkRustEnvironment by tasks.registering(Exec::class) {
-  description = "Check if Rust environment."
-  group = "verification"
   commandLine("bash", "-c", "cargo --version")
   standardOutput = System.out
   errorOutput = System.err
@@ -30,36 +28,30 @@ val checkRustEnvironment by tasks.registering(Exec::class) {
 
 val buildRustProject by tasks.registering(Exec::class) {
   dependsOn(checkRustEnvironment)
-  description = "Compile the Rust project"
   workingDir = file("$projectDir")
-  commandLine("bash", "-c", "cargo build --release")
+  commandLine("bash", "-c", "make build")
 }
 
 val checkRustProject by tasks.registering(Exec::class) {
   dependsOn(checkRustEnvironment)
-  description = "Check the Rust project"
   workingDir = file("$projectDir")
 
-  commandLine(
-    "bash",
-    "-c",
-    """
-          set -e
-          echo "Checking the code format"
-          cargo fmt --all -- --check
-
-          echo "Running clippy"
-          cargo clippy --all-targets --all-features --workspace -- -D warnings
-    """.trimIndent()
-  )
+  commandLine("bash", "-c", "make check")
 }
 
 val testRustProject by tasks.registering(Exec::class) {
   dependsOn(checkRustEnvironment)
-  description = "Run tests in the Rust project"
-  group = "verification"
   workingDir = file("$projectDir")
-  commandLine("bash", "-c", "cargo test --release")
+  commandLine("bash", "-c", "make test")
+
+  standardOutput = System.out
+  errorOutput = System.err
+}
+
+val cleanRustProject by tasks.registering(Exec::class) {
+  dependsOn(checkRustEnvironment)
+  workingDir = file("$projectDir")
+  commandLine("bash", "-c", "make clean")
 
   standardOutput = System.out
   errorOutput = System.err
@@ -85,3 +77,7 @@ tasks.named("check") {
 tasks.named("test") {
   dependsOn(testRustProject)
 }
+
+tasks.named("clean") {
+  dependsOn(cleanRustProject)
+}
diff --git a/clients/filesystem-fuse/src/default_raw_filesystem.rs 
b/clients/filesystem-fuse/src/default_raw_filesystem.rs
new file mode 100644
index 000000000..9a66cd551
--- /dev/null
+++ b/clients/filesystem-fuse/src/default_raw_filesystem.rs
@@ -0,0 +1,394 @@
+/*
+ * 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.
+ */
+use crate::filesystem::{FileStat, PathFileSystem, RawFileSystem, Result};
+use crate::opened_file::{FileHandle, OpenFileFlags};
+use crate::opened_file_manager::OpenedFileManager;
+use crate::utils::join_file_path;
+use async_trait::async_trait;
+use bytes::Bytes;
+use fuse3::{Errno, FileType};
+use std::collections::HashMap;
+use std::sync::atomic::AtomicU64;
+use tokio::sync::RwLock;
+
+/// DefaultRawFileSystem is a simple implementation for the file system.
+/// it is used to manage the file metadata and file handle.
+/// The operations of the file system are implemented by the PathFileSystem.
+pub struct DefaultRawFileSystem<T: PathFileSystem> {
+    /// file entries
+    file_entry_manager: RwLock<FileEntryManager>,
+    /// opened files
+    opened_file_manager: OpenedFileManager,
+    /// file id generator
+    file_id_generator: AtomicU64,
+
+    /// real filesystem
+    fs: T,
+}
+
+impl<T: PathFileSystem> DefaultRawFileSystem<T> {
+    const INITIAL_FILE_ID: u64 = 10000;
+    const ROOT_DIR_PARENT_FILE_ID: u64 = 1;
+    const ROOT_DIR_FILE_ID: u64 = 1;
+    const ROOT_DIR_NAME: &'static str = "";
+
+    pub(crate) fn new(fs: T) -> Self {
+        Self {
+            file_entry_manager: RwLock::new(FileEntryManager::new()),
+            opened_file_manager: OpenedFileManager::new(),
+            file_id_generator: AtomicU64::new(Self::INITIAL_FILE_ID),
+            fs,
+        }
+    }
+
+    fn next_file_id(&self) -> u64 {
+        self.file_id_generator
+            .fetch_add(1, std::sync::atomic::Ordering::SeqCst)
+    }
+
+    async fn get_file_entry(&self, file_id: u64) -> Result<FileEntry> {
+        self.file_entry_manager
+            .read()
+            .await
+            .get_file_entry_by_id(file_id)
+            .ok_or(Errno::from(libc::ENOENT))
+    }
+
+    async fn get_file_entry_by_path(&self, path: &str) -> Option<FileEntry> {
+        self.file_entry_manager
+            .read()
+            .await
+            .get_file_entry_by_path(path)
+    }
+
+    async fn resolve_file_id_to_filestat(&self, file_stat: &mut FileStat, 
parent_file_id: u64) {
+        let mut file_manager = self.file_entry_manager.write().await;
+        let file_entry = file_manager.get_file_entry_by_path(&file_stat.path);
+        match file_entry {
+            None => {
+                // allocate new file id
+                file_stat.set_file_id(parent_file_id, self.next_file_id());
+                file_manager.insert(file_stat.parent_file_id, 
file_stat.file_id, &file_stat.path);
+            }
+            Some(file) => {
+                // use the exist file id
+                file_stat.set_file_id(file.parent_file_id, file.file_id);
+            }
+        }
+    }
+
+    async fn open_file_internal(
+        &self,
+        file_id: u64,
+        flags: u32,
+        kind: FileType,
+    ) -> Result<FileHandle> {
+        let file_entry = self.get_file_entry(file_id).await?;
+
+        let mut opened_file = {
+            match kind {
+                FileType::Directory => {
+                    self.fs
+                        .open_dir(&file_entry.path, OpenFileFlags(flags))
+                        .await?
+                }
+                FileType::RegularFile => {
+                    self.fs
+                        .open_file(&file_entry.path, OpenFileFlags(flags))
+                        .await?
+                }
+                _ => return Err(Errno::from(libc::EINVAL)),
+            }
+        };
+        // set the exists file id
+        opened_file.set_file_id(file_entry.parent_file_id, file_id);
+        let file = self.opened_file_manager.put(opened_file);
+        let file = file.lock().await;
+        Ok(file.file_handle())
+    }
+
+    async fn remove_file_entry_locked(&self, path: &str) {
+        let mut file_manager = self.file_entry_manager.write().await;
+        file_manager.remove(path);
+    }
+
+    async fn insert_file_entry_locked(&self, parent_file_id: u64, file_id: 
u64, path: &str) {
+        let mut file_manager = self.file_entry_manager.write().await;
+        file_manager.insert(parent_file_id, file_id, path);
+    }
+}
+
+#[async_trait]
+impl<T: PathFileSystem> RawFileSystem for DefaultRawFileSystem<T> {
+    async fn init(&self) -> Result<()> {
+        // init root directory
+        self.insert_file_entry_locked(
+            Self::ROOT_DIR_PARENT_FILE_ID,
+            Self::ROOT_DIR_FILE_ID,
+            Self::ROOT_DIR_NAME,
+        )
+        .await;
+        self.fs.init().await
+    }
+
+    async fn get_file_path(&self, file_id: u64) -> Result<String> {
+        let file_entry = self.get_file_entry(file_id).await?;
+        Ok(file_entry.path)
+    }
+
+    async fn valid_file_handle_id(&self, file_id: u64, fh: u64) -> Result<()> {
+        let fh_file_id = self
+            .opened_file_manager
+            .get(fh)
+            .ok_or(Errno::from(libc::EBADF))?
+            .lock()
+            .await
+            .file_stat
+            .file_id;
+
+        (file_id == fh_file_id)
+            .then_some(())
+            .ok_or(Errno::from(libc::EBADF))
+    }
+
+    async fn stat(&self, file_id: u64) -> Result<FileStat> {
+        let file_entry = self.get_file_entry(file_id).await?;
+        let mut file_stat = self.fs.stat(&file_entry.path).await?;
+        file_stat.set_file_id(file_entry.parent_file_id, file_entry.file_id);
+        Ok(file_stat)
+    }
+
+    async fn lookup(&self, parent_file_id: u64, name: &str) -> 
Result<FileStat> {
+        let parent_file_entry = self.get_file_entry(parent_file_id).await?;
+        let mut file_stat = self.fs.lookup(&parent_file_entry.path, 
name).await?;
+        // fill the file id to file stat
+        self.resolve_file_id_to_filestat(&mut file_stat, parent_file_id)
+            .await;
+        Ok(file_stat)
+    }
+
+    async fn read_dir(&self, file_id: u64) -> Result<Vec<FileStat>> {
+        let file_entry = self.get_file_entry(file_id).await?;
+        let mut child_filestats = self.fs.read_dir(&file_entry.path).await?;
+        for file_stat in child_filestats.iter_mut() {
+            self.resolve_file_id_to_filestat(file_stat, file_stat.file_id)
+                .await;
+        }
+        Ok(child_filestats)
+    }
+
+    async fn open_file(&self, file_id: u64, flags: u32) -> Result<FileHandle> {
+        self.open_file_internal(file_id, flags, FileType::RegularFile)
+            .await
+    }
+
+    async fn open_dir(&self, file_id: u64, flags: u32) -> Result<FileHandle> {
+        self.open_file_internal(file_id, flags, FileType::Directory)
+            .await
+    }
+
+    async fn create_file(&self, parent_file_id: u64, name: &str, flags: u32) 
-> Result<FileHandle> {
+        let parent_file_entry = self.get_file_entry(parent_file_id).await?;
+        let mut file_without_id = self
+            .fs
+            .create_file(&parent_file_entry.path, name, OpenFileFlags(flags))
+            .await?;
+
+        file_without_id.set_file_id(parent_file_id, self.next_file_id());
+
+        // insert the new file to file entry manager
+        self.insert_file_entry_locked(
+            parent_file_id,
+            file_without_id.file_stat.file_id,
+            &file_without_id.file_stat.path,
+        )
+        .await;
+
+        // put the openfile to the opened file manager and allocate a file 
handle id
+        let file_with_id = self.opened_file_manager.put(file_without_id);
+        let opened_file_with_file_handle_id = file_with_id.lock().await;
+        Ok(opened_file_with_file_handle_id.file_handle())
+    }
+
+    async fn create_dir(&self, parent_file_id: u64, name: &str) -> Result<u64> 
{
+        let parent_file_entry = self.get_file_entry(parent_file_id).await?;
+        let mut filestat = self.fs.create_dir(&parent_file_entry.path, 
name).await?;
+
+        filestat.set_file_id(parent_file_id, self.next_file_id());
+
+        // insert the new file to file entry manager
+        self.insert_file_entry_locked(parent_file_id, filestat.file_id, 
&filestat.path)
+            .await;
+        Ok(filestat.file_id)
+    }
+
+    async fn set_attr(&self, file_id: u64, file_stat: &FileStat) -> Result<()> 
{
+        let file_entry = self.get_file_entry(file_id).await?;
+        self.fs.set_attr(&file_entry.path, file_stat, true).await
+    }
+
+    async fn remove_file(&self, parent_file_id: u64, name: &str) -> Result<()> 
{
+        let parent_file_entry = self.get_file_entry(parent_file_id).await?;
+        self.fs.remove_file(&parent_file_entry.path, name).await?;
+
+        // remove the file from file entry manager
+        self.remove_file_entry_locked(&join_file_path(&parent_file_entry.path, 
name))
+            .await;
+        Ok(())
+    }
+
+    async fn remove_dir(&self, parent_file_id: u64, name: &str) -> Result<()> {
+        let parent_file_entry = self.get_file_entry(parent_file_id).await?;
+        self.fs.remove_dir(&parent_file_entry.path, name).await?;
+
+        // remove the dir from file entry manager
+        self.remove_file_entry_locked(&join_file_path(&parent_file_entry.path, 
name))
+            .await;
+        Ok(())
+    }
+
+    async fn close_file(&self, _file_id: u64, fh: u64) -> Result<()> {
+        let opened_file = self
+            .opened_file_manager
+            .remove(fh)
+            .ok_or(Errno::from(libc::EBADF))?;
+        let mut file = opened_file.lock().await;
+        file.close().await
+    }
+
+    async fn read(
+        &self,
+        _file_id: u64,
+        fh: u64,
+        offset: u64,
+        size: u32,
+    ) -> crate::filesystem::Result<Bytes> {
+        let (data, file_stat) = {
+            let opened_file = self
+                .opened_file_manager
+                .get(fh)
+                .ok_or(Errno::from(libc::EBADF))?;
+            let mut opened_file = opened_file.lock().await;
+            let data = opened_file.read(offset, size).await;
+            (data, opened_file.file_stat.clone())
+        };
+
+        // update the file atime
+        self.fs.set_attr(&file_stat.path, &file_stat, false).await?;
+
+        data
+    }
+
+    async fn write(
+        &self,
+        _file_id: u64,
+        fh: u64,
+        offset: u64,
+        data: &[u8],
+    ) -> crate::filesystem::Result<u32> {
+        let (len, file_stat) = {
+            let opened_file = self
+                .opened_file_manager
+                .get(fh)
+                .ok_or(Errno::from(libc::EBADF))?;
+            let mut opened_file = opened_file.lock().await;
+            let len = opened_file.write(offset, data).await;
+            (len, opened_file.file_stat.clone())
+        };
+
+        // update the file size, mtime and atime
+        self.fs.set_attr(&file_stat.path, &file_stat, false).await?;
+
+        len
+    }
+}
+
+/// File entry is represent the abstract file.
+#[derive(Debug, Clone)]
+struct FileEntry {
+    file_id: u64,
+    parent_file_id: u64,
+    path: String,
+}
+
+/// FileEntryManager is manage all the file entries in memory. it is used 
manger the file relationship and name mapping.
+struct FileEntryManager {
+    // file_id_map is a map of file_id to file entry.
+    file_id_map: HashMap<u64, FileEntry>,
+
+    // file_path_map is a map of file path to file entry.
+    file_path_map: HashMap<String, FileEntry>,
+}
+
+impl FileEntryManager {
+    fn new() -> Self {
+        Self {
+            file_id_map: HashMap::new(),
+            file_path_map: HashMap::new(),
+        }
+    }
+
+    fn get_file_entry_by_id(&self, file_id: u64) -> Option<FileEntry> {
+        self.file_id_map.get(&file_id).cloned()
+    }
+
+    fn get_file_entry_by_path(&self, path: &str) -> Option<FileEntry> {
+        self.file_path_map.get(path).cloned()
+    }
+
+    fn insert(&mut self, parent_file_id: u64, file_id: u64, path: &str) {
+        let file_entry = FileEntry {
+            file_id,
+            parent_file_id,
+            path: path.to_string(),
+        };
+        self.file_id_map.insert(file_id, file_entry.clone());
+        self.file_path_map.insert(path.to_string(), file_entry);
+    }
+
+    fn remove(&mut self, path: &str) {
+        if let Some(file) = self.file_path_map.remove(path) {
+            self.file_id_map.remove(&file.file_id);
+        }
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_file_entry_manager() {
+        let mut manager = FileEntryManager::new();
+        manager.insert(1, 2, "a/b");
+        let file = manager.get_file_entry_by_id(2).unwrap();
+        assert_eq!(file.file_id, 2);
+        assert_eq!(file.parent_file_id, 1);
+        assert_eq!(file.path, "a/b");
+
+        let file = manager.get_file_entry_by_path("a/b").unwrap();
+        assert_eq!(file.file_id, 2);
+        assert_eq!(file.parent_file_id, 1);
+        assert_eq!(file.path, "a/b");
+
+        manager.remove("a/b");
+        assert!(manager.get_file_entry_by_id(2).is_none());
+        assert!(manager.get_file_entry_by_path("a/b").is_none());
+    }
+}
diff --git a/clients/filesystem-fuse/src/filesystem.rs 
b/clients/filesystem-fuse/src/filesystem.rs
index 6d1d8fa25..b0d32ded2 100644
--- a/clients/filesystem-fuse/src/filesystem.rs
+++ b/clients/filesystem-fuse/src/filesystem.rs
@@ -16,9 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+use crate::opened_file::{FileHandle, OpenFileFlags, OpenedFile};
+use crate::utils::{join_file_path, split_file_path};
 use async_trait::async_trait;
 use bytes::Bytes;
 use fuse3::{Errno, FileType, Timestamp};
+use std::time::SystemTime;
 
 pub(crate) type Result<T> = std::result::Result<T, Errno>;
 
@@ -35,15 +38,15 @@ pub(crate) trait RawFileSystem: Send + Sync {
     async fn init(&self) -> Result<()>;
 
     /// Get the file path by file id, if the file id is valid, return the file 
path
-    async fn get_file_path(&self, file_id: u64) -> String;
+    async fn get_file_path(&self, file_id: u64) -> Result<String>;
 
     /// Validate the file id and file handle, if file id and file handle is 
valid and it associated, return Ok
-    async fn valid_file_id(&self, file_id: u64, fh: u64) -> Result<()>;
+    async fn valid_file_handle_id(&self, file_id: u64, fh: u64) -> Result<()>;
 
     /// Get the file stat by file id. if the file id is valid, return the file 
stat
     async fn stat(&self, file_id: u64) -> Result<FileStat>;
 
-    /// Lookup the file by parent file id and file name, if the file is exist, 
return the file stat
+    /// Lookup the file by parent file id and file name, if the file exists, 
return the file stat
     async fn lookup(&self, parent_file_id: u64, name: &str) -> 
Result<FileStat>;
 
     /// Read the directory by file id, if the file id is a valid directory, 
return the file stat list
@@ -87,22 +90,22 @@ pub(crate) trait PathFileSystem: Send + Sync {
     /// Init the file system
     async fn init(&self) -> Result<()>;
 
-    /// Get the file stat by file path, if the file is exist, return the file 
stat
-    async fn stat(&self, name: &str) -> Result<FileStat>;
+    /// Get the file stat by file path, if the file exists, return the file 
stat
+    async fn stat(&self, path: &str) -> Result<FileStat>;
 
-    /// Get the file stat by parent file path and file name, if the file is 
exist, return the file stat
+    /// Get the file stat by parent file path and file name, if the file 
exists, return the file stat
     async fn lookup(&self, parent: &str, name: &str) -> Result<FileStat>;
 
-    /// Read the directory by file path, if the file is a valid directory, 
return the file stat list
-    async fn read_dir(&self, name: &str) -> Result<Vec<FileStat>>;
+    /// Read the directory by file path, if the directory exists, return the 
file stat list
+    async fn read_dir(&self, path: &str) -> Result<Vec<FileStat>>;
 
-    /// Open the file by file path and flags, if the file is exist, return the 
opened file
-    async fn open_file(&self, name: &str, flags: OpenFileFlags) -> 
Result<OpenedFile>;
+    /// Open the file by file path and flags, if the file exists, return the 
opened file
+    async fn open_file(&self, path: &str, flags: OpenFileFlags) -> 
Result<OpenedFile>;
 
-    /// Open the directory by file path and flags, if the file is exist, 
return the opened file
-    async fn open_dir(&self, name: &str, flags: OpenFileFlags) -> 
Result<OpenedFile>;
+    /// Open the directory by file path and flags, if the file exists, return 
the opened file
+    async fn open_dir(&self, path: &str, flags: OpenFileFlags) -> 
Result<OpenedFile>;
 
-    /// Create the file by parent file path and file name and flags, if 
successful, return the opened file
+    /// Create the file by parent file path and file name and flags, if 
successful return the opened file
     async fn create_file(
         &self,
         parent: &str,
@@ -114,7 +117,7 @@ pub(crate) trait PathFileSystem: Send + Sync {
     async fn create_dir(&self, parent: &str, name: &str) -> Result<FileStat>;
 
     /// Set the file attribute by file path and file stat
-    async fn set_attr(&self, name: &str, file_stat: &FileStat, flush: bool) -> 
Result<()>;
+    async fn set_attr(&self, path: &str, file_stat: &FileStat, flush: bool) -> 
Result<()>;
 
     /// Remove the file by parent file path and file name
     async fn remove_file(&self, parent: &str, name: &str) -> Result<()>;
@@ -174,9 +177,6 @@ pub struct FileStat {
     // file type like regular file or directory and so on
     pub(crate) kind: FileType,
 
-    // file permission
-    pub(crate) perm: u16,
-
     // file access time
     pub(crate) atime: Timestamp,
 
@@ -190,27 +190,48 @@ pub struct FileStat {
     pub(crate) nlink: u32,
 }
 
-/// Opened file for read or write, it is used to read or write the file 
content.
-pub(crate) struct OpenedFile {
-    pub(crate) file_stat: FileStat,
+impl FileStat {
+    pub fn new_file_filestat_with_path(path: &str, size: u64) -> Self {
+        let (parent, name) = split_file_path(path);
+        Self::new_file_filestat(parent, name, size)
+    }
 
-    pub(crate) handle_id: u64,
+    pub fn new_dir_filestat_with_path(path: &str) -> Self {
+        let (parent, name) = split_file_path(path);
+        Self::new_dir_filestat(parent, name)
+    }
 
-    pub reader: Option<Box<dyn FileReader>>,
+    pub fn new_file_filestat(parent: &str, name: &str, size: u64) -> Self {
+        Self::new_filestat(parent, name, size, FileType::RegularFile)
+    }
 
-    pub writer: Option<Box<dyn FileWriter>>,
-}
+    pub fn new_dir_filestat(parent: &str, name: &str) -> Self {
+        Self::new_filestat(parent, name, 0, FileType::Directory)
+    }
 
-// FileHandle is the file handle for the opened file.
-pub(crate) struct FileHandle {
-    pub(crate) file_id: u64,
+    pub fn new_filestat(parent: &str, name: &str, size: u64, kind: FileType) 
-> Self {
+        let atime = Timestamp::from(SystemTime::now());
+        Self {
+            file_id: 0,
+            parent_file_id: 0,
+            name: name.into(),
+            path: join_file_path(parent, name),
+            size: size,
+            kind: kind,
+            atime: atime,
+            mtime: atime,
+            ctime: atime,
+            nlink: 1,
+        }
+    }
 
-    pub(crate) handle_id: u64,
+    pub(crate) fn set_file_id(&mut self, parent_file_id: u64, file_id: u64) {
+        debug_assert!(file_id != 0 && parent_file_id != 0);
+        self.parent_file_id = parent_file_id;
+        self.file_id = file_id;
+    }
 }
 
-// OpenFileFlags is the open file flags for the file system.
-pub struct OpenFileFlags(u32);
-
 /// File reader interface  for read file content
 #[async_trait]
 pub(crate) trait FileReader: Sync + Send {
@@ -239,3 +260,54 @@ pub trait FileWriter: Sync + Send {
         Ok(())
     }
 }
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_create_file_stat() {
+        //test new file
+        let file_stat = FileStat::new_file_filestat("a", "b", 10);
+        assert_eq!(file_stat.name, "b");
+        assert_eq!(file_stat.path, "a/b");
+        assert_eq!(file_stat.size, 10);
+        assert_eq!(file_stat.kind, FileType::RegularFile);
+
+        //test new dir
+        let file_stat = FileStat::new_dir_filestat("a", "b");
+        assert_eq!(file_stat.name, "b");
+        assert_eq!(file_stat.path, "a/b");
+        assert_eq!(file_stat.size, 0);
+        assert_eq!(file_stat.kind, FileType::Directory);
+
+        //test new file with path
+        let file_stat = FileStat::new_file_filestat_with_path("a/b", 10);
+        assert_eq!(file_stat.name, "b");
+        assert_eq!(file_stat.path, "a/b");
+        assert_eq!(file_stat.size, 10);
+        assert_eq!(file_stat.kind, FileType::RegularFile);
+
+        //test new dir with path
+        let file_stat = FileStat::new_dir_filestat_with_path("a/b");
+        assert_eq!(file_stat.name, "b");
+        assert_eq!(file_stat.path, "a/b");
+        assert_eq!(file_stat.size, 0);
+        assert_eq!(file_stat.kind, FileType::Directory);
+    }
+
+    #[test]
+    fn test_file_stat_set_file_id() {
+        let mut file_stat = FileStat::new_file_filestat("a", "b", 10);
+        file_stat.set_file_id(1, 2);
+        assert_eq!(file_stat.file_id, 2);
+        assert_eq!(file_stat.parent_file_id, 1);
+    }
+
+    #[test]
+    #[should_panic(expected = "assertion failed: file_id != 0 && 
parent_file_id != 0")]
+    fn test_file_stat_set_file_id_panic() {
+        let mut file_stat = FileStat::new_file_filestat("a", "b", 10);
+        file_stat.set_file_id(1, 0);
+    }
+}
diff --git a/clients/filesystem-fuse/src/fuse_api_handle.rs 
b/clients/filesystem-fuse/src/fuse_api_handle.rs
index 8c065df02..7dc5461ce 100644
--- a/clients/filesystem-fuse/src/fuse_api_handle.rs
+++ b/clients/filesystem-fuse/src/fuse_api_handle.rs
@@ -52,10 +52,6 @@ impl<T: RawFileSystem> FuseApiHandle<T> {
         }
     }
 
-    pub async fn get_file_path(&self, file_id: u64) -> String {
-        self.fs.get_file_path(file_id).await
-    }
-
     async fn get_modified_file_stat(
         &self,
         file_id: u64,
@@ -117,7 +113,7 @@ impl<T: RawFileSystem> Filesystem for FuseApiHandle<T> {
     ) -> fuse3::Result<ReplyAttr> {
         // check the fh is associated with the file_id
         if let Some(fh) = fh {
-            self.fs.valid_file_id(inode, fh).await?;
+            self.fs.valid_file_handle_id(inode, fh).await?;
         }
 
         let file_stat = self.fs.stat(inode).await?;
@@ -136,7 +132,7 @@ impl<T: RawFileSystem> Filesystem for FuseApiHandle<T> {
     ) -> fuse3::Result<ReplyAttr> {
         // check the fh is associated with the file_id
         if let Some(fh) = fh {
-            self.fs.valid_file_id(inode, fh).await?;
+            self.fs.valid_file_handle_id(inode, fh).await?;
         }
 
         let new_file_stat = self
@@ -401,6 +397,10 @@ impl<T: RawFileSystem> Filesystem for FuseApiHandle<T> {
 
 const fn fstat_to_file_attr(file_st: &FileStat, context: &FileSystemContext) 
-> FileAttr {
     debug_assert!(file_st.file_id != 0 && file_st.parent_file_id != 0);
+    let perm = match file_st.kind {
+        Directory => context.default_dir_perm,
+        _ => context.default_file_perm,
+    };
     FileAttr {
         ino: file_st.file_id,
         size: file_st.size,
@@ -409,7 +409,7 @@ const fn fstat_to_file_attr(file_st: &FileStat, context: 
&FileSystemContext) ->
         mtime: file_st.mtime,
         ctime: file_st.ctime,
         kind: file_st.kind,
-        perm: file_st.perm,
+        perm: perm,
         nlink: file_st.nlink,
         uid: context.uid,
         gid: context.gid,
@@ -469,7 +469,6 @@ mod test {
             path: "".to_string(),
             size: 10032,
             kind: FileType::RegularFile,
-            perm: 0,
             atime: Timestamp { sec: 10, nsec: 3 },
             mtime: Timestamp { sec: 12, nsec: 5 },
             ctime: Timestamp { sec: 15, nsec: 7 },
@@ -493,7 +492,7 @@ mod test {
         assert_eq!(file_attr.mtime, Timestamp { sec: 12, nsec: 5 });
         assert_eq!(file_attr.ctime, Timestamp { sec: 15, nsec: 7 });
         assert_eq!(file_attr.kind, FileType::RegularFile);
-        assert_eq!(file_attr.perm, 0);
+        assert_eq!(file_attr.perm, context.default_file_perm);
         assert_eq!(file_attr.nlink, 0);
         assert_eq!(file_attr.uid, 1);
         assert_eq!(file_attr.gid, 2);
diff --git a/clients/filesystem-fuse/src/lib.rs 
b/clients/filesystem-fuse/src/lib.rs
index 54fb59a51..c1689bac4 100644
--- a/clients/filesystem-fuse/src/lib.rs
+++ b/clients/filesystem-fuse/src/lib.rs
@@ -16,5 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+mod default_raw_filesystem;
 mod filesystem;
 mod fuse_api_handle;
+mod opened_file;
+mod opened_file_manager;
+mod utils;
diff --git a/clients/filesystem-fuse/src/main.rs 
b/clients/filesystem-fuse/src/main.rs
index f6a7e69ec..3d8e9dbb9 100644
--- a/clients/filesystem-fuse/src/main.rs
+++ b/clients/filesystem-fuse/src/main.rs
@@ -16,8 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+mod default_raw_filesystem;
 mod filesystem;
 mod fuse_api_handle;
+mod opened_file;
+mod opened_file_manager;
+mod utils;
 
 use log::debug;
 use log::info;
@@ -30,3 +34,51 @@ async fn main() {
     debug!("Shutdown filesystem...");
     exit(0);
 }
+
+async fn create_gvfs_fuse_filesystem() {
+    // Gvfs-fuse filesystem structure:
+    // FuseApiHandle
+    // ├─ DefaultRawFileSystem (RawFileSystem)
+    // │ └─ FileSystemLog (PathFileSystem)
+    // │    ├─ GravitinoComposedFileSystem (PathFileSystem)
+    // │    │  ├─ GravitinoFilesetFileSystem (PathFileSystem)
+    // │    │  │  └─ S3FileSystem (PathFileSystem)
+    // │    │  │     └─ OpenDALFileSystem (PathFileSystem)
+    // │    │  ├─ GravitinoFilesetFileSystem (PathFileSystem)
+    // │    │  │  └─ HDFSFileSystem (PathFileSystem)
+    // │    │  │     └─ OpenDALFileSystem (PathFileSystem)
+    // │    │  ├─ GravitinoFilesetFileSystem (PathFileSystem)
+    // │    │  │  └─ JuiceFileSystem (PathFileSystem)
+    // │    │  │     └─ NasFileSystem (PathFileSystem)
+    // │    │  ├─ GravitinoFilesetFileSystem (PathFileSystem)
+    // │    │  │  └─ XXXFileSystem (PathFileSystem)
+    //
+    // `SimpleFileSystem` is a low-level filesystem designed to communicate 
with FUSE APIs.
+    // It manages file and directory relationships, as well as file mappings.
+    // It delegates file operations to the PathFileSystem
+    //
+    // `FileSystemLog` is a decorator that adds extra debug logging 
functionality to file system APIs.
+    // Similar implementations include permissions, caching, and metrics.
+    //
+    // `GravitinoComposeFileSystem` is a composite file system that can 
combine multiple `GravitinoFilesetFileSystem`.
+    // It use the part of catalog and schema of fileset path to a find actual 
GravitinoFilesetFileSystem. delegate the operation to the real storage.
+    // If the user only mounts a fileset, this layer is not present. There 
will only be one below layer.
+    //
+    // `GravitinoFilesetFileSystem` is a file system that can access a 
fileset.It translates the fileset path to the real storage path.
+    // and delegate the operation to the real storage.
+    //
+    // `OpenDALFileSystem` is a file system that use the OpenDAL to access 
real storage.
+    // it can assess the S3, HDFS, gcs, azblob and other storage.
+    //
+    // `S3FileSystem` is a file system that use `OpenDALFileSystem` to access 
S3 storage.
+    //
+    // `HDFSFileSystem` is a file system that use `OpenDALFileSystem` to 
access HDFS storage.
+    //
+    // `NasFileSystem` is a filesystem that uses a locally accessible path 
mounted by NAS tools, such as JuiceFS.
+    //
+    // `JuiceFileSystem` is a file that use `NasFileSystem` to access JuiceFS 
storage.
+    //
+    // `XXXFileSystem is a filesystem that allows you to implement file access 
through your own extensions.
+
+    todo!("Implement the createGvfsFuseFileSystem function");
+}
diff --git a/clients/filesystem-fuse/src/opened_file.rs 
b/clients/filesystem-fuse/src/opened_file.rs
new file mode 100644
index 000000000..ba3e41595
--- /dev/null
+++ b/clients/filesystem-fuse/src/opened_file.rs
@@ -0,0 +1,141 @@
+/*
+ * 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.
+ */
+use crate::filesystem::{FileReader, FileStat, FileWriter, Result};
+use bytes::Bytes;
+use fuse3::{Errno, Timestamp};
+use std::time::SystemTime;
+
+/// Opened file for read or write, it is used to read or write the file 
content.
+pub(crate) struct OpenedFile {
+    pub(crate) file_stat: FileStat,
+
+    pub(crate) handle_id: u64,
+
+    pub reader: Option<Box<dyn FileReader>>,
+
+    pub writer: Option<Box<dyn FileWriter>>,
+}
+
+impl OpenedFile {
+    pub(crate) fn new(file_stat: FileStat) -> Self {
+        OpenedFile {
+            file_stat: file_stat,
+            handle_id: 0,
+            reader: None,
+            writer: None,
+        }
+    }
+
+    pub(crate) async fn read(&mut self, offset: u64, size: u32) -> 
Result<Bytes> {
+        let reader = self.reader.as_mut().ok_or(Errno::from(libc::EBADF))?;
+        let result = reader.read(offset, size).await?;
+
+        // update the atime
+        self.file_stat.atime = Timestamp::from(SystemTime::now());
+
+        Ok(result)
+    }
+
+    pub(crate) async fn write(&mut self, offset: u64, data: &[u8]) -> 
Result<u32> {
+        let writer = self.writer.as_mut().ok_or(Errno::from(libc::EBADF))?;
+        let written = writer.write(offset, data).await?;
+
+        // update the file size ,mtime and atime
+        let end = offset + written as u64;
+        if end > self.file_stat.size {
+            self.file_stat.size = end;
+        }
+        self.file_stat.atime = Timestamp::from(SystemTime::now());
+        self.file_stat.mtime = self.file_stat.atime;
+
+        Ok(written)
+    }
+
+    pub(crate) async fn close(&mut self) -> Result<()> {
+        let mut errors = Vec::new();
+        if let Some(mut reader) = self.reader.take() {
+            if let Err(e) = reader.close().await {
+                errors.push(e);
+            }
+        }
+
+        if let Some(mut writer) = self.writer.take() {
+            if let Err(e) = self.flush().await {
+                errors.push(e);
+            }
+            if let Err(e) = writer.close().await {
+                errors.push(e);
+            }
+        }
+
+        if !errors.is_empty() {
+            return Err(errors.remove(0));
+        }
+        Ok(())
+    }
+
+    pub(crate) async fn flush(&mut self) -> Result<()> {
+        if let Some(writer) = &mut self.writer {
+            writer.flush().await?;
+        }
+        Ok(())
+    }
+
+    pub(crate) fn file_handle(&self) -> FileHandle {
+        debug_assert!(self.handle_id != 0);
+        debug_assert!(self.file_stat.file_id != 0);
+        FileHandle {
+            file_id: self.file_stat.file_id,
+            handle_id: self.handle_id,
+        }
+    }
+
+    pub(crate) fn set_file_id(&mut self, parent_file_id: u64, file_id: u64) {
+        debug_assert!(file_id != 0 && parent_file_id != 0);
+        self.file_stat.set_file_id(parent_file_id, file_id)
+    }
+}
+
+// FileHandle is the file handle for the opened file.
+pub(crate) struct FileHandle {
+    pub(crate) file_id: u64,
+
+    pub(crate) handle_id: u64,
+}
+
+// OpenFileFlags is the open file flags for the file system.
+pub(crate) struct OpenFileFlags(pub(crate) u32);
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::filesystem::FileStat;
+
+    #[test]
+    fn test_open_file() {
+        let mut open_file = OpenedFile::new(FileStat::new_file_filestat("a", 
"b", 10));
+        assert_eq!(open_file.file_stat.name, "b");
+        assert_eq!(open_file.file_stat.size, 10);
+
+        open_file.set_file_id(1, 2);
+
+        assert_eq!(open_file.file_stat.file_id, 2);
+        assert_eq!(open_file.file_stat.parent_file_id, 1);
+    }
+}
diff --git a/clients/filesystem-fuse/src/opened_file_manager.rs 
b/clients/filesystem-fuse/src/opened_file_manager.rs
new file mode 100644
index 000000000..17bfe00a3
--- /dev/null
+++ b/clients/filesystem-fuse/src/opened_file_manager.rs
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ */
+use crate::opened_file::OpenedFile;
+use dashmap::DashMap;
+use std::sync::atomic::AtomicU64;
+use std::sync::Arc;
+use tokio::sync::Mutex;
+
+// OpenedFileManager is a manager all the opened files. and allocate a file 
handle id for the opened file.
+pub(crate) struct OpenedFileManager {
+    // file_handle_map is a map of file_handle_id to opened file.
+    file_handle_map: DashMap<u64, Arc<Mutex<OpenedFile>>>,
+
+    // file_handle_id_generator is used to generate unique file handle IDs.
+    handle_id_generator: AtomicU64,
+}
+
+impl OpenedFileManager {
+    pub fn new() -> Self {
+        Self {
+            file_handle_map: Default::default(),
+            handle_id_generator: AtomicU64::new(1),
+        }
+    }
+
+    pub(crate) fn next_handle_id(&self) -> u64 {
+        self.handle_id_generator
+            .fetch_add(1, std::sync::atomic::Ordering::SeqCst)
+    }
+
+    pub(crate) fn put(&self, mut file: OpenedFile) -> Arc<Mutex<OpenedFile>> {
+        // Put the file into the file handle map, and allocate a file handle 
id for the file.
+        let file_handle_id = self.next_handle_id();
+        file.handle_id = file_handle_id;
+        let file_handle = Arc::new(Mutex::new(file));
+        self.file_handle_map
+            .insert(file_handle_id, file_handle.clone());
+        file_handle
+    }
+
+    pub(crate) fn get(&self, handle_id: u64) -> Option<Arc<Mutex<OpenedFile>>> 
{
+        self.file_handle_map
+            .get(&handle_id)
+            .map(|x| x.value().clone())
+    }
+
+    pub(crate) fn remove(&self, handle_id: u64) -> 
Option<Arc<Mutex<OpenedFile>>> {
+        self.file_handle_map.remove(&handle_id).map(|x| x.1)
+    }
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::filesystem::FileStat;
+
+    #[tokio::test]
+    async fn test_opened_file_manager() {
+        let manager = OpenedFileManager::new();
+
+        let file1_stat = FileStat::new_file_filestat("", "a.txt", 13);
+        let file2_stat = FileStat::new_file_filestat("", "b.txt", 18);
+
+        let file1 = OpenedFile::new(file1_stat.clone());
+        let file2 = OpenedFile::new(file2_stat.clone());
+
+        let handle_id1 = manager.put(file1).lock().await.handle_id;
+        let handle_id2 = manager.put(file2).lock().await.handle_id;
+
+        // Test the file handle id is assigned.
+        assert!(handle_id1 > 0 && handle_id2 > 0);
+        assert_ne!(handle_id1, handle_id2);
+
+        // test get file by handle id
+        assert_eq!(
+            manager.get(handle_id1).unwrap().lock().await.file_stat.name,
+            file1_stat.name
+        );
+
+        assert_eq!(
+            manager.get(handle_id2).unwrap().lock().await.file_stat.name,
+            file2_stat.name
+        );
+
+        // test remove file by handle id
+        assert_eq!(
+            manager.remove(handle_id1).unwrap().lock().await.handle_id,
+            handle_id1
+        );
+
+        // test get file by handle id after remove
+        assert!(manager.get(handle_id1).is_none());
+        assert!(manager.get(handle_id2).is_some());
+    }
+}
diff --git a/clients/filesystem-fuse/src/utils.rs 
b/clients/filesystem-fuse/src/utils.rs
new file mode 100644
index 000000000..0c0cc80a1
--- /dev/null
+++ b/clients/filesystem-fuse/src/utils.rs
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+use crate::filesystem::RawFileSystem;
+
+// join the parent and name to a path
+pub fn join_file_path(parent: &str, name: &str) -> String {
+    //TODO handle corner cases
+    if parent.is_empty() {
+        name.to_string()
+    } else {
+        format!("{}/{}", parent, name)
+    }
+}
+
+// split the path to parent and name
+pub fn split_file_path(path: &str) -> (&str, &str) {
+    match path.rfind('/') {
+        Some(pos) => (&path[..pos], &path[pos + 1..]),
+        None => ("", path),
+    }
+}
+
+// convert file id to file path string if file id is invalid return "Unknown"
+pub async fn file_id_to_file_path_string(file_id: u64, fs: &impl 
RawFileSystem) -> String {
+    fs.get_file_path(file_id)
+        .await
+        .unwrap_or("Unknown".to_string())
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+
+    #[test]
+    fn test_join_file_path() {
+        assert_eq!(join_file_path("", "a"), "a");
+        assert_eq!(join_file_path("", "a.txt"), "a.txt");
+        assert_eq!(join_file_path("a", "b"), "a/b");
+        assert_eq!(join_file_path("a/b", "c"), "a/b/c");
+        assert_eq!(join_file_path("a/b", "c.txt"), "a/b/c.txt");
+    }
+
+    #[test]
+    fn test_split_file_path() {
+        assert_eq!(split_file_path("a"), ("", "a"));
+        assert_eq!(split_file_path("a.txt"), ("", "a.txt"));
+        assert_eq!(split_file_path("a/b"), ("a", "b"));
+        assert_eq!(split_file_path("a/b/c"), ("a/b", "c"));
+        assert_eq!(split_file_path("a/b/c.txt"), ("a/b", "c.txt"));
+    }
+}


Reply via email to