You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zh...@apache.org on 2019/11/11 12:25:51 UTC

[incubator-doris] branch master updated: Refactor and reorganize the file utils (#2089)

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

zhaoc pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new d0316d1  Refactor and reorganize the file utils  (#2089)
d0316d1 is described below

commit d0316d158dfa4cbfe3521bfdeb7d64b30176aaf9
Author: Seaven <se...@qq.com>
AuthorDate: Mon Nov 11 20:25:41 2019 +0800

    Refactor and reorganize the file utils  (#2089)
---
 be/src/common/status.h                             |   8 +
 be/src/env/env.h                                   |  16 +-
 be/src/env/env_posix.cpp                           |  32 ++-
 be/src/gutil/gscoped_ptr.h                         |  26 +--
 be/src/olap/data_dir.cpp                           |  72 +++---
 be/src/olap/rowset/beta_rowset.cpp                 |   5 +-
 be/src/olap/rowset/segment_group.cpp               |  58 +++--
 be/src/olap/snapshot_manager.cpp                   |  29 ++-
 be/src/olap/storage_engine.cpp                     |   8 +-
 be/src/olap/tablet_manager.cpp                     |  23 +-
 be/src/olap/task/engine_clone_task.cpp             |  40 ++--
 be/src/olap/task/engine_storage_migration_task.cpp |  18 +-
 be/src/olap/utils.cpp                              | 247 +--------------------
 be/src/olap/utils.h                                |  20 --
 be/src/util/file_utils.cpp                         | 138 +++++++++---
 be/src/util/file_utils.h                           |  34 ++-
 be/test/olap/delete_handler_test.cpp               |  29 +--
 be/test/olap/delta_writer_test.cpp                 |   7 +-
 be/test/olap/file_utils_test.cpp                   | 149 +++++++++++++
 be/test/olap/memtable_flush_executor_test.cpp      |   7 +-
 be/test/olap/olap_reader_test.cpp                  |  23 +-
 be/test/olap/olap_snapshot_converter_test.cpp      |   3 +-
 be/test/olap/rowset/alpha_rowset_test.cpp          |  20 +-
 be/test/olap/rowset/beta_rowset_test.cpp           |  15 +-
 be/test/olap/tablet_mgr_test.cpp                   |  13 +-
 be/test/olap/vectorized_olap_reader_test.cpp       |  15 +-
 be/test/runtime/data_spliter_test.cpp              |   3 +-
 27 files changed, 582 insertions(+), 476 deletions(-)

diff --git a/be/src/common/status.h b/be/src/common/status.h
index e8cc7d3..cf9926c 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -256,6 +256,14 @@ private:
         } \
     } while (0);
 
+#define RETURN_WITH_WARN_IF_ERROR(stmt, ret_code, warning_prefix) \
+    do {    \
+        const Status& _s = (stmt);  \
+        if (UNLIKELY(!_s.ok())) {   \
+            LOG(WARNING) << (warning_prefix) << ", error: " << _s.to_string(); \
+            return ret_code;    \
+        }   \
+    } while (0);
 }
 
 #define WARN_UNUSED_RESULT __attribute__((warn_unused_result))
diff --git a/be/src/env/env.h b/be/src/env/env.h
index 704c860..a4f402e 100644
--- a/be/src/env/env.h
+++ b/be/src/env/env.h
@@ -101,12 +101,12 @@ public:
                                       const std::string& fname,
                                       std::unique_ptr<RandomRWFile>* result) = 0;
 
-    // Returns OK if the named file exists.
+    // Returns OK if the path exists.
     //         NotFound if the named file does not exist,
     //                  the calling process does not have permission to determine
     //                  whether this file exists, or if the path is invalid.
     //         IOError if an IO Error was encountered
-    virtual Status file_exists(const std::string& fname) = 0;
+    virtual Status path_exists(const std::string& fname) = 0;
 
     // Store in *result the names of the children of the specified directory.
     // The names are relative to "dir".
@@ -140,6 +140,18 @@ public:
     // Create the specified directory. Returns error if directory exists.
     virtual Status create_dir(const std::string& dirname) = 0;
 
+    // Checks if the file is a directory. Returns an error if it doesn't
+    // exist, otherwise writes true or false into 'is_dir' appropriately.
+    virtual Status is_directory(const std::string& path, bool* is_dir) = 0;
+
+    // Canonicalize 'path' by applying the following conversions:
+    // - Converts a relative path into an absolute one using the cwd.
+    // - Converts '.' and '..' references.
+    // - Resolves all symbolic links.
+    //
+    // All directory entries in 'path' must exist on the filesystem.
+    virtual Status canonicalize(const std::string& path, std::string* result) = 0;
+
     // Creates directory if missing. Return Ok if it exists, or successful in
     // Creating.
     virtual Status create_dir_if_missing(const std::string& dirname) = 0;
diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp
index 4bb087b..954944f 100644
--- a/be/src/env/env_posix.cpp
+++ b/be/src/env/env_posix.cpp
@@ -16,10 +16,12 @@
 #include <sys/types.h>
 #include <unistd.h>
 #include <sys/uio.h>
+#include <memory>
 
 #include "common/logging.h"
 #include "gutil/macros.h"
 #include "gutil/port.h"
+#include "gutil/gscoped_ptr.h"
 #include "gutil/strings/substitute.h"
 #include "util/errno.h"
 #include "util/slice.h"
@@ -36,16 +38,16 @@ static Status io_error(const std::string& context, int err_number) {
     case ENAMETOOLONG:
     case ENOENT:
     case ENOTDIR:
-        return Status::NotFound(context, 1, errno_to_string(err_number));
+        return Status::NotFound(context, err_number, errno_to_string(err_number));
     case EEXIST:
-        return Status::AlreadyExist(context, 1, errno_to_string(err_number));
+        return Status::AlreadyExist(context, err_number, errno_to_string(err_number));
     case EOPNOTSUPP:
     case EXDEV: // No cross FS links allowed
-        return Status::NotSupported(context, 1, errno_to_string(err_number));
+        return Status::NotSupported(context, err_number, errno_to_string(err_number));
     case EIO:
         LOG(ERROR) << "I/O error, context=" << context;
     }
-    return Status::IOError(context, 1, errno_to_string(err_number));
+    return Status::IOError(context, err_number, errno_to_string(err_number));
 }
 
 Status do_sync(int fd, const string& filename) {
@@ -543,7 +545,7 @@ public:
         return Status::OK();
     }
 
-    Status file_exists(const std::string& fname) override {
+    Status path_exists(const std::string& fname) override {
         if (access(fname.c_str(), F_OK) != 0) {
             return io_error(fname, errno);
         }
@@ -597,6 +599,26 @@ public:
         return Status::OK();
     }
 
+    Status is_directory(const std::string& path, bool* is_dir) override {
+        struct stat path_stat;
+        if (stat(path.c_str(), &path_stat) != 0) {
+            return io_error(path, errno);
+        } else {
+            *is_dir = S_ISDIR(path_stat.st_mode);
+        }
+
+        return Status::OK();
+    }
+
+    Status canonicalize(const std::string& path, std::string* result) override {
+        std::unique_ptr<char[], FreeDeleter> r(realpath(path.c_str(), nullptr));
+        if (r == nullptr) {
+            return io_error(Substitute("Unable to canonicalize $0", path), errno);
+        }
+        *result = std::string(r.get());
+        return Status::OK();
+    }
+
     // Creates directory if missing. Return Ok if it exists, or successful in
     // Creating.
     Status create_dir_if_missing(const std::string& name) override {
diff --git a/be/src/gutil/gscoped_ptr.h b/be/src/gutil/gscoped_ptr.h
index be1bb48..21d8b1a 100644
--- a/be/src/gutil/gscoped_ptr.h
+++ b/be/src/gutil/gscoped_ptr.h
@@ -109,7 +109,7 @@
 #include "gutil/type_traits.h"
 #include "gutil/move.h"
 
-namespace kudu {
+namespace doris {
 
 namespace subtle {
 class RefCountedBase;
@@ -174,7 +174,7 @@ struct DefaultDeleter<T[n]> {
 // Function object which invokes 'free' on its parameter, which must be
 // a pointer. Can be used to store malloc-allocated pointers in gscoped_ptr:
 //
-// gscoped_ptr<int, kudu::FreeDeleter> foo_ptr(
+// gscoped_ptr<int, doris::FreeDeleter> foo_ptr(
 //     static_cast<int*>(malloc(sizeof(int))));
 struct FreeDeleter {
   inline void operator()(void* ptr) const {
@@ -186,8 +186,8 @@ namespace internal {
 
 template <typename T> struct IsNotRefCounted {
   enum {
-    value = !base::is_convertible<T*, kudu::subtle::RefCountedBase*>::value &&
-        !base::is_convertible<T*, kudu::subtle::RefCountedThreadSafeBase*>::
+    value = !base::is_convertible<T*, doris::subtle::RefCountedBase*>::value &&
+        !base::is_convertible<T*, doris::subtle::RefCountedThreadSafeBase*>::
             value
   };
 };
@@ -296,7 +296,7 @@ class gscoped_ptr_impl {
 
 }  // namespace internal
 
-}  // namespace kudu
+}  // namespace doris
 
 // A gscoped_ptr<T> is like a T*, except that the destructor of gscoped_ptr<T>
 // automatically deletes the pointer it holds (if any).
@@ -314,11 +314,11 @@ class gscoped_ptr_impl {
 // unique_ptr<> features. Known deficiencies include not supporting move-only
 // deleteres, function pointers as deleters, and deleters with reference
 // types.
-template <class T, class D = kudu::DefaultDeleter<T> >
+template <class T, class D = doris::DefaultDeleter<T> >
 class gscoped_ptr {
   MOVE_ONLY_TYPE_FOR_CPP_03(gscoped_ptr, RValue)
 
-  COMPILE_ASSERT(kudu::internal::IsNotRefCounted<T>::value,
+  COMPILE_ASSERT(doris::internal::IsNotRefCounted<T>::value,
                  T_is_refcounted_type_and_needs_scoped_refptr);
 
  public:
@@ -393,7 +393,7 @@ class gscoped_ptr {
   // Allow gscoped_ptr<element_type> to be used in boolean expressions, but not
   // implicitly convertible to a real bool (which is dangerous).
  private:
-  typedef kudu::internal::gscoped_ptr_impl<element_type, deleter_type>
+  typedef doris::internal::gscoped_ptr_impl<element_type, deleter_type>
       gscoped_ptr::*Testable;
 
  public:
@@ -433,7 +433,7 @@ class gscoped_ptr {
  private:
   // Needed to reach into |impl_| in the constructor.
   template <typename U, typename V> friend class gscoped_ptr;
-  kudu::internal::gscoped_ptr_impl<element_type, deleter_type> impl_;
+  doris::internal::gscoped_ptr_impl<element_type, deleter_type> impl_;
 
   // Forbid comparison of gscoped_ptr types.  If U != T, it totally
   // doesn't make sense, and if U == T, it still doesn't make sense
@@ -500,7 +500,7 @@ class gscoped_ptr<T[], D> {
   // Allow gscoped_ptr<element_type> to be used in boolean expressions, but not
   // implicitly convertible to a real bool (which is dangerous).
  private:
-  typedef kudu::internal::gscoped_ptr_impl<element_type, deleter_type>
+  typedef doris::internal::gscoped_ptr_impl<element_type, deleter_type>
       gscoped_ptr::*Testable;
 
  public:
@@ -531,7 +531,7 @@ class gscoped_ptr<T[], D> {
   enum { type_must_be_complete = sizeof(element_type) };
 
   // Actually hold the data.
-  kudu::internal::gscoped_ptr_impl<element_type, deleter_type> impl_;
+  doris::internal::gscoped_ptr_impl<element_type, deleter_type> impl_;
 
   // Disable initialization from any type other than element_type*, by
   // providing a constructor that matches such an initialization, but is
@@ -691,12 +691,12 @@ bool operator!=(C* p1, const gscoped_array<C>& p2) {
   return p1 != p2.get();
 }
 
-// DEPRECATED: Use gscoped_ptr<C, kudu::FreeDeleter> instead.
+// DEPRECATED: Use gscoped_ptr<C, doris::FreeDeleter> instead.
 //
 // gscoped_ptr_malloc<> is similar to gscoped_ptr<>, but it accepts a
 // second template argument, the functor used to free the object.
 
-template<class C, class FreeProc = kudu::FreeDeleter>
+template<class C, class FreeProc = doris::FreeDeleter>
 class gscoped_ptr_malloc {
   MOVE_ONLY_TYPE_FOR_CPP_03(gscoped_ptr_malloc, RValue)
 
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
old mode 100755
new mode 100644
index 39c5b54..630979d
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -93,7 +93,7 @@ Status DataDir::init() {
         LOG(WARNING) << "fail to allocate memory. size=" <<  TEST_FILE_BUF_SIZE;
         return Status::InternalError("No memory");
     }
-    if (!check_dir_existed(_path)) {
+    if (!FileUtils::check_exist(_path)) {
         LOG(WARNING) << "opendir failed, path=" << _path;
         return Status::InternalError("opendir failed");
     }
@@ -203,7 +203,7 @@ Status DataDir::_init_extension_and_capacity() {
     }
 
     std::string data_path = _path + DATA_PREFIX;
-    if (!check_dir_existed(data_path) && create_dir(data_path) != OLAP_SUCCESS) {
+    if (!FileUtils::check_exist(data_path) && !FileUtils::create_dir(data_path).ok()) {
         LOG(WARNING) << "failed to create data root path. path=" << data_path;
         return Status::InternalError("invalid store path: failed to create data directory");
     }
@@ -388,7 +388,6 @@ OLAPStatus DataDir::_read_and_write_test_file() {
 }
 
 OLAPStatus DataDir::get_shard(uint64_t* shard) {
-    OLAPStatus res = OLAP_SUCCESS;
     std::lock_guard<std::mutex> l(_mutex);
 
     std::stringstream shard_path_stream;
@@ -396,12 +395,9 @@ OLAPStatus DataDir::get_shard(uint64_t* shard) {
     _current_shard = (_current_shard + 1) % MAX_SHARD_NUM;
     shard_path_stream << _path << DATA_PREFIX << "/" << next_shard;
     std::string shard_path = shard_path_stream.str();
-    if (!check_dir_existed(shard_path)) {
-        res = create_dir(shard_path);
-        if (res != OLAP_SUCCESS) {
-            LOG(WARNING) << "fail to create path. [path='" << shard_path << "']";
-            return res;
-        }
+    if (!FileUtils::check_exist(shard_path)) {
+        RETURN_WITH_WARN_IF_ERROR(FileUtils::create_dir(shard_path), OLAP_ERR_CANNOT_CREATE_DIR,
+                                  "fail to create path. path=" + shard_path);
     }
 
     *shard = next_shard;
@@ -636,21 +632,21 @@ OLAPStatus DataDir::remove_old_meta_and_files() {
 
         // remove incremental dir and pending dir
         std::string pending_delta_path = data_path_prefix + PENDING_DELTA_PREFIX;
-        if (check_dir_existed(pending_delta_path)) {
+        if (FileUtils::check_exist(pending_delta_path)) {
             LOG(INFO) << "remove pending delta path:" << pending_delta_path;
-            if(remove_all_dir(pending_delta_path) != OLAP_SUCCESS) {
-                LOG(INFO) << "errors while remove pending delta path. tablet_path=" << data_path_prefix;
-                return true;
-            }
+
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove_all(pending_delta_path), true,
+                                      "errors while remove pending delta path. tablet_path=" +
+                                      data_path_prefix);
         }
 
         std::string incremental_delta_path = data_path_prefix + INCREMENTAL_DELTA_PREFIX;
-        if (check_dir_existed(incremental_delta_path)) {
+        if (FileUtils::check_exist(incremental_delta_path)) {
             LOG(INFO) << "remove incremental delta path:" << incremental_delta_path;
-            if(remove_all_dir(incremental_delta_path) != OLAP_SUCCESS) {
-                LOG(INFO) << "errors while remove incremental delta path. tablet_path=" << data_path_prefix;
-                return true;
-            }
+
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove_all(incremental_delta_path), true,
+                                      "errors while remove incremental delta path. tablet_path=" +
+                                      data_path_prefix);
         }
 
         TabletMetaManager::remove(this, tablet_id, schema_hash, OLD_HEADER_PREFIX);
@@ -931,31 +927,44 @@ void DataDir::perform_path_scan() {
         LOG(INFO) << "start to scan data dir path:" << _path;
         std::set<std::string> shards;
         std::string data_path = _path + DATA_PREFIX;
-        if (dir_walk(data_path, &shards, nullptr) != OLAP_SUCCESS) {
-            LOG(WARNING) << "fail to walk dir. [path=" << data_path << "]";
-            return;
+
+        Status ret = FileUtils::list_dirs_files(data_path, &shards, nullptr, Env::Default());
+        if (!ret.ok()) {
+            LOG(WARNING) << "fail to walk dir. path=[" + data_path 
+                          << "] error[" << ret.to_string() << "]";
+            return ;
         }
+        
         for (const auto& shard : shards) {
             std::string shard_path = data_path + "/" + shard;
             std::set<std::string> tablet_ids;
-            if (dir_walk(shard_path, &tablet_ids, nullptr) != OLAP_SUCCESS) {
-                LOG(WARNING) << "fail to walk dir. [path=" << shard_path << "]";
+            ret = FileUtils::list_dirs_files(shard_path, &tablet_ids, nullptr, Env::Default());
+            if (!ret.ok()) {
+                LOG(WARNING) << "fail to walk dir. [path=" << shard_path 
+                             << "] error[" << ret.to_string() << "]";
                 continue;
             }
             for (const auto& tablet_id : tablet_ids) {
                 std::string tablet_id_path = shard_path + "/" + tablet_id;
                 _all_check_paths.insert(tablet_id_path);
                 std::set<std::string> schema_hashes;
-                if (dir_walk(tablet_id_path, &schema_hashes, nullptr) != OLAP_SUCCESS) {
-                    LOG(WARNING) << "fail to walk dir. [path=" << tablet_id_path << "]";
+                ret = FileUtils::list_dirs_files(tablet_id_path, &schema_hashes, nullptr,
+                                           Env::Default());
+                if (!ret.ok()) {
+                    LOG(WARNING) << "fail to walk dir. [path=" << tablet_id_path << "]"
+                                 << " error[" << ret.to_string() << "]";
                     continue;
                 }
                 for (const auto& schema_hash : schema_hashes) {
                     std::string tablet_schema_hash_path = tablet_id_path + "/" + schema_hash;
                     _all_check_paths.insert(tablet_schema_hash_path);
                     std::set<std::string> rowset_files;
-                    if (dir_walk(tablet_schema_hash_path, nullptr, &rowset_files) != OLAP_SUCCESS) {
-                        LOG(WARNING) << "fail to walk dir. [path=" << tablet_schema_hash_path << "]";
+                    
+                    ret = FileUtils::list_dirs_files(tablet_schema_hash_path, nullptr, &rowset_files,
+                                                     Env::Default()); 
+                    if (!ret.ok()) {
+                        LOG(WARNING) << "fail to walk dir. [path=" << tablet_schema_hash_path 
+                                     << "] error[" << ret.to_string() << "]";
                         continue;
                     }
                     for (const auto& rowset_file : rowset_files) {
@@ -971,12 +980,9 @@ void DataDir::perform_path_scan() {
 }
 
 void DataDir::_process_garbage_path(const std::string& path) {
-    if (check_dir_existed(path)) {
+    if (FileUtils::check_exist(path)) {
         LOG(INFO) << "collect garbage dir path: " << path;
-        OLAPStatus status = remove_all_dir(path);
-        if (status != OLAP_SUCCESS) {
-            LOG(WARNING) << "remove garbage dir path: " << path << " failed";
-        }
+        WARN_IF_ERROR(FileUtils::remove_all(path), "remove garbage dir failed. path: " + path);
     }
 }
 
diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp
index 9d52490..2e0b7d5 100644
--- a/be/src/olap/rowset/beta_rowset.cpp
+++ b/be/src/olap/rowset/beta_rowset.cpp
@@ -20,6 +20,7 @@
 #include <set>
 #include <stdio.h>  // for remove()
 #include <unistd.h> // for link()
+#include <util/file_utils.h>
 #include "gutil/strings/substitute.h"
 #include "olap/rowset/beta_rowset_reader.h"
 #include "olap/utils.h"
@@ -96,7 +97,7 @@ OLAPStatus BetaRowset::remove() {
 OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset_id) {
     for (int i = 0; i < num_segments(); ++i) {
         std::string dst_link_path = segment_file_path(dir, new_rowset_id, i);
-        if (check_dir_existed(dst_link_path)) {
+        if (FileUtils::check_exist(dst_link_path)) {
             LOG(WARNING) << "failed to create hard link, file already exist: " << dst_link_path;
             return OLAP_ERR_FILE_ALREADY_EXIST;
         }
@@ -113,7 +114,7 @@ OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset
 OLAPStatus BetaRowset::copy_files_to(const std::string& dir) {
     for (int i = 0; i < num_segments(); ++i) {
         std::string dst_path = segment_file_path(dir, rowset_id(), i);
-        if (check_dir_existed(dst_path)) {
+        if (FileUtils::check_exist(dst_path)) {
             LOG(WARNING) << "file already exist: " << dst_path;
             return OLAP_ERR_FILE_ALREADY_EXIST;
         }
diff --git a/be/src/olap/rowset/segment_group.cpp b/be/src/olap/rowset/segment_group.cpp
index 764ae4d..41fcab5 100644
--- a/be/src/olap/rowset/segment_group.cpp
+++ b/be/src/olap/rowset/segment_group.cpp
@@ -33,6 +33,7 @@
 #include "olap/utils.h"
 #include "olap/wrapper_field.h"
 #include "util/stack_util.h"
+#include "util/file_utils.h"
 
 using std::ifstream;
 using std::string;
@@ -722,7 +723,7 @@ OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) {
     }
     for (int segment_id = 0; segment_id < _num_segments; segment_id++) {
         std::string dest_data_file = construct_data_file_path(dir, segment_id);
-        if (check_dir_existed(dest_data_file)) {
+        if (FileUtils::check_exist(dest_data_file)) {
             LOG(WARNING) << "file already exists:" << dest_data_file;
             return OLAP_ERR_FILE_ALREADY_EXIST;
         }
@@ -734,7 +735,7 @@ OLAPStatus SegmentGroup::copy_files_to(const std::string& dir) {
             return OLAP_ERR_OS_ERROR;
         }
         std::string dest_index_file = construct_index_file_path(dir, segment_id);
-        if (check_dir_existed(dest_index_file)) {
+        if (FileUtils::check_exist(dest_index_file)) {
             LOG(WARNING) << "file already exists:" << dest_index_file;
             return OLAP_ERR_FILE_ALREADY_EXIST;
         }
@@ -762,9 +763,10 @@ OLAPStatus SegmentGroup::convert_from_old_files(const std::string& snapshot_path
     for (int segment_id = 0; segment_id < _num_segments; segment_id++) {
         std::string new_data_file_name = construct_data_file_path(_rowset_path_prefix, segment_id);
         // if file exist should remove it because same file name does not mean same data
-        if (check_dir_existed(new_data_file_name)) {
+        if (FileUtils::check_exist(new_data_file_name)) {
             LOG(INFO) << "file already exist, remove it. file=" << new_data_file_name;
-            RETURN_NOT_OK(remove_dir(new_data_file_name));
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(new_data_file_name), OLAP_ERR_CANNOT_CREATE_DIR, 
+                    "remove path failed. path=" + new_data_file_name);
         }
         std::string old_data_file_name = construct_old_data_file_path(snapshot_path, segment_id);
         if (link(old_data_file_name.c_str(), new_data_file_name.c_str()) != 0) {
@@ -777,9 +779,12 @@ OLAPStatus SegmentGroup::convert_from_old_files(const std::string& snapshot_path
         }
         success_links->push_back(new_data_file_name);
         std::string new_index_file_name = construct_index_file_path(_rowset_path_prefix, segment_id);
-        if (check_dir_existed(new_index_file_name)) {
+        if (FileUtils::check_exist(new_index_file_name)) {
             LOG(INFO) << "file already exist, remove it. file=" << new_index_file_name;
-            RETURN_NOT_OK(remove_dir(new_index_file_name));
+            
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(new_index_file_name), 
+                    OLAP_ERR_CANNOT_CREATE_DIR,
+                    "remove path failed. path=" + new_index_file_name);
         }
         std::string old_index_file_name = construct_old_index_file_path(snapshot_path, segment_id);
         if (link(old_index_file_name.c_str(), new_index_file_name.c_str()) != 0) {
@@ -804,7 +809,7 @@ OLAPStatus SegmentGroup::convert_to_old_files(const std::string& snapshot_path,
     for (int segment_id = 0; segment_id < _num_segments; segment_id++) {
         std::string new_data_file_name = construct_data_file_path(_rowset_path_prefix, segment_id);
         std::string old_data_file_name = construct_old_data_file_path(snapshot_path, segment_id);
-        if (!check_dir_existed(old_data_file_name)) {
+        if (!FileUtils::check_exist(old_data_file_name)) {
             if (link(new_data_file_name.c_str(), old_data_file_name.c_str()) != 0) {
                 LOG(WARNING) << "fail to create hard link. from=" << new_data_file_name << ", "
                     << "to=" << old_data_file_name << ", " << "errno=" << Errno::no();
@@ -816,7 +821,7 @@ OLAPStatus SegmentGroup::convert_to_old_files(const std::string& snapshot_path,
                 << "to=" << old_data_file_name;
         std::string new_index_file_name = construct_index_file_path(_rowset_path_prefix, segment_id);
         std::string old_index_file_name = construct_old_index_file_path(snapshot_path, segment_id);
-        if (!check_dir_existed(old_index_file_name)) {
+        if (!FileUtils::check_exist(old_index_file_name)) {
             if (link(new_index_file_name.c_str(), old_index_file_name.c_str()) != 0) {
                 LOG(WARNING) << "fail to create hard link. from=" << new_index_file_name << ", "
                     << "to=" << old_index_file_name << ", " << "errno=" << Errno::no();
@@ -833,33 +838,46 @@ OLAPStatus SegmentGroup::convert_to_old_files(const std::string& snapshot_path,
 OLAPStatus SegmentGroup::remove_old_files(std::vector<std::string>* links_to_remove) {
     for (int segment_id = 0; segment_id < _num_segments; segment_id++) {
         std::string old_data_file_name = construct_old_data_file_path(_rowset_path_prefix, segment_id);
-        if (check_dir_existed(old_data_file_name)) {
-            RETURN_NOT_OK(remove_dir(old_data_file_name));
+        if (FileUtils::check_exist(old_data_file_name)) {
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(old_data_file_name), 
+                    OLAP_ERR_CANNOT_CREATE_DIR, 
+                    "remove path failed. path" + old_data_file_name);
+            
             links_to_remove->push_back(old_data_file_name);
         }
         std::string old_index_file_name = construct_old_index_file_path(_rowset_path_prefix, segment_id);
-        if (check_dir_existed(old_index_file_name)) {
-            RETURN_NOT_OK(remove_dir(old_index_file_name));
+        if (FileUtils::check_exist(old_index_file_name)) {
+            RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(old_index_file_name),
+                                      OLAP_ERR_CANNOT_CREATE_DIR,
+                                      "remove path failed. path" + old_index_file_name);
+
             links_to_remove->push_back(old_index_file_name);
         }
         // if segment group id == 0, it maybe convert from old files which do not have segment group id in file path
         if (_segment_group_id == 0) {
             old_data_file_name = _construct_err_sg_data_file_path(_rowset_path_prefix, segment_id);
-            if (check_dir_existed(old_data_file_name)) {
-                RETURN_NOT_OK(remove_dir(old_data_file_name));
+            if (FileUtils::check_exist(old_data_file_name)) {
+                RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(old_data_file_name),
+                                          OLAP_ERR_CANNOT_CREATE_DIR,
+                                          "remove path failed. path" + old_data_file_name);
                 links_to_remove->push_back(old_data_file_name);
             }
             old_index_file_name = _construct_err_sg_index_file_path(_rowset_path_prefix, segment_id);
-            if (check_dir_existed(old_index_file_name)) {
-                RETURN_NOT_OK(remove_dir(old_index_file_name));
+            if (FileUtils::check_exist(old_index_file_name)) {
+                RETURN_WITH_WARN_IF_ERROR(FileUtils::remove(old_index_file_name),
+                                          OLAP_ERR_CANNOT_CREATE_DIR,
+                                          "remove path failed. path" + old_index_file_name);
+
                 links_to_remove->push_back(old_index_file_name);
             }
         }
     }
     std::string pending_delta_path = _rowset_path_prefix + PENDING_DELTA_PREFIX;
-    if (check_dir_existed(pending_delta_path)) {
+    if (FileUtils::check_exist(pending_delta_path)) {
         LOG(INFO) << "remove pending delta path:" << pending_delta_path;
-        RETURN_NOT_OK(remove_all_dir(pending_delta_path));
+        RETURN_WITH_WARN_IF_ERROR(FileUtils::remove_all(pending_delta_path),
+                                  OLAP_ERR_CANNOT_CREATE_DIR,
+                                  "remove path failed. path" + pending_delta_path);
     }
     return OLAP_SUCCESS;
 }
@@ -872,7 +890,7 @@ OLAPStatus SegmentGroup::link_segments_to_path(const std::string& dest_path, con
     for (int segment_id = 0; segment_id < _num_segments; segment_id++) {
         std::string data_file_name = _construct_file_name(rowset_id, segment_id, ".dat");
         std::string new_data_file_path = dest_path + "/" + data_file_name;
-        if (!check_dir_existed(new_data_file_path)) {
+        if (!FileUtils::check_exist(new_data_file_path)) {
             std::string origin_data_file_path = construct_data_file_path(_rowset_path_prefix, segment_id);
             if (link(origin_data_file_path.c_str(), new_data_file_path.c_str()) != 0) {
                 LOG(WARNING) << "fail to create hard link. from=" << origin_data_file_path
@@ -882,7 +900,7 @@ OLAPStatus SegmentGroup::link_segments_to_path(const std::string& dest_path, con
         }
         std::string index_file_name = _construct_file_name(rowset_id, segment_id, ".idx");
         std::string new_index_file_path = dest_path + "/" + index_file_name;
-        if (!check_dir_existed(new_index_file_path)) {
+        if (!FileUtils::check_exist(new_index_file_path)) {
             std::string origin_idx_file_path = construct_index_file_path(_rowset_path_prefix, segment_id);
             if (link(origin_idx_file_path.c_str(), new_index_file_path.c_str()) != 0) {
                 LOG(WARNING) << "fail to create hard link. from=" << origin_idx_file_path
diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp
old mode 100755
new mode 100644
index 9e11658..7e141dd
--- a/be/src/olap/snapshot_manager.cpp
+++ b/be/src/olap/snapshot_manager.cpp
@@ -36,6 +36,8 @@
 #include "olap/rowset/rowset_id_generator.h"
 #include "olap/rowset/rowset_writer.h"
 
+#include "env/env.h"
+
 using boost::filesystem::canonical;
 using boost::filesystem::copy_file;
 using boost::filesystem::copy_option;
@@ -107,7 +109,7 @@ OLAPStatus SnapshotManager::release_snapshot(const string& snapshot_path) {
         if (snapshot_path.compare(0, abs_path.size(), abs_path) == 0
                 && snapshot_path.compare(abs_path.size(),
                         SNAPSHOT_PREFIX.size(), SNAPSHOT_PREFIX) == 0) {
-            remove_all_dir(snapshot_path);
+            FileUtils::remove_all(snapshot_path);
             LOG(INFO) << "success to release snapshot path. [path='" << snapshot_path << "']";
 
             return OLAP_SUCCESS;
@@ -123,7 +125,7 @@ OLAPStatus SnapshotManager::convert_rowset_ids(const string& clone_dir, int64_t
     const int32_t& schema_hash, TabletSharedPtr tablet) {
     OLAPStatus res = OLAP_SUCCESS;   
     // check clone dir existed
-    if (!check_dir_existed(clone_dir)) {
+    if (!FileUtils::check_exist(clone_dir)) {
         res = OLAP_ERR_DIR_NOT_EXIST;
         LOG(WARNING) << "clone dir not existed when convert rowsetids. clone_dir=" 
                      << clone_dir;
@@ -329,11 +331,11 @@ OLAPStatus SnapshotManager::_create_snapshot_files(
     string schema_full_path = get_schema_hash_full_path(
             ref_tablet, snapshot_id_path);
     string header_path = _get_header_full_path(ref_tablet, schema_full_path);
-    if (check_dir_existed(schema_full_path)) {
+    if (FileUtils::check_exist(schema_full_path)) {
         VLOG(10) << "remove the old schema_full_path.";
-        remove_all_dir(schema_full_path);
+        FileUtils::remove_all(schema_full_path);
     }
-    create_dirs(schema_full_path);
+    FileUtils::create_dir(schema_full_path);
     path boost_path(snapshot_id_path);
     string snapshot_id = canonical(boost_path).string();
     do {
@@ -446,9 +448,12 @@ OLAPStatus SnapshotManager::_create_snapshot_files(
         }
         if (snapshot_version < PREFERRED_SNAPSHOT_VERSION) {
             set<string> exist_old_files;
-            if ((res = dir_walk(schema_full_path, nullptr, &exist_old_files)) != OLAP_SUCCESS) {
+            Status ret = FileUtils::list_dirs_files(schema_full_path, nullptr, &exist_old_files,
+                                       Env::Default());
+            if (!ret.ok()) { 
+                res = OLAP_ERR_DISK_FAILURE;
                 LOG(WARNING) << "failed to dir walk when convert old files. dir=" 
-                             << schema_full_path;
+                             << schema_full_path << ", error:" << ret.to_string();
                 break;
             }
             OlapSnapshotConverter converter;
@@ -466,8 +471,10 @@ OLAPStatus SnapshotManager::_create_snapshot_files(
                 files_to_delete.push_back(full_file_path);
             }
             // remove all files
-            res = remove_files(files_to_delete);
-            if (res != OLAP_SUCCESS) {
+            ret = FileUtils::remove_paths(files_to_delete);
+            if (!ret.ok()) {
+                res = OLAP_ERR_IO_ERROR;
+                LOG(WARNING) << "remove paths failed. error: " << ret.to_string(); 
                 break;
             }
             // save new header to snapshot header path
@@ -508,9 +515,9 @@ OLAPStatus SnapshotManager::_create_snapshot_files(
         LOG(WARNING) << "fail to make snapshot, try to delete the snapshot path. path="
                      << snapshot_id_path.c_str();
 
-        if (check_dir_existed(snapshot_id_path)) {
+        if (FileUtils::check_exist(snapshot_id_path)) {
             VLOG(3) << "remove snapshot path. [path=" << snapshot_id_path << "]";
-            remove_all_dir(snapshot_id_path);
+            FileUtils::remove_all(snapshot_id_path);
         }
     } else {
         *snapshot_path = snapshot_id;
diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp
index bfb7ac9..99cb877 100644
--- a/be/src/olap/storage_engine.cpp
+++ b/be/src/olap/storage_engine.cpp
@@ -690,7 +690,7 @@ void StorageEngine::_clean_unused_txns() {
 OLAPStatus StorageEngine::_do_sweep(
         const string& scan_root, const time_t& local_now, const int32_t expire) {
     OLAPStatus res = OLAP_SUCCESS;
-    if (!check_dir_existed(scan_root)) {
+    if (!FileUtils::check_exist(scan_root)) {
         // dir not existed. no need to sweep trash.
         return res;
     }
@@ -720,8 +720,10 @@ OLAPStatus StorageEngine::_do_sweep(
             VLOG(10) << "get actual expire time " << actual_expire << " of dir: " << dir_name;
 
             if (difftime(local_now, mktime(&local_tm_create)) >= actual_expire) {
-                if (remove_all_dir(path_name) != OLAP_SUCCESS) {
-                    LOG(WARNING) << "fail to remove file or directory. path=" << path_name;
+                Status ret = FileUtils::remove_all(path_name);
+                if (!ret.ok()) {
+                    LOG(WARNING) << "fail to remove file or directory. path=" << path_name
+                                 << ", error=" << ret.to_string();
                     res = OLAP_ERR_OS_ERROR;
                     continue;
                 }
diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp
old mode 100755
new mode 100644
index 156ca5b..f47f7c3
--- a/be/src/olap/tablet_manager.cpp
+++ b/be/src/olap/tablet_manager.cpp
@@ -52,6 +52,7 @@
 #include "util/time.h"
 #include "util/doris_metrics.h"
 #include "util/pretty_printer.h"
+#include "util/file_utils.h"
 
 using apache::thrift::ThriftDebugString;
 using boost::filesystem::canonical;
@@ -432,14 +433,16 @@ TabletSharedPtr TabletManager::_create_tablet_meta_and_dir(
         std::string tablet_dir = tablet_path.string();
         // because the tablet is removed async, so that the dir may still exist
         // when be receive create tablet again. For example redo schema change
-        if (check_dir_existed(schema_hash_dir)) {
+        if (FileUtils::check_exist(schema_hash_dir)) {
             LOG(WARNING) << "skip this dir because tablet path exist, path="<< schema_hash_dir;
             continue;
         } else {
             data_dir->add_pending_ids(TABLET_ID_PREFIX + std::to_string(request.tablet_id));
-            res = create_dirs(schema_hash_dir);
-            if (res != OLAP_SUCCESS) {
-                LOG(WARNING) << "create dir fail. [res=" << res << " path:" << schema_hash_dir;
+            Status ret = FileUtils::create_dir(schema_hash_dir);
+            if(!ret.ok()) {
+                LOG(WARNING) << "create dir fail. [res=" << res << " path:" << schema_hash_dir
+                             << " error: " << ret.to_string(); 
+                res = OLAP_ERR_CANNOT_CREATE_DIR;
                 continue;
             }
         }
@@ -447,9 +450,11 @@ TabletSharedPtr TabletManager::_create_tablet_meta_and_dir(
         tablet = Tablet::create_tablet_from_meta(tablet_meta, data_dir);
         if (tablet == nullptr) {
             LOG(WARNING) << "fail to load tablet from tablet_meta. root_path:" << data_dir->path();
-            res = remove_all_dir(tablet_dir);
-            if (res != OLAP_SUCCESS) {
-                LOG(WARNING) << "remove tablet dir:" << tablet_dir;
+            Status ret = FileUtils::remove_all(tablet_dir);
+            if (!ret.ok()) {
+                LOG(WARNING) << "remove tablet dir:" << tablet_dir 
+                             << ", error: " << ret.to_string();
+                res = OLAP_ERR_IO_ERROR;
             }
             continue;
         }
@@ -1020,7 +1025,7 @@ OLAPStatus TabletManager::start_trash_sweep() {
                     it = _shutdown_tablets.erase(it);
                     continue;
                 }
-                if (check_dir_existed((*it)->tablet_path())) {
+                if (FileUtils::check_exist((*it)->tablet_path())) {
                     // take snapshot of tablet meta
                     std::string meta_file = (*it)->tablet_path() + "/" + std::to_string((*it)->tablet_id()) + ".hdr";
                     (*it)->tablet_meta()->save(meta_file);
@@ -1043,7 +1048,7 @@ OLAPStatus TabletManager::start_trash_sweep() {
                 ++ clean_num;
             } else {
                 // if could not find tablet info in meta store, then check if dir existed
-                if (check_dir_existed((*it)->tablet_path())) {
+                if (FileUtils::check_exist((*it)->tablet_path())) {
                     LOG(WARNING) << "errors while load meta from store, skip this tablet" 
                                 << " tablet id " << (*it)->tablet_id()
                                 << " schema hash " << (*it)->schema_hash();
diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp
index 64a8682..773d705 100644
--- a/be/src/olap/task/engine_clone_task.cpp
+++ b/be/src/olap/task/engine_clone_task.cpp
@@ -25,6 +25,8 @@
 #include "olap/rowset/rowset.h"
 #include "olap/rowset/rowset_factory.h"
 
+#include "env/env.h"
+
 using std::set;
 using std::stringstream;
 
@@ -194,7 +196,7 @@ OLAPStatus EngineCloneTask::execute() {
             }
             // clone success, delete .hdr file because tablet meta is stored in rocksdb
             string cloned_meta_file = tablet_dir_stream.str() + "/" + std::to_string(_clone_req.tablet_id) + ".hdr";
-            remove_dir(cloned_meta_file);
+            FileUtils::remove(cloned_meta_file);
         }
         // Clean useless dir, if failed, ignore it.
         if (status != DORIS_SUCCESS && status != DORIS_CREATE_TABLE_EXIST) {
@@ -557,7 +559,7 @@ AgentStatus EngineCloneTask::_clone_copy(
 OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, int64_t tablet_id) {
     OLAPStatus res = OLAP_SUCCESS;
     // check clone dir existed
-    if (!check_dir_existed(clone_dir)) {
+    if (!FileUtils::check_exist(clone_dir)) {
         res = OLAP_ERR_DIR_NOT_EXIST;
         LOG(WARNING) << "clone dir not existed when clone. clone_dir=" << clone_dir.c_str();
         return res;
@@ -580,10 +582,11 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in
     }
 
     set<string> clone_files;
-    if ((res = dir_walk(clone_dir, NULL, &clone_files)) != OLAP_SUCCESS) {
-        LOG(WARNING) << "failed to dir walk when clone. [clone_dir=" << clone_dir << "]";
-        return res;
-    }
+    
+    RETURN_WITH_WARN_IF_ERROR(
+            FileUtils::list_dirs_files(clone_dir, NULL, &clone_files, Env::Default()),
+            OLAP_ERR_DISK_FAILURE,
+            "failed to dir walk when clone. clone_dir=" + clone_dir);
 
     try {
        olap_header_msg.CopyFrom(file_header.message());
@@ -606,8 +609,9 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in
         files_to_delete.push_back(full_file_path);
     }
     // remove all files
-    RETURN_NOT_OK(remove_files(files_to_delete));
-
+    RETURN_WITH_WARN_IF_ERROR(FileUtils::remove_paths(files_to_delete), OLAP_ERR_IO_ERROR,
+            "remove paths failed.")
+    
     res = TabletMeta::save(cloned_meta_file, tablet_meta_pb);
     if (res != OLAP_SUCCESS) {
         LOG(WARNING) << "fail to save converted tablet meta to dir='" << clone_dir;
@@ -631,7 +635,7 @@ OLAPStatus EngineCloneTask::_finish_clone(TabletSharedPtr tablet, const string&
     tablet->obtain_header_wrlock();
     do {
         // check clone dir existed
-        if (!check_dir_existed(clone_dir)) {
+        if (!FileUtils::check_exist(clone_dir)) {
             res = OLAP_ERR_DIR_NOT_EXIST;
             LOG(WARNING) << "clone dir not existed when clone. clone_dir=" << clone_dir.c_str();
             break;
@@ -646,20 +650,26 @@ OLAPStatus EngineCloneTask::_finish_clone(TabletSharedPtr tablet, const string&
             break;
         }
         // remove the cloned meta file
-        remove_dir(cloned_tablet_meta_file);
+        FileUtils::remove(cloned_tablet_meta_file);
 
         // TODO(ygl): convert old format file into rowset
         // check all files in /clone and /tablet
         set<string> clone_files;
-        if ((res = dir_walk(clone_dir, NULL, &clone_files)) != OLAP_SUCCESS) {
-            LOG(WARNING) << "failed to dir walk when clone. [clone_dir=" << clone_dir << "]";
+        Status ret = FileUtils::list_dirs_files(clone_dir, NULL, &clone_files, Env::Default());
+        if (!ret.ok()) {
+            LOG(WARNING) << "failed to dir walk when clone. [clone_dir=" << clone_dir << "]"
+                         << " error: " << ret.to_string();
+            res = OLAP_ERR_DISK_FAILURE;
             break;
         }
 
         set<string> local_files;
         string tablet_dir = tablet->tablet_path();
-        if ((res = dir_walk(tablet_dir, NULL, &local_files)) != OLAP_SUCCESS) {
-            LOG(WARNING) << "failed to dir walk when clone. [tablet_dir=" << tablet_dir << "]";
+        ret = FileUtils::list_dirs_files(tablet_dir, NULL, &local_files, Env::Default());
+        if (!ret.ok()) {
+            LOG(WARNING) << "failed to dir walk when clone. [tablet_dir=" << tablet_dir << "]"
+                         << " error: " << ret.to_string();
+            res = OLAP_ERR_DISK_FAILURE;
             break;
         }
 
@@ -704,7 +714,7 @@ OLAPStatus EngineCloneTask::_finish_clone(TabletSharedPtr tablet, const string&
 
     // clear linked files if errors happen
     if (res != OLAP_SUCCESS) {
-        remove_files(linked_success_files);
+        FileUtils::remove_paths(linked_success_files);
     }
     tablet->release_header_lock();
     tablet->release_push_lock();
diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp
index df14f1d..a48cd39 100644
--- a/be/src/olap/task/engine_storage_migration_task.cpp
+++ b/be/src/olap/task/engine_storage_migration_task.cpp
@@ -138,7 +138,7 @@ OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate(
         string schema_hash_path = SnapshotManager::instance()->get_schema_hash_full_path(tablet, root_path_stream.str());
         // if dir already exist then return err, it should not happen
         // should not remove the dir directly
-        if (check_dir_existed(schema_hash_path)) {
+        if (FileUtils::check_exist(schema_hash_path)) {
             LOG(INFO) << "schema hash path already exist, skip this path. "
                       << "schema_hash_path=" << schema_hash_path;
             res = OLAP_ERR_FILE_ALREADY_EXIST;
@@ -154,7 +154,15 @@ OLAPStatus EngineStorageMigrationTask::_storage_medium_migrate(
             res = OLAP_ERR_META_ALREADY_EXIST;
             break;
         }
-        create_dirs(schema_hash_path);
+        
+        Status st = FileUtils::create_dir(schema_hash_path);
+
+        if (!st.ok()) {
+            res = OLAP_ERR_CANNOT_CREATE_DIR;
+            LOG(WARNING) << "fail to create path. path=" << schema_hash_path << ", error:" << st.to_string();
+            break;
+        }
+
 
         // migrate all index and data files but header file
         res = _copy_index_and_data_files(schema_hash_path, tablet, consistent_rowsets);
@@ -262,9 +270,11 @@ OLAPStatus EngineStorageMigrationTask::_copy_index_and_data_files(
     for (auto& rs : consistent_rowsets) {
         status = rs->copy_files_to(schema_hash_path);
         if (status != OLAP_SUCCESS) {
-            if (remove_all_dir(schema_hash_path) != OLAP_SUCCESS) {
+            Status ret = FileUtils::remove_all(schema_hash_path);
+            if (!ret.ok()) {
                 LOG(FATAL) << "remove storage migration path failed. "
-                           << "schema_hash_path:" << schema_hash_path;
+                           << "schema_hash_path:" << schema_hash_path
+                           << " error: " << ret.to_string();
             }
             break;
         }
diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp
index 75d4e04..7b0784f 100644
--- a/be/src/olap/utils.cpp
+++ b/be/src/olap/utils.cpp
@@ -29,6 +29,7 @@
 #include <boost/regex.hpp>
 #include <errno.h>
 #include <lz4/lz4.h>
+#include "util/file_utils.h"
 
 #ifdef DORIS_WITH_LZO
 #include <lzo/lzo1c.h>
@@ -43,6 +44,7 @@
 #include "gutil/strings/substitute.h"
 #include "olap/olap_common.h"
 #include "olap/olap_define.h"
+#include "env/env.h"
 
 using std::string;
 using std::set;
@@ -996,7 +998,7 @@ OLAPStatus move_to_trash(const boost::filesystem::path& schema_hash_root,
     string new_file_dir = new_file_dir_stream.str();
     string new_file_path = new_file_dir + "/" + old_file_name;
     // create target dir, or the rename() function will fail.
-    if (!check_dir_existed(new_file_dir) && create_dirs(new_file_dir) != OLAP_SUCCESS) {
+    if (!FileUtils::check_exist(new_file_dir) && !FileUtils::create_dir(new_file_dir).ok()) {
         OLAP_LOG_WARNING("delete file failed. due to mkdir failed. [file=%s new_dir=%s]",
                 old_file_path.c_str(), new_file_dir.c_str());
         return OLAP_ERR_OS_ERROR;
@@ -1013,15 +1015,16 @@ OLAPStatus move_to_trash(const boost::filesystem::path& schema_hash_root,
     // 4. check parent dir of source file, delete it when empty
     string source_parent_dir = schema_hash_root.parent_path().string(); // tablet_id level
     std::set<std::string> sub_dirs, sub_files;
-    if (dir_walk(source_parent_dir, &sub_dirs, &sub_files) != OLAP_SUCCESS) {
-        LOG(INFO) << "access dir failed. [dir=" << source_parent_dir << "]";
-        // This error is nothing serious. so we still return success.
-        return OLAP_SUCCESS;
-    }
+
+    RETURN_WITH_WARN_IF_ERROR(
+            FileUtils::list_dirs_files(source_parent_dir, &sub_dirs, &sub_files, Env::Default()),
+            OLAP_SUCCESS,
+            "access dir failed. [dir=" + source_parent_dir);
+    
     if (sub_dirs.empty() && sub_files.empty()) {
         LOG(INFO) << "remove empty dir " << source_parent_dir;
         // no need to exam return status
-        remove_dir(source_parent_dir);
+        Env::Default()->delete_dir(source_parent_dir);
     }
 
     return OLAP_SUCCESS;
@@ -1218,63 +1221,6 @@ COPY_EXIT:
     return res;
 }
 
-bool check_dir_existed(const string& path) {
-    boost::filesystem::path p(path.c_str());
-
-    try {
-        if (boost::filesystem::exists(p)) {
-            return true;
-        } else {
-            return false;
-        }
-    } catch (...) {
-        // do nothing
-    }
-
-    LOG(WARNING) << "boost exception when check exist and return false. [path=" << path << "]";
-    
-    return false;
-}
-
-OLAPStatus create_dirs(const string& path) {
-    boost::filesystem::path p(path.c_str());
-
-    try {
-        if (boost::filesystem::create_directories(p)) {
-            VLOG(3) << "create dir success. [path='" << path << "']";
-            return OLAP_SUCCESS;
-        }
-    } catch (const boost::filesystem::filesystem_error& e) {
-        LOG(WARNING) << "error message: [err_msg='" << e.code().message() << "']";
-    } catch (std::exception& e) { 
-        LOG(WARNING) << "error message: [exception='" << e.what() << "']";
-    } catch (...) {
-        // do nothing
-        OLAP_LOG_WARNING("unknown exception.");
-    }
-
-    LOG(WARNING) << "fail to create dir. [path='" << path << "']";
-    
-    return OLAP_ERR_CANNOT_CREATE_DIR;
-}
-
-OLAPStatus create_dir(const string& path) {
-    boost::filesystem::path p(path.c_str());
-
-    try {
-        if (boost::filesystem::create_directory(p)) {
-            VLOG(3) << "create dir success. [path='" << path << "']";
-            return OLAP_SUCCESS;
-        }
-    } catch (...) {
-        // do nothing
-    }
-
-    LOG(WARNING) << "fail to create dir. [path='" << path << "']";
-    
-    return OLAP_ERR_CANNOT_CREATE_DIR;
-}
-
 OLAPStatus copy_dir(const string &src_dir, const string &dst_dir) {
     boost::filesystem::path src_path(src_dir.c_str());
     boost::filesystem::path dst_path(dst_dir.c_str());
@@ -1334,78 +1280,6 @@ OLAPStatus copy_dir(const string &src_dir, const string &dst_dir) {
     return OLAP_SUCCESS;
 }
 
-OLAPStatus remove_files(const vector<string>& files) {
-    OLAPStatus res = OLAP_SUCCESS;
-    for (const string& file : files) {
-        boost::filesystem::path file_path(file);
-
-        try {
-            if (boost::filesystem::remove(file_path)) {
-                VLOG(3) << "remove file. [file=" << file << "]";
-            } else {
-                OLAP_LOG_WARNING("failed to remove file. [file=%s errno=%d]",
-                                 file.c_str(), Errno::no());
-                res = OLAP_ERR_IO_ERROR;
-            }
-        } catch (...) {
-        // do nothing
-        }
-    }
-    return res;
-}
-
-// failed when there are files or dirs under thr dir
-OLAPStatus remove_dir(const string& path) {
-    boost::filesystem::path p(path.c_str());
-
-    try {
-        if (boost::filesystem::remove(p)) {
-            return OLAP_SUCCESS;
-        }
-    } catch (...) {
-        // do nothing
-    }
-
-    LOG(WARNING) << "fail to del dir. [path='" << path << "' errno=" << Errno::no() << "]";
-
-    return OLAP_ERR_CANNOT_CREATE_DIR;
-}
-
-OLAPStatus remove_parent_dir(const string& path) {
-    OLAPStatus res = OLAP_SUCCESS;
-
-    try {
-        boost::filesystem::path path_name(path);
-        boost::filesystem::path parent_path = path_name.parent_path();
-
-        if (boost::filesystem::exists(parent_path)) {
-            boost::filesystem::remove(parent_path);
-        }
-    } catch (...) {
-        LOG(WARNING) << "fail to del parent path. [chile path='" << path << "']";
-        res = OLAP_ERR_STL_ERROR;
-    }
-
-    return res;
-}
-
-// remove all files or dirs under the dir.
-OLAPStatus remove_all_dir(const string& path) {
-    boost::filesystem::path p(path.c_str());
-
-    try {
-        if (boost::filesystem::remove_all(p)) {
-            return OLAP_SUCCESS;
-        }
-    } catch (...) {
-        // do nothing
-    }
-
-    LOG(WARNING) << "fail to del all dir. [path='" << path << "' errno=" << Errno::no() << "]";
-
-    return OLAP_ERR_CANNOT_CREATE_DIR;
-}
-
 __thread char Errno::_buf[BUF_SIZE]; ///< buffer instance
 
 const char *Errno::str() {
@@ -1425,107 +1299,6 @@ int Errno::no() {
     return errno;
 }
 
-static Status disk_error(const std::string& context, int16_t err) {
-    switch (err) {
-        case ENOENT:
-            return Status::NotFound(context, err, errno_to_string(err));
-        case EEXIST:
-            return Status::AlreadyExist(context, err, errno_to_string(err));
-        case EOPNOTSUPP:
-            return Status::NotSupported(context, err, errno_to_string(err));
-        case EIO:
-        case ENODEV:
-        case ENXIO:
-        case EROFS:
-            return Status::IOError(context, err, errno_to_string(err));
-        default:
-            return Status::InternalError(context, err, errno_to_string(err));
-    }
-}
-
-OLAPStatus dir_walk(const string& root,
-                    set<string>* dirs,
-                    set<string>* files) {
-    DIR* dirp = NULL;
-    struct stat stat_data;
-    struct dirent* direntp = NULL;
-    dirp = opendir(root.c_str());
-    if (dirp == nullptr) {
-        Status status = disk_error("opendir failed", errno);
-        
-        LOG(WARNING) << status.to_string();
-        if (status.is_io_error()) {
-            return OLAP_ERR_DISK_FAILURE;
-        } else {
-            return OLAP_ERR_INIT_FAILED;
-        }
-    }
-    
-    while ((direntp = readdir(dirp)) != NULL) {
-        // 去掉. .. 和.开头的隐藏文件
-        if ('.' == direntp->d_name[0]) {
-            continue;
-        }
-        // 检查找到的目录项是文件还是目录
-        string tmp_ent = root + '/' + direntp->d_name;
-        if (lstat(tmp_ent.c_str(), &stat_data) < 0) {
-            LOG(WARNING) << "lstat error.";
-            continue;
-        }
-
-        if (S_ISDIR(stat_data.st_mode)) {
-            if (NULL != dirs) {
-                dirs->insert(direntp->d_name);
-            }
-        } else {
-            if (NULL != files) {
-                files->insert(direntp->d_name);
-            }
-        }
-    }
-    closedir(dirp);
-
-    return OLAP_SUCCESS;
-}
-
-OLAPStatus remove_unused_files(const string& schema_hash_root,
-                           const set<string>& files,
-                           const string& header,
-                           const set<string>& indices,
-                           const set<string>& datas) {
-    // 从列表中去掉使用的的index文件
-    set<string> tmp_set;
-    set_difference(files.begin(),
-                   files.end(),
-                   indices.begin(),
-                   indices.end(),
-                   inserter(tmp_set, tmp_set.end()));
-    
-    // 从列表中去掉使用的的data文件
-    set<string> different_set;
-    set_difference(tmp_set.begin(),
-                   tmp_set.end(),
-                   datas.begin(),
-                   datas.end(),
-                   inserter(different_set, different_set.end()));
-    
-    // 从列表中去掉使用的header文件
-    different_set.erase(header);
-    // 遍历所有没有使用的文件
-    for (set<string>::const_iterator it = different_set.begin(); it != different_set.end(); ++it) {
-        if (ENDSWITH(*it, ".hdr") || ENDSWITH(*it, ".idx") || ENDSWITH(*it, ".dat")) {
-            LOG(INFO) << "delete unused file. [file='" << schema_hash_root + "/" + *it << "']";
-            move_to_trash(boost::filesystem::path(schema_hash_root),
-                          boost::filesystem::path(schema_hash_root + "/" + *it));
-        } else {
-            // 除了.hdr, .idx, .dat其他文件均忽略
-            continue;
-        }
-    }
-
-    return OLAP_SUCCESS;
-}
-
 template <>
 bool valid_signed_number<int128_t>(const std::string& value_str) {
     char* endptr = NULL;
diff --git a/be/src/olap/utils.h b/be/src/olap/utils.h
index f24d3db..8c58ad4 100644
--- a/be/src/olap/utils.h
+++ b/be/src/olap/utils.h
@@ -386,21 +386,8 @@ unsigned int crc32c_lut(char const* b, unsigned int off, unsigned int len, unsig
 
 OLAPStatus copy_file(const std::string& src, const std::string& dest);
 
-bool check_dir_existed(const std::string& path);
-
-OLAPStatus create_dir(const std::string& path);
-OLAPStatus create_dirs(const std::string& path);
-
 OLAPStatus copy_dir(const std::string &src_dir, const std::string &dst_dir);
 
-OLAPStatus remove_files(const std::vector<std::string>& files);
-
-OLAPStatus remove_dir(const std::string& path);
-
-OLAPStatus remove_parent_dir(const std::string& path);
-
-OLAPStatus remove_all_dir(const std::string& path);
-
 //转换两个list
 template<typename T1, typename T2>
 void static_cast_assign_vector(std::vector<T1>* v1, const std::vector<T2>& v2) {
@@ -425,8 +412,6 @@ private:
     static __thread char _buf[BUF_SIZE];
 };
 
-OLAPStatus dir_walk(const std::string& root, std::set<std::string>* dirs, std::set<std::string>* files);
-
 inline bool is_io_error(OLAPStatus status) {
     return (((OLAP_ERR_IO_ERROR == status || OLAP_ERR_READ_UNENOUGH == status)&& errno == EIO)
                 || OLAP_ERR_CHECKSUM_ERROR == status
@@ -438,11 +423,6 @@ inline bool is_io_error(OLAPStatus status) {
 #define ENDSWITH(str, suffix)   \
     ((str).rfind(suffix) == (str).size() - strlen(suffix))
 
-OLAPStatus remove_unused_files(const std::string& schema_hash_root,
-                           const std::set<std::string>& files,
-                           const std::string& header,
-                           const std::set<std::string>& indices,
-                           const std::set<std::string>& datas);
 
 // 检查int8_t, int16_t, int32_t, int64_t的值是否溢出
 template <typename T>
diff --git a/be/src/util/file_utils.cpp b/be/src/util/file_utils.cpp
index 7caa90e..ce1247e 100644
--- a/be/src/util/file_utils.cpp
+++ b/be/src/util/file_utils.cpp
@@ -33,36 +33,59 @@
 
 #include <openssl/md5.h>
 
+#include "gutil/strings/substitute.h"
+
 #include "env/env.h"
 #include "olap/file_helper.h"
 #include "util/defer_op.h"
 
 namespace doris {
 
-Status FileUtils::create_dir(const std::string& dir_path) {
-    try {
-        if (boost::filesystem::exists(dir_path.c_str())) {
-            // No need to create one
-            if (!boost::filesystem::is_directory(dir_path.c_str())) {
-                std::stringstream ss;
-                ss << "Path(" << dir_path << ") already exists, but not a directory.";
-                return Status::InternalError(ss.str());
+using strings::Substitute;
+
+Status FileUtils::create_dir(const std::string& path, Env* env) {
+    if (path.empty()) {
+        return Status::InvalidArgument(Substitute("Unknown primitive type($0)", path));
+    }
+
+    boost::filesystem::path p(path);
+
+    string partial_path;
+    for (boost::filesystem::path::iterator it = p.begin(); it != p.end(); ++it) {
+        partial_path = partial_path.empty() ? it->string() : partial_path + "/" + it->string();
+        bool is_dir = false;
+        
+        Status s = env->is_directory(partial_path, &is_dir);
+        
+        if (s.ok()) {
+            if (is_dir) {
+                // It's a normal directory.
+                continue;
             }
-        } else {
-            if (!boost::filesystem::create_directories(dir_path.c_str())) {
-                std::stringstream ss;
-                ss << "make directory failed. path=" << dir_path;
-                return Status::InternalError(ss.str());
+
+            // Maybe a file or a symlink. Let's try to follow the symlink.
+            string real_partial_path;
+            RETURN_IF_ERROR(env->canonicalize(partial_path, &real_partial_path));
+            
+            RETURN_IF_ERROR(env->is_directory(real_partial_path, &is_dir));
+            if (is_dir) {
+                // It's a symlink to a directory.
+                continue;
+            } else {
+                return Status::IOError(partial_path + " exists but is not a directory");
             }
         }
-    } catch (...) {
-        std::stringstream ss;
-        ss << "make directory failed. path=" << dir_path;
-        return Status::InternalError(ss.str());
+        
+        RETURN_IF_ERROR(env->create_dir(partial_path));
     }
+
     return Status::OK();
 }
 
+Status FileUtils::create_dir(const std::string& dir_path) {
+    return create_dir(dir_path, Env::Default());
+}
+
 Status FileUtils::remove_all(const std::string& file_path) {
     try {
         boost::filesystem::path boost_path(file_path);
@@ -82,6 +105,28 @@ Status FileUtils::remove_all(const std::string& file_path) {
     return Status::OK();
 }
 
+Status FileUtils::remove(const std::string& path, doris::Env* env) {
+    bool is_dir;
+    RETURN_IF_ERROR(env->is_directory(path, &is_dir));
+ 
+    if (is_dir) {
+        return env->delete_dir(path);
+    } else {
+        return env->delete_file(path);
+    }
+}
+
+Status FileUtils::remove(const std::string& path) {
+    return remove(path, Env::Default());
+}
+
+Status FileUtils::remove_paths(const std::vector<string>& paths) {
+    for (const string& p : paths) {
+        RETURN_IF_ERROR(remove(p));
+    }
+    return Status::OK();
+}
+
 Status FileUtils::list_files(Env* env, const std::string& dir,
                              std::vector<std::string>* files) {
     auto cb = [files](const char* name) -> bool {
@@ -93,6 +138,35 @@ Status FileUtils::list_files(Env* env, const std::string& dir,
     return env->iterate_dir(dir, cb);
 }
 
+Status FileUtils::list_dirs_files(const std::string& path, std::set<std::string>* dirs,
+                             std::set<std::string>* files, Env* env) {
+    auto cb = [path, dirs, files, env](const char* name) -> bool {
+        if (is_dot_or_dotdot(name)) {
+            return true;
+        }
+        
+        string temp_path =  path + "/" + name;
+        bool is_dir;
+        
+        auto st = env->is_directory(temp_path, &is_dir);
+        if (st.ok()) {
+            if (is_dir) {
+                if (dirs != nullptr) {
+                    dirs->insert(name);
+                }
+            } else if (files != nullptr) {
+                files->insert(name);
+            }
+        } else {
+            LOG(WARNING) << "check path " << path << "is directory error: " << st.to_string();
+        }
+        
+        return true;
+    };
+    
+    return env->iterate_dir(path, cb);
+}
+
 Status FileUtils::get_children_count(Env* env, const std::string& dir, int64_t* count) {
     auto cb = [count](const char* name) -> bool {
         if (!is_dot_or_dotdot(name)) {
@@ -103,19 +177,19 @@ Status FileUtils::get_children_count(Env* env, const std::string& dir, int64_t*
     return env->iterate_dir(dir, cb);
 }
 
-bool FileUtils::is_dir(const std::string& path) {
-    struct stat path_stat;    
-    if (stat(path.c_str(), &path_stat) != 0) {
-        return false;
+bool FileUtils::is_dir(const std::string& file_path, Env* env) {
+    bool ret;
+    if (env->is_directory(file_path, &ret).ok()) {
+        return ret;   
     }
-
-    if (path_stat.st_mode & S_IFDIR) {
-        return true;
-    }
-
+    
     return false;
 }
 
+bool FileUtils::is_dir(const std::string& path) {
+    return is_dir(path, Env::Default());
+}
+
 // Through proc filesystem
 std::string FileUtils::path_of_fd(int fd) {
     const int PATH_SIZE = 256;
@@ -234,13 +308,11 @@ Status FileUtils::md5sum(const std::string& file, std::string* md5sum) {
 }
 
 bool FileUtils::check_exist(const std::string& path) {
-    boost::system::error_code errcode;
-    bool exist = boost::filesystem::exists(path, errcode);
-    if (errcode != boost::system::errc::success && errcode != boost::system::errc::no_such_file_or_directory) {
-        LOG(WARNING) << "error when check path:" << path << ", error code:" << errcode;
-        return false;
-    }
-    return exist;
+    return Env::Default()->path_exists(path).ok();
+}
+
+bool FileUtils::check_exist(const std::string& path, Env* env) {
+    return env->path_exists(path).ok();
 }
 
 }
diff --git a/be/src/util/file_utils.h b/be/src/util/file_utils.h
index 8043841..4bf11a6 100644
--- a/be/src/util/file_utils.h
+++ b/be/src/util/file_utils.h
@@ -19,6 +19,7 @@
 #define DORIS_BE_UTIL_FILE_UTILS_H
 
 #include <string>
+#include <vector>
 #include <functional>
 
 #include "common/status.h"
@@ -35,7 +36,7 @@ inline bool is_dot_or_dotdot(const char* name) {
 class FileUtils {
 public:
 
-    // Create directory of dir_path, 
+    // Create directory of dir_path with default Env, 
     // This function will create directory recursively,
     // if dir's parent directory doesn't exist
     //
@@ -43,9 +44,24 @@ public:
     //  Status::OK()      if create directory success or directory already exists
     static Status create_dir(const std::string& dir_path);
 
+    // Create directory of dir_path, 
+    // This function will create directory recursively,
+    // if dir's parent directory doesn't exist
+    //
+    // RETURNS:
+    //  Status::OK()      if create directory success or directory already exists
+    static Status create_dir(const std::string& dir_path, Env* env);
+
     // Delete file recursively.
     static Status remove_all(const std::string& dir_path);
-
+    
+    // Delete dir or file, failed when there are files or dirs under the path
+    static Status remove(const std::string& path, Env* env);
+
+    static Status remove(const std::string& path);
+    
+    static Status remove_paths(const std::vector<std::string>& paths);
+    
     // List all files in the specified directory without '.' and '..'.
     // If you want retreive all files, you can use Env::iterate_dir.
     // All valid files will be stored in given *files.
@@ -53,12 +69,22 @@ public:
         Env* env,
         const std::string& dir,
         std::vector<std::string>* files);
+    
+    // List all dirs and files in the specified directory
+    static Status list_dirs_files(
+            const std::string& path,
+            std::set<std::string>* dirs,
+            std::set<std::string>* files, 
+            Env* env);
 
     // Get the number of children belong to the specified directory, this
     // funciton also exclude '.' and '..'.
     // Return OK with *count is set to the count, if execute successful.
     static Status get_children_count(Env* env, const std::string& dir, int64_t* count);
 
+    // Check the file_path is not exist with default env, or is not a dir, return false.
+    static bool is_dir(const std::string& file_path, Env* env);
+
     // If the file_path is not exist, or is not a dir, return false.
     static bool is_dir(const std::string& file_path);
 
@@ -80,7 +106,11 @@ public:
     // calc md5sum of a local file
     static Status md5sum(const std::string& file, std::string* md5sum);
 
+    // check path(file or directory) exist with default env
     static bool check_exist(const std::string& path);
+    
+    // check path(file or directory) exist with env
+    static bool check_exist(const std::string& path, Env* env);
 };
 
 }
diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp
index 4daa740..ccfe0ca 100644
--- a/be/test/olap/delete_handler_test.cpp
+++ b/be/test/olap/delete_handler_test.cpp
@@ -33,6 +33,7 @@
 #include "olap/utils.h"
 #include "olap/options.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 
 using namespace std;
 using namespace doris;
@@ -48,9 +49,9 @@ void set_up() {
     char buffer[MAX_PATH_LEN];
     getcwd(buffer, MAX_PATH_LEN);
     config::storage_root_path = string(buffer) + "/data_test";
-    remove_all_dir(config::storage_root_path);
-    remove_all_dir(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
-    create_dir(config::storage_root_path);
+    FileUtils::remove_all(config::storage_root_path);
+    FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::create_dir(config::storage_root_path);
     std::vector<StorePath> paths;
     paths.emplace_back(config::storage_root_path, -1);
     config::min_file_descriptor_number = 1000;
@@ -64,8 +65,8 @@ void tear_down() {
     char buffer[MAX_PATH_LEN];
     getcwd(buffer, MAX_PATH_LEN);
     config::storage_root_path = string(buffer) + "/data_test";
-    remove_all_dir(config::storage_root_path);
-    remove_all_dir(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::remove_all(config::storage_root_path);
+    FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
 }
 
 void set_default_create_tablet_request(TCreateTabletReq* request) {
@@ -163,8 +164,8 @@ protected:
         char buffer[MAX_PATH_LEN];
         getcwd(buffer, MAX_PATH_LEN);
         config::storage_root_path = string(buffer) + "/data_delete_condition";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -186,7 +187,7 @@ protected:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok());
     }
 
     std::string _tablet_path;
@@ -272,8 +273,8 @@ protected:
         char buffer[MAX_PATH_LEN];
         getcwd(buffer, MAX_PATH_LEN);
         config::storage_root_path = string(buffer) + "/data_delete_condition";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -295,7 +296,7 @@ protected:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok());
     }
 
     std::string _tablet_path;
@@ -609,8 +610,8 @@ protected:
         char buffer[MAX_PATH_LEN];
         getcwd(buffer, MAX_PATH_LEN);
         config::storage_root_path = string(buffer) + "/data_delete_condition";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -636,7 +637,7 @@ protected:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok());
     }
 
     std::string _tablet_path;
diff --git a/be/test/olap/delta_writer_test.cpp b/be/test/olap/delta_writer_test.cpp
index 134f09b..c4175f1 100644
--- a/be/test/olap/delta_writer_test.cpp
+++ b/be/test/olap/delta_writer_test.cpp
@@ -34,6 +34,7 @@
 #include "runtime/mem_pool.h"
 #include "runtime/mem_tracker.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 #include "olap/options.h"
 #include "olap/tablet_meta_manager.h"
 
@@ -52,8 +53,8 @@ void set_up() {
     char buffer[MAX_PATH_LEN];
     getcwd(buffer, MAX_PATH_LEN);
     config::storage_root_path = std::string(buffer) + "/data_test";
-    remove_all_dir(config::storage_root_path);
-    create_dir(config::storage_root_path);
+    FileUtils::remove_all(config::storage_root_path);
+    FileUtils::create_dir(config::storage_root_path);
     std::vector<StorePath> paths;
     paths.emplace_back(config::storage_root_path, -1);
 
@@ -71,7 +72,7 @@ void tear_down() {
     delete k_engine;
     k_engine = nullptr;
     system("rm -rf ./data_test");
-    remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
     delete k_mem_tracker;
 }
 
diff --git a/be/test/olap/file_utils_test.cpp b/be/test/olap/file_utils_test.cpp
index 2e56bd5..0869352 100644
--- a/be/test/olap/file_utils_test.cpp
+++ b/be/test/olap/file_utils_test.cpp
@@ -17,6 +17,8 @@
 
 #include <algorithm>
 #include <fstream>
+#include <set>
+#include <vector>
 #include "gtest/gtest.h"
 #include "gmock/gmock.h"
 #include "agent/status.h"
@@ -25,6 +27,7 @@
 #include "olap/file_helper.h"
 #include "util/file_utils.h"
 #include "util/logging.h"
+#include "env/env.h"
 
 #ifndef BE_TEST
 #define BE_TEST
@@ -86,6 +89,152 @@ TEST_F(FileUtilsTest, TestCopyFile) {
     ASSERT_EQ(src_length, dst_length);
 }
 
+TEST_F(FileUtilsTest, TestRemove) {
+    // remove_all
+    ASSERT_TRUE(FileUtils::remove_all("./file_test").ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test"));
+    
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/123/456/789").ok());
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/abc/def/zxc").ok());
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/abc/123").ok());
+    
+    boost::filesystem::save_string_file("./file_test/s1", "123");
+    boost::filesystem::save_string_file("./file_test/123/s2", "123");
+    
+    ASSERT_TRUE(FileUtils::check_exist("./file_test"));
+    ASSERT_TRUE(FileUtils::remove_all("./file_test").ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test"));
+    
+    // remove 
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/abc/123").ok());
+   boost::filesystem::save_string_file("./file_test/abc/123/s2", "123");
+
+    ASSERT_FALSE(FileUtils::remove("./file_test").ok());
+    ASSERT_FALSE(FileUtils::remove("./file_test/abc/").ok());
+    ASSERT_FALSE(FileUtils::remove("./file_test/abc/123").ok());
+    
+    ASSERT_TRUE(FileUtils::check_exist("./file_test/abc/123/s2"));
+    ASSERT_TRUE(FileUtils::remove("./file_test/abc/123/s2").ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/abc/123/s2"));
+
+    ASSERT_TRUE(FileUtils::check_exist("./file_test/abc/123"));
+    ASSERT_TRUE(FileUtils::remove("./file_test/abc/123/").ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/abc/123"));
+    
+    ASSERT_TRUE(FileUtils::remove_all("./file_test").ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test"));
+    
+    // remove paths
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/123/456/789").ok());
+    ASSERT_TRUE(FileUtils::create_dir("./file_test/abc/def/zxc").ok());
+    boost::filesystem::save_string_file("./file_test/s1", "123");
+    boost::filesystem::save_string_file("./file_test/s2", "123");
+    
+    std::vector<std::string> ps;
+    ps.push_back("./file_test/123/456/789");
+    ps.push_back("./file_test/123/456");
+    ps.push_back("./file_test/123");
+
+    ASSERT_TRUE(FileUtils::check_exist("./file_test/123"));
+    ASSERT_TRUE(FileUtils::remove_paths(ps).ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/123"));
+    
+    ps.clear();
+    ps.push_back("./file_test/s1");
+    ps.push_back("./file_test/abc/def");
+    
+    ASSERT_FALSE(FileUtils::remove_paths(ps).ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/s1"));
+    ASSERT_TRUE(FileUtils::check_exist("./file_test/abc/def/"));
+    
+    ps.clear();
+    ps.push_back("./file_test/abc/def/zxc");
+    ps.push_back("./file_test/s2");
+    ps.push_back("./file_test/abc/def");
+    ps.push_back("./file_test/abc");
+
+    ASSERT_TRUE(FileUtils::remove_paths(ps).ok());
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/s2"));
+    ASSERT_FALSE(FileUtils::check_exist("./file_test/abc"));
+    
+    ASSERT_TRUE(FileUtils::remove_all("./file_test").ok());
+}
+
+TEST_F(FileUtilsTest, TestCreateDir) {
+    // normal
+    std::string path = "./file_test/123/456/789";
+    FileUtils::remove_all("./file_test");
+    ASSERT_FALSE(FileUtils::check_exist(path));
+
+    ASSERT_TRUE(FileUtils::create_dir(path).ok());
+
+    ASSERT_TRUE(FileUtils::check_exist(path));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123/456"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123/456/789"));
+    
+    FileUtils::remove_all("./file_test");
+
+    // normal
+    path = "./file_test/123/456/789/";
+    FileUtils::remove_all("./file_test");
+    ASSERT_FALSE(FileUtils::check_exist(path));
+
+    ASSERT_TRUE(FileUtils::create_dir(path).ok());
+
+    ASSERT_TRUE(FileUtils::check_exist(path));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123/456"));
+    ASSERT_TRUE(FileUtils::is_dir("./file_test/123/456/789"));
+
+    FileUtils::remove_all("./file_test");
+}
+
+TEST_F(FileUtilsTest, TestListDirsFiles) {
+    std::string path = "./file_test/";
+    FileUtils::remove_all(path);
+    FileUtils::create_dir("./file_test/1");
+    FileUtils::create_dir("./file_test/2");
+    FileUtils::create_dir("./file_test/3");
+    FileUtils::create_dir("./file_test/4");
+    FileUtils::create_dir("./file_test/5");
+    
+    std::set<string> dirs;
+    std::set<string> files;
+    
+    ASSERT_TRUE(FileUtils::list_dirs_files("./file_test", &dirs, &files, Env::Default()).ok());
+    ASSERT_EQ(5, dirs.size());
+    ASSERT_EQ(0, files.size());
+
+    dirs.clear();
+    files.clear();
+
+    ASSERT_TRUE(FileUtils::list_dirs_files("./file_test", &dirs, nullptr, Env::Default()).ok());
+    ASSERT_EQ(5, dirs.size());
+    ASSERT_EQ(0, files.size());
+    
+    boost::filesystem::save_string_file("./file_test/f1", "just test");
+    boost::filesystem::save_string_file("./file_test/f2", "just test");
+    boost::filesystem::save_string_file("./file_test/f3", "just test");
+    
+    dirs.clear();
+    files.clear();
+    
+    ASSERT_TRUE(FileUtils::list_dirs_files("./file_test", &dirs, &files, Env::Default()).ok());
+    ASSERT_EQ(5, dirs.size());
+    ASSERT_EQ(3, files.size());
+
+    dirs.clear();
+    files.clear();
+    
+    ASSERT_TRUE(FileUtils::list_dirs_files("./file_test", nullptr, &files, Env::Default()).ok());
+    ASSERT_EQ(0, dirs.size());
+    ASSERT_EQ(3, files.size());
+    
+    FileUtils::remove_all(path);
+}
 }  // namespace doris
 
 int main(int argc, char **argv) {
diff --git a/be/test/olap/memtable_flush_executor_test.cpp b/be/test/olap/memtable_flush_executor_test.cpp
index cfbcab5..d028fe6 100644
--- a/be/test/olap/memtable_flush_executor_test.cpp
+++ b/be/test/olap/memtable_flush_executor_test.cpp
@@ -35,6 +35,7 @@
 #include "runtime/descriptor_helper.h"
 #include "runtime/exec_env.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 #include "olap/options.h"
 #include "olap/tablet_meta_manager.h"
 
@@ -47,8 +48,8 @@ void set_up() {
     char buffer[1024];
     getcwd(buffer, 1024);
     config::storage_root_path = std::string(buffer) + "/flush_test";
-    remove_all_dir(config::storage_root_path);
-    create_dir(config::storage_root_path);
+    FileUtils::remove_all(config::storage_root_path);
+    FileUtils::create_dir(config::storage_root_path);
     std::vector<StorePath> paths;
     paths.emplace_back(config::storage_root_path, -1);
 
@@ -66,7 +67,7 @@ void tear_down() {
     delete k_engine;
     k_engine = nullptr;
     system("rm -rf ./flush_test");
-    remove_all_dir(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::remove_all(std::string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
 }
 
 Schema create_schema() {
diff --git a/be/test/olap/olap_reader_test.cpp b/be/test/olap/olap_reader_test.cpp
old mode 100755
new mode 100644
index d7f9261..f866bd2
--- a/be/test/olap/olap_reader_test.cpp
+++ b/be/test/olap/olap_reader_test.cpp
@@ -33,6 +33,7 @@
 #include "util/runtime_profile.h"
 #include "util/debug_util.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 
 using namespace testing;
 using namespace doris;
@@ -43,13 +44,13 @@ namespace doris {
 void set_up() {
     config::storage_root_path = "./test_run/data_test";
     system("rm -rf ./test_run && mkdir -p ./test_run");
-    create_dir(config::storage_root_path);
+    FileUtils::create_dir(config::storage_root_path);
     touch_all_singleton();    
 }
 
 void tear_down() {
     system("rm -rf ./test_run");
-    remove_all_dir(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
 }
 
 void set_default_create_tablet_request(TCreateTabletReq* request) {
@@ -151,14 +152,14 @@ public:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(, FileUtils::remove_all(config::storage_root_path).ok());
     }
 
     void init_olap() {
         // Create local data dir for StorageEngine.
         config::storage_root_path = "./test_run/row_tablet";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -683,14 +684,14 @@ public:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path));
     }
 
     void init_olap() {
         // Create local data dir for StorageEngine.
         config::storage_root_path = "./test_run/column_tablet";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -1164,14 +1165,14 @@ public:
         while (0 == access(_tablet_path.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path));
     }
 
     void init_olap() {
         // Create local data dir for StorageEngine.
         config::storage_root_path = "./test_run/row_tablet";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
diff --git a/be/test/olap/olap_snapshot_converter_test.cpp b/be/test/olap/olap_snapshot_converter_test.cpp
index 1b30ddd..75348eb 100644
--- a/be/test/olap/olap_snapshot_converter_test.cpp
+++ b/be/test/olap/olap_snapshot_converter_test.cpp
@@ -32,6 +32,7 @@
 #include <boost/algorithm/string.hpp>
 #include "boost/filesystem.hpp"
 #include "json2pb/json_to_pb.h"
+#include "util/file_utils.h"
 
 #ifndef BE_TEST
 #define BE_TEST
@@ -63,7 +64,7 @@ public:
         string test_engine_data_path = "./be/test/olap/test_data/converter_test_data/data";
         _engine_data_path = "./be/test/olap/test_data/converter_test_data/tmp";
         boost::filesystem::remove_all(_engine_data_path);
-        create_dirs(_engine_data_path);
+        FileUtils::create_dir(_engine_data_path);
 
         _data_dir = new DataDir(_engine_data_path, 1000000000);
         _data_dir->init();
diff --git a/be/test/olap/rowset/alpha_rowset_test.cpp b/be/test/olap/rowset/alpha_rowset_test.cpp
index 9401792..7cdef70 100644
--- a/be/test/olap/rowset/alpha_rowset_test.cpp
+++ b/be/test/olap/rowset/alpha_rowset_test.cpp
@@ -24,6 +24,7 @@
 #include "boost/filesystem.hpp"
 #include "json2pb/json_to_pb.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 #include "olap/olap_meta.h"
 #include "olap/rowset/rowset_writer.h"
 #include "olap/rowset/rowset_writer_context.h"
@@ -52,27 +53,22 @@ void set_up() {
     char buffer[MAX_PATH_LEN];
     getcwd(buffer, MAX_PATH_LEN);
     config::storage_root_path = std::string(buffer) + "/data_test";
-    remove_all_dir(config::storage_root_path);
-    OLAPStatus res = create_dir(config::storage_root_path);
-    ASSERT_EQ(OLAP_SUCCESS, res);
+    FileUtils::remove_all(config::storage_root_path);
+    ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
     std::vector<StorePath> paths;
     paths.emplace_back(config::storage_root_path, -1);
     std::string data_path = config::storage_root_path + "/data";
-    res = create_dir(data_path);
-    ASSERT_EQ(OLAP_SUCCESS, res);
+    ASSERT_TRUE(FileUtils::create_dir(data_path).ok());
     std::string shard_path = data_path + "/0";
-    res = create_dir(shard_path);
-    ASSERT_EQ(OLAP_SUCCESS, res);
+    ASSERT_TRUE(FileUtils::create_dir(shard_path).ok());
     std::string tablet_path = shard_path + "/12345";
-    res = create_dir(tablet_path);
-    ASSERT_EQ(OLAP_SUCCESS, res);
+    ASSERT_TRUE(FileUtils::create_dir(tablet_path).ok());
     std::string schema_hash_path = tablet_path + "/1111";
-    res = create_dir(schema_hash_path);
-    ASSERT_EQ(OLAP_SUCCESS, res);
+    ASSERT_TRUE(FileUtils::create_dir(schema_hash_path).ok());
 }
 
 void tear_down() {
-    remove_all_dir(config::storage_root_path);
+    FileUtils::remove_all(config::storage_root_path);
 }
 
 void create_rowset_writer_context(TabletSchema* tablet_schema,
diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp
index 1222a86..9cc3eb6 100644
--- a/be/test/olap/rowset/beta_rowset_test.cpp
+++ b/be/test/olap/rowset/beta_rowset_test.cpp
@@ -34,6 +34,7 @@
 #include "runtime/mem_tracker.h"
 #include "runtime/mem_pool.h"
 #include "util/slice.h"
+#include "util/file_utils.h"
 
 using std::string;
 
@@ -45,18 +46,14 @@ protected:
     OlapReaderStatistics _stats;
 
     void SetUp() override {
-        OLAPStatus s;
-        if (check_dir_existed(kRowsetDir)) {
-            s = remove_all_dir(kRowsetDir);
-            ASSERT_EQ(OLAP_SUCCESS, s);
+        if (FileUtils::check_exist(kRowsetDir)) {
+            ASSERT_TRUE(FileUtils::remove_all(kRowsetDir).ok());
         }
-        s = create_dir(kRowsetDir);
-        ASSERT_EQ(OLAP_SUCCESS, s);
+        ASSERT_TRUE(FileUtils::create_dir(kRowsetDir).ok());
     }
     void TearDown() override {
-        if (check_dir_existed(kRowsetDir)) {
-            auto s = remove_all_dir(kRowsetDir);
-            ASSERT_EQ(OLAP_SUCCESS, s);
+        if (FileUtils::check_exist(kRowsetDir)) {
+            ASSERT_TRUE(FileUtils::remove_all(kRowsetDir).ok());
         }
     }
 
diff --git a/be/test/olap/tablet_mgr_test.cpp b/be/test/olap/tablet_mgr_test.cpp
index b5af488..15ba8fe 100644
--- a/be/test/olap/tablet_mgr_test.cpp
+++ b/be/test/olap/tablet_mgr_test.cpp
@@ -29,6 +29,7 @@
 #include "olap/txn_manager.h"
 #include "boost/filesystem.hpp"
 #include "json2pb/json_to_pb.h"
+#include "util/file_utils.h"
 
 #ifndef BE_TEST
 #define BE_TEST
@@ -51,8 +52,8 @@ public:
         string test_engine_data_path = "./be/test/olap/test_data/converter_test_data/data";
         _engine_data_path = "./be/test/olap/test_data/converter_test_data/tmp";
         boost::filesystem::remove_all(_engine_data_path);
-        create_dirs(_engine_data_path);
-        create_dirs(_engine_data_path + "/meta");
+        FileUtils::create_dir(_engine_data_path);
+        FileUtils::create_dir(_engine_data_path + "/meta");
 
         std::vector<StorePath> paths;
         paths.emplace_back("_engine_data_path", -1);
@@ -124,7 +125,7 @@ TEST_F(TabletMgrTest, CreateTablet) {
     TabletSharedPtr tablet = _tablet_mgr.get_tablet(111, 3333);
     ASSERT_TRUE(tablet != nullptr);
     // check dir exist
-    bool dir_exist = check_dir_existed(tablet->tablet_path());
+    bool dir_exist = FileUtils::check_exist(tablet->tablet_path());
     ASSERT_TRUE(dir_exist);
     // check meta has this tablet
     TabletMetaSharedPtr new_tablet_meta(new TabletMeta());
@@ -186,7 +187,7 @@ TEST_F(TabletMgrTest, DropTablet) {
 
     // check dir exist
     std::string tablet_path = tablet->tablet_path();
-    bool dir_exist = check_dir_existed(tablet_path);
+    bool dir_exist = FileUtils::check_exist(tablet_path);
     ASSERT_TRUE(dir_exist);
 
     // do trash sweep, tablet will not be garbage collected
@@ -195,7 +196,7 @@ TEST_F(TabletMgrTest, DropTablet) {
     ASSERT_TRUE(trash_st == OLAP_SUCCESS);
     tablet = _tablet_mgr.get_tablet(111, 3333, true);
     ASSERT_TRUE(tablet != nullptr);
-    dir_exist = check_dir_existed(tablet_path);
+    dir_exist = FileUtils::check_exist(tablet_path);
     ASSERT_TRUE(dir_exist);
 
     // reset tablet ptr
@@ -204,7 +205,7 @@ TEST_F(TabletMgrTest, DropTablet) {
     ASSERT_TRUE(trash_st == OLAP_SUCCESS);
     tablet = _tablet_mgr.get_tablet(111, 3333, true);
     ASSERT_TRUE(tablet == nullptr);
-    dir_exist = check_dir_existed(tablet_path);
+    dir_exist = FileUtils::check_exist(tablet_path);
     ASSERT_TRUE(!dir_exist);
 }
 
diff --git a/be/test/olap/vectorized_olap_reader_test.cpp b/be/test/olap/vectorized_olap_reader_test.cpp
index 545367c..c07ff68 100644
--- a/be/test/olap/vectorized_olap_reader_test.cpp
+++ b/be/test/olap/vectorized_olap_reader_test.cpp
@@ -36,6 +36,7 @@
 #include "util/runtime_profile.h"
 #include "util/debug_util.h"
 #include "util/logging.h"
+#include "util/file_utils.h"
 
 using namespace testing;
 using namespace doris;
@@ -46,13 +47,13 @@ namespace doris {
 void set_up() {
     config::storage_root_path = "./test_run/data_test";
     system("rm -rf ./test_run && mkdir -p ./test_run");
-    create_dir(config::storage_root_path);
+    FileUtils::create_dir(config::storage_root_path);
     touch_all_singleton();
 }
 
 void tear_down() {
     system("rm -rf ./test_run");
-    remove_all_dir(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
+    FileUtils::remove_all(string(getenv("DORIS_HOME")) + UNUSED_PREFIX);
 }
 
 void set_default_create_tablet_request(TCreateTabletReq* request) {
@@ -153,14 +154,14 @@ public:
         while (0 == access(_tablet_name.c_str(), F_OK)) {
             sleep(1);
         }
-        ASSERT_EQ(OLAP_SUCCESS, remove_all_dir(config::storage_root_path));
+        ASSERT_TRUE(FileUtils::remove_all(config::storage_root_path).ok());
     }
 
     void init_olap_row() {
         // Create local data dir for StorageEngine.
         config::storage_root_path = "./test_run/row_tablet";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
@@ -185,8 +186,8 @@ public:
     void init_olap_column() {
         // Create local data dir for StorageEngine.
         config::storage_root_path = "./test_run/column_tablet";
-        remove_all_dir(config::storage_root_path);
-        ASSERT_EQ(create_dir(config::storage_root_path), OLAP_SUCCESS);
+        FileUtils::remove_all(config::storage_root_path);
+        ASSERT_TRUE(FileUtils::create_dir(config::storage_root_path).ok());
 
         // 1. Prepare for query split key.
         // create base tablet
diff --git a/be/test/runtime/data_spliter_test.cpp b/be/test/runtime/data_spliter_test.cpp
index c5db787..7b9aecd 100644
--- a/be/test/runtime/data_spliter_test.cpp
+++ b/be/test/runtime/data_spliter_test.cpp
@@ -29,6 +29,7 @@
 #include "gen_cpp/DataSinks_types.h"
 #include "gen_cpp/Types_types.h"
 #include "olap/olap_main.cpp"
+#include "util/file_utils.h"
 
 namespace doris {
 
@@ -276,7 +277,7 @@ int main(int argc, char** argv) {
     }
     // 覆盖be.conf中的配置
     doris::config::storage_root_path = "./test_run/mini_load";
-    doris::create_dirs(doris::config::storage_root_path);
+    doris::FileUtils::create_dir(doris::config::storage_root_path);
     doris::touch_all_singleton();
 
     doris::CpuInfo::init();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org