You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/07/02 10:58:30 UTC

[doris] branch master updated: [refactor] Refactoring Status static methods to format message using fmt(#9533)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c9f86bc7e2 [refactor] Refactoring Status static methods to format message using fmt(#9533)
c9f86bc7e2 is described below

commit c9f86bc7e287b3fc07caa23dda64b8a44b54d17b
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Sat Jul 2 18:58:23 2022 +0800

    [refactor] Refactoring Status static methods to format message using fmt(#9533)
---
 be/src/agent/agent_server.cpp                      |  35 ++---
 be/src/agent/cgroups_mgr.cpp                       |  10 +-
 be/src/common/configbase.cpp                       |  60 ++++----
 be/src/common/status.h                             | 156 ++++++++++++---------
 be/src/env/env_posix.cpp                           |  35 ++---
 be/src/exec/assert_num_rows_node.cpp               |   5 +-
 be/src/exec/base_scanner.cpp                       |  24 ++--
 be/src/exec/broker_scan_node.cpp                   |   8 +-
 be/src/exec/broker_scanner.cpp                     |  14 +-
 be/src/exec/csv_scanner.cpp                        |   2 +-
 be/src/exec/decompressor.cpp                       |  63 +++------
 be/src/exec/es/es_predicate.cpp                    |   9 +-
 be/src/exec/es/es_scan_reader.cpp                  |   4 +-
 be/src/exec/es/es_scroll_parser.cpp                |  20 +--
 be/src/exec/es_http_scan_node.cpp                  |   4 +-
 be/src/exec/es_http_scanner.cpp                    |   4 +-
 be/src/exec/json_scanner.cpp                       |   6 +-
 be/src/exec/mysql_scan_node.cpp                    |  13 +-
 be/src/exec/odbc_connector.cpp                     |   6 +-
 be/src/exec/odbc_scan_node.cpp                     |  13 +-
 be/src/exec/olap_scanner.cpp                       |   8 +-
 be/src/exec/parquet_scanner.cpp                    |   5 +-
 be/src/exec/parquet_writer.cpp                     |  78 +++++------
 be/src/exec/partitioned_hash_table.cc              |   6 +-
 be/src/exec/plain_text_line_reader.cpp             |   5 +-
 be/src/exec/read_write_util.h                      |  10 +-
 be/src/exec/tablet_info.cpp                        |  61 ++++----
 be/src/exec/tablet_sink.cpp                        |  12 +-
 be/src/exprs/agg_fn.cpp                            |   6 +-
 be/src/exprs/agg_fn_evaluator.cpp                  |   4 +-
 be/src/exprs/block_bloom_filter_impl.cc            |  15 +-
 be/src/exprs/expr.cpp                              |   4 +-
 be/src/exprs/rpc_fn.cpp                            |  13 +-
 be/src/exprs/rpc_fn_call.cpp                       |   4 +-
 be/src/exprs/scalar_fn_call.cpp                    |   4 +-
 be/src/exprs/slot_ref.cpp                          |  23 ++-
 be/src/exprs/table_function/table_function.h       |  11 +-
 .../table_function/table_function_factory.cpp      |   4 +-
 be/src/http/action/compaction_action.cpp           |  11 +-
 be/src/http/action/download_action.cpp             |   8 +-
 be/src/http/action/meta_action.cpp                 |   2 +-
 be/src/http/action/stream_load.cpp                 |  21 ++-
 be/src/http/action/stream_load_2pc.cpp             |   2 +-
 be/src/http/action/tablet_migration_action.cpp     |   8 +-
 be/src/http/action/tablets_distribution_action.cpp |   3 +-
 be/src/http/ev_http_server.cpp                     |   4 +-
 be/src/http/http_client.h                          |   3 +-
 be/src/io/buffered_reader.cpp                      |   4 +-
 be/src/io/hdfs_file_reader.cpp                     |  39 ++----
 be/src/io/hdfs_writer.cpp                          |   8 +-
 be/src/io/local_file_writer.cpp                    |  25 ++--
 be/src/io/s3_reader.cpp                            |  11 +-
 be/src/io/s3_writer.cpp                            |  30 ++--
 be/src/olap/column_vector.cpp                      |   4 +-
 be/src/olap/data_dir.cpp                           |  21 ++-
 be/src/olap/fs/remote_block_manager.cpp            |  23 ++-
 be/src/olap/key_coder.h                            |  12 +-
 be/src/olap/olap_server.cpp                        |  22 +--
 be/src/olap/row_block2.cpp                         |   4 +-
 be/src/olap/rowset/segment_v2/binary_dict_page.cpp |  13 +-
 be/src/olap/rowset/segment_v2/binary_plain_page.h  |  12 +-
 .../olap/rowset/segment_v2/binary_prefix_page.cpp  |   9 +-
 .../olap/rowset/segment_v2/bitmap_index_writer.cpp |   2 +-
 be/src/olap/rowset/segment_v2/bitshuffle_page.h    |  32 ++---
 .../segment_v2/bitshuffle_page_pre_decoder.h       |   7 +-
 be/src/olap/rowset/segment_v2/bloom_filter.cpp     |   3 +-
 be/src/olap/rowset/segment_v2/bloom_filter.h       |   6 +-
 .../segment_v2/bloom_filter_index_writer.cpp       |   2 +-
 be/src/olap/rowset/segment_v2/column_reader.cpp    |  19 ++-
 be/src/olap/rowset/segment_v2/column_writer.cpp    |   7 +-
 be/src/olap/rowset/segment_v2/encoding_info.cpp    |   5 +-
 .../rowset/segment_v2/indexed_column_reader.cpp    |   3 +-
 be/src/olap/rowset/segment_v2/page_io.cpp          |  12 +-
 be/src/olap/rowset/segment_v2/plain_page.h         |  13 +-
 be/src/olap/rowset/segment_v2/segment.cpp          |  23 ++-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |   3 +-
 be/src/olap/short_key_index.cpp                    |   5 +-
 be/src/olap/storage_engine.cpp                     |  16 +--
 be/src/olap/tablet.cpp                             |   6 +-
 be/src/olap/task/engine_batch_load_task.cpp        |   5 +-
 be/src/runtime/buffered_tuple_stream3.cc           |   6 +-
 be/src/runtime/bufferpool/system_allocator.cc      |   5 +-
 be/src/runtime/collection_value.cpp                |   2 +-
 be/src/runtime/disk_io_mgr.cc                      |  25 ++--
 be/src/runtime/disk_io_mgr_scan_range.cc           |  15 +-
 be/src/runtime/dpp_sink_internal.cpp               |  12 +-
 be/src/runtime/dpp_writer.cpp                      |   4 +-
 be/src/runtime/export_task_mgr.cpp                 |   8 +-
 be/src/runtime/file_result_writer.cpp              |   7 +-
 be/src/runtime/fragment_mgr.cpp                    |  19 ++-
 be/src/runtime/load_channel_mgr.h                  |   4 +-
 be/src/runtime/mem_tracker.h                       |   8 +-
 be/src/runtime/memory/chunk_allocator.cpp          |   3 +-
 be/src/runtime/mysql_table_writer.cpp              |   6 +-
 be/src/runtime/routine_load/data_consumer.cpp      |  10 +-
 be/src/runtime/routine_load/data_consumer_pool.cpp |   4 +-
 .../routine_load/routine_load_task_executor.cpp    |   2 +-
 be/src/runtime/row_batch.cpp                       |   4 +-
 be/src/runtime/small_file_mgr.cpp                  |  12 +-
 be/src/runtime/snapshot_loader.cpp                 |   8 +-
 be/src/runtime/stream_load/stream_load_pipe.h      |  10 +-
 .../runtime/stream_load/stream_load_recorder.cpp   |   4 +-
 be/src/runtime/tablets_channel.cpp                 |   4 +-
 be/src/runtime/tablets_channel.h                   |  11 +-
 be/src/runtime/tmp_file_mgr.cc                     |   8 +-
 be/src/tools/meta_tool.cpp                         |  20 ++-
 be/src/util/array_parser.h                         |   4 +-
 be/src/util/arrow/row_batch.cpp                    |  36 ++---
 be/src/util/arrow/row_block.cpp                    |   4 +-
 be/src/util/block_compression.cpp                  |  81 +++++------
 be/src/util/broker_storage_backend.cpp             |  12 +-
 be/src/util/cgroup_util.cpp                        |  38 +++--
 be/src/util/dynamic_util.cpp                       |   8 +-
 be/src/util/file_utils.cpp                         |   8 +-
 be/src/util/filesystem_util.cc                     |  51 +++----
 be/src/util/jni-util.cpp                           |   2 +-
 be/src/util/jni-util.h                             |   6 +-
 be/src/util/load_error_hub.cpp                     |   4 +-
 be/src/util/network_util.cpp                       |  12 +-
 be/src/util/pprof_utils.cpp                        |   8 +-
 be/src/util/proto_util.h                           |   8 +-
 be/src/util/s3_storage_backend.cpp                 |  22 +--
 be/src/util/storage_backend_mgr.cpp                |  10 +-
 be/src/util/thread.cpp                             |   8 +-
 be/src/util/threadpool.cpp                         |   6 +-
 be/src/util/thrift_util.h                          |  16 +--
 be/src/util/zip_util.cpp                           |  14 +-
 be/src/util/zlib.cpp                               |   2 +-
 be/src/vec/core/block.cpp                          |   8 +-
 be/src/vec/data_types/data_type_array.cpp          |   4 +-
 be/src/vec/data_types/data_type_date.cpp           |   4 +-
 be/src/vec/data_types/data_type_date_time.cpp      |   4 +-
 be/src/vec/data_types/data_type_decimal.cpp        |   4 +-
 be/src/vec/data_types/data_type_number_base.cpp    |  10 +-
 be/src/vec/exec/vaggregation_node.cpp              |   2 +-
 be/src/vec/exec/vanalytic_eval_node.cpp            |   2 +-
 be/src/vec/exec/varrow_scanner.cpp                 |   5 +-
 be/src/vec/exec/vassert_num_rows_node.cpp          |   5 +-
 be/src/vec/exec/vbroker_scan_node.cpp              |   8 +-
 be/src/vec/exec/ves_http_scan_node.cpp             |   4 +-
 be/src/vec/exec/vmysql_scan_node.cpp               |   7 +-
 be/src/vec/exec/vrepeat_node.cpp                   |   5 +-
 be/src/vec/exec/vschema_scan_node.cpp              |   7 +-
 be/src/vec/exprs/table_function/vexplode.cpp       |   5 +-
 be/src/vec/exprs/vcase_expr.cpp                    |   4 +-
 be/src/vec/exprs/vcast_expr.cpp                    |   3 +-
 be/src/vec/exprs/vectorized_agg_fn.cpp             |   3 +-
 be/src/vec/exprs/vectorized_fn_call.cpp            |   3 +-
 be/src/vec/exprs/vexpr.cpp                         |   3 +-
 be/src/vec/exprs/vin_predicate.cpp                 |   3 +-
 be/src/vec/exprs/vslot_ref.cpp                     |   2 +-
 .../functions/array/function_array_aggregation.cpp |   2 +-
 .../vec/functions/array/function_array_element.h   |   7 +-
 be/src/vec/functions/array/function_array_index.h  |   8 +-
 be/src/vec/functions/array/function_array_size.h   |   5 +-
 be/src/vec/functions/date_time_transforms.h        |   6 +-
 be/src/vec/functions/function.cpp                  |  10 +-
 .../vec/functions/function_always_not_nullable.h   |   6 +-
 be/src/vec/functions/function_binary_arithmetic.h  |   4 +-
 be/src/vec/functions/function_case.h               |   3 +-
 be/src/vec/functions/function_cast.h               |  59 ++++----
 be/src/vec/functions/function_coalesce.cpp         |   3 +-
 .../function_date_or_datetime_computation.h        |  12 +-
 .../function_date_or_datetime_to_string.h          |   6 +-
 .../vec/functions/function_datetime_floor_ceil.cpp |   6 +-
 .../functions/function_datetime_string_to_string.h |  11 +-
 be/src/vec/functions/function_fake.h               |   2 +-
 be/src/vec/functions/function_hash.cpp             |  15 +-
 be/src/vec/functions/function_math_unary.h         |   4 +-
 .../functions/function_math_unary_to_null_type.h   |   4 +-
 be/src/vec/functions/function_string.h             |  12 +-
 be/src/vec/functions/function_string_to_string.h   |   6 +-
 be/src/vec/functions/function_totype.h             |  26 ++--
 be/src/vec/functions/function_unary_arithmetic.h   |   4 +-
 be/src/vec/functions/functions_comparison.h        |  15 +-
 be/src/vec/functions/hll_cardinality.cpp           |   6 +-
 be/src/vec/functions/if.cpp                        |  21 +--
 be/src/vec/functions/in.cpp                        |   4 +-
 be/src/vec/functions/like.cpp                      |  10 +-
 be/src/vec/runtime/vfile_result_writer.cpp         |   9 +-
 be/src/vec/sink/vtablet_sink.cpp                   |   2 +-
 181 files changed, 958 insertions(+), 1248 deletions(-)

diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp
index 58ce4e00b5..84ea9d3929 100644
--- a/be/src/agent/agent_server.cpp
+++ b/be/src/agent/agent_server.cpp
@@ -121,14 +121,14 @@ void AgentServer::submit_tasks(TAgentResult& agent_result,
         TTaskType::type task_type = task.task_type;
         int64_t signature = task.signature;
 
-#define HANDLE_TYPE(t_task_type, work_pool, req_member)                         \
-    case t_task_type:                                                           \
-        if (task.__isset.req_member) {                                          \
-            work_pool->submit_task(task);                                       \
-        } else {                                                                \
-            ret_st = Status::InvalidArgument(strings::Substitute(               \
-                    "task(signature=$0) has wrong request member", signature)); \
-        }                                                                       \
+#define HANDLE_TYPE(t_task_type, work_pool, req_member)                                          \
+    case t_task_type:                                                                            \
+        if (task.__isset.req_member) {                                                           \
+            work_pool->submit_task(task);                                                        \
+        } else {                                                                                 \
+            ret_st = Status::InvalidArgument("task(signature={}) has wrong request member = {}", \
+                                             signature, #req_member);                            \
+        }                                                                                        \
         break;
 
         // TODO(lingbin): It still too long, divided these task types into several categories
@@ -158,8 +158,8 @@ void AgentServer::submit_tasks(TAgentResult& agent_result,
         case TTaskType::REALTIME_PUSH:
         case TTaskType::PUSH:
             if (!task.__isset.push_req) {
-                ret_st = Status::InvalidArgument(strings::Substitute(
-                        "task(signature=$0) has wrong request member", signature));
+                ret_st = Status::InvalidArgument(
+                        "task(signature={}) has wrong request member = push_req", signature);
                 break;
             }
             if (task.push_req.push_type == TPushType::LOAD ||
@@ -168,22 +168,23 @@ void AgentServer::submit_tasks(TAgentResult& agent_result,
             } else if (task.push_req.push_type == TPushType::DELETE) {
                 _delete_workers->submit_task(task);
             } else {
-                ret_st = Status::InvalidArgument(strings::Substitute(
-                        "task(signature=$0, type=$1, push_type=$2) has wrong push_type", signature,
-                        task_type, task.push_req.push_type));
+                ret_st = Status::InvalidArgument(
+                        "task(signature={}, type={}, push_type={}) has wrong push_type", signature,
+                        task_type, task.push_req.push_type);
             }
             break;
         case TTaskType::ALTER:
             if (task.__isset.alter_tablet_req || task.__isset.alter_tablet_req_v2) {
                 _alter_tablet_workers->submit_task(task);
             } else {
-                ret_st = Status::InvalidArgument(strings::Substitute(
-                        "task(signature=$0) has wrong request member", signature));
+                ret_st = Status::InvalidArgument(
+                        "task(signature={}) has wrong request member = alter_tablet_req",
+                        signature);
             }
             break;
         default:
-            ret_st = Status::InvalidArgument(strings::Substitute(
-                    "task(signature=$0, type=$1) has wrong task type", signature, task_type));
+            ret_st = Status::InvalidArgument("task(signature={}, type={}) has wrong task type",
+                                             signature, task_type);
             break;
         }
 #undef HANDLE_TYPE
diff --git a/be/src/agent/cgroups_mgr.cpp b/be/src/agent/cgroups_mgr.cpp
index 936ed73427..e130a7b247 100644
--- a/be/src/agent/cgroups_mgr.cpp
+++ b/be/src/agent/cgroups_mgr.cpp
@@ -170,7 +170,7 @@ Status CgroupsMgr::_config_disk_throttle(std::string user_name, std::string leve
     if (!is_file_exist(cgroups_path.c_str())) {
         if (!std::filesystem::create_directory(cgroups_path)) {
             LOG(ERROR) << "Create cgroups: " << cgroups_path << " failed";
-            return Status::InternalError("Create cgroups " + cgroups_path + " failed");
+            return Status::InternalError("Create cgroups {} failed", cgroups_path);
         }
     }
 
@@ -236,7 +236,7 @@ Status CgroupsMgr::modify_user_cgroups(const string& user_name,
     if (!is_file_exist(user_cgroups_path.c_str())) {
         if (!std::filesystem::create_directory(user_cgroups_path)) {
             LOG(ERROR) << "Create cgroups for user " << user_name << " failed";
-            return Status::InternalError("Create cgroups for user " + user_name + " failed");
+            return Status::InternalError("Create cgroups for user {} failed", user_name);
         }
     }
 
@@ -294,7 +294,7 @@ Status CgroupsMgr::init_cgroups() {
         if (fs_type.f_type != CGROUP_SUPER_MAGIC) {
             LOG(ERROR) << _root_cgroups_path << " is not a cgroups file system.";
             _is_cgroups_init_success = false;
-            return Status::InternalError(_root_cgroups_path + " is not a cgroups file system");
+            return Status::InternalError("{} is not a cgroups file system", _root_cgroups_path);
             ;
         }
 #endif
@@ -302,7 +302,7 @@ Status CgroupsMgr::init_cgroups() {
         if (access(_root_cgroups_path.c_str(), W_OK) != 0) {
             LOG(ERROR) << "Doris does not have write permission to " << _root_cgroups_path;
             _is_cgroups_init_success = false;
-            return Status::InternalError("Doris does not have write permission to " +
+            return Status::InternalError("Doris does not have write permission to {}",
                                          _root_cgroups_path);
         }
         // If root folder exists, then delete all subfolders under it
@@ -314,7 +314,7 @@ Status CgroupsMgr::init_cgroups() {
                 if (!delete_user_cgroups(item_begin->path().filename().string()).ok()) {
                     LOG(ERROR) << "Could not clean subfolder " << item_begin->path().string();
                     _is_cgroups_init_success = false;
-                    return Status::InternalError("Could not clean subfolder " +
+                    return Status::InternalError("Could not clean subfolder {}",
                                                  item_begin->path().string());
                 }
             }
diff --git a/be/src/common/configbase.cpp b/be/src/common/configbase.cpp
index a34f6b08ab..35d89a69e4 100644
--- a/be/src/common/configbase.cpp
+++ b/be/src/common/configbase.cpp
@@ -358,33 +358,31 @@ bool init(const char* conf_file, bool fill_conf_map, bool must_exist, bool set_t
     return true;
 }
 
-#define UPDATE_FIELD(FIELD, VALUE, TYPE, PERSIST)                                           \
-    if (strcmp((FIELD).type, #TYPE) == 0) {                                                 \
-        TYPE new_value;                                                                     \
-        if (!convert((VALUE), new_value)) {                                                 \
-            return Status::InvalidArgument(                                                 \
-                    strings::Substitute("convert '$0' as $1 failed", VALUE, #TYPE));        \
-        }                                                                                   \
-        TYPE& ref_conf_value = *reinterpret_cast<TYPE*>((FIELD).storage);                   \
-        TYPE old_value = ref_conf_value;                                                    \
-        ref_conf_value = new_value;                                                         \
-        auto validator = RegisterConfValidator::_s_field_validator->find((FIELD).name);     \
-        if (validator != RegisterConfValidator::_s_field_validator->end() &&                \
-            !(validator->second)()) {                                                       \
-            ref_conf_value = old_value;                                                     \
-            return Status::InvalidArgument(                                                 \
-                    strings::Substitute("validate $0=$1 failed", (FIELD).name, new_value)); \
-        }                                                                                   \
-        ref_conf_value = new_value;                                                         \
-        if (full_conf_map != nullptr) {                                                     \
-            std::ostringstream oss;                                                         \
-            oss << new_value;                                                               \
-            (*full_conf_map)[(FIELD).name] = oss.str();                                     \
-        }                                                                                   \
-        if (PERSIST) {                                                                      \
-            persist_config(std::string((FIELD).name), VALUE);                               \
-        }                                                                                   \
-        return Status::OK();                                                                \
+#define UPDATE_FIELD(FIELD, VALUE, TYPE, PERSIST)                                             \
+    if (strcmp((FIELD).type, #TYPE) == 0) {                                                   \
+        TYPE new_value;                                                                       \
+        if (!convert((VALUE), new_value)) {                                                   \
+            return Status::InvalidArgument("convert '{}' as {} failed", VALUE, #TYPE);        \
+        }                                                                                     \
+        TYPE& ref_conf_value = *reinterpret_cast<TYPE*>((FIELD).storage);                     \
+        TYPE old_value = ref_conf_value;                                                      \
+        ref_conf_value = new_value;                                                           \
+        auto validator = RegisterConfValidator::_s_field_validator->find((FIELD).name);       \
+        if (validator != RegisterConfValidator::_s_field_validator->end() &&                  \
+            !(validator->second)()) {                                                         \
+            ref_conf_value = old_value;                                                       \
+            return Status::InvalidArgument("validate {}={} failed", (FIELD).name, new_value); \
+        }                                                                                     \
+        ref_conf_value = new_value;                                                           \
+        if (full_conf_map != nullptr) {                                                       \
+            std::ostringstream oss;                                                           \
+            oss << new_value;                                                                 \
+            (*full_conf_map)[(FIELD).name] = oss.str();                                       \
+        }                                                                                     \
+        if (PERSIST) {                                                                        \
+            persist_config(std::string((FIELD).name), VALUE);                                 \
+        }                                                                                     \
+        return Status::OK();                                                                  \
     }
 
 // write config to be_custom.conf
@@ -408,11 +406,11 @@ bool persist_config(const std::string& field, const std::string& value) {
 Status set_config(const std::string& field, const std::string& value, bool need_persist) {
     auto it = Register::_s_field_map->find(field);
     if (it == Register::_s_field_map->end()) {
-        return Status::NotFound(strings::Substitute("'$0' is not found", field));
+        return Status::NotFound("'{}' is not found", field);
     }
 
     if (!it->second.valmutable) {
-        return Status::NotSupported(strings::Substitute("'$0' is not support to modify", field));
+        return Status::NotSupported("'{}' is not support to modify", field);
     }
 
     UPDATE_FIELD(it->second, value, bool, need_persist);
@@ -427,8 +425,8 @@ Status set_config(const std::string& field, const std::string& value, bool need_
     }
 
     // The other types are not thread safe to change dynamically.
-    return Status::NotSupported(strings::Substitute(
-            "'$0' is type of '$1' which is not support to modify", field, it->second.type));
+    return Status::NotSupported("'{}' is type of '{}' which is not support to modify", field,
+                                it->second.type);
 }
 
 std::mutex* get_mutable_string_config_lock() {
diff --git a/be/src/common/status.h b/be/src/common/status.h
index e26f756abd..68824df80a 100644
--- a/be/src/common/status.h
+++ b/be/src/common/status.h
@@ -279,102 +279,119 @@ public:
 
     static Status OK() { return Status(); }
 
-    static Status PublishTimeout(const Slice& msg, int16_t precise_code = 1,
-                                 const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::PUBLISH_TIMEOUT, msg, precise_code, msg2);
-    }
-    static Status MemoryAllocFailed(const Slice& msg, int16_t precise_code = 1,
-                                    const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::MEM_ALLOC_FAILED, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status ErrorFmt(TStatusCode::type code, const std::string& fmt, Args&&... args) {
+        // In some cases, fmt contains '{}' but there are no args.
+        if (sizeof...(args) == 0) {
+            return Status(code, fmt);
+        }
+        return Status(code, fmt::format(fmt, std::forward<Args>(args)...));
     }
-    static Status BufferAllocFailed(const Slice& msg, int16_t precise_code = 1,
-                                    const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::BUFFER_ALLOCATION_FAILED, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status PublishTimeout(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::PUBLISH_TIMEOUT, fmt, std::forward<Args>(args)...);
     }
-    static Status InvalidArgument(const Slice& msg, int16_t precise_code = 1,
-                                  const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::INVALID_ARGUMENT, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status MemoryAllocFailed(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::MEM_ALLOC_FAILED, fmt, std::forward<Args>(args)...);
     }
-    static Status MinimumReservationUnavailable(const Slice& msg, int16_t precise_code = 1,
-                                                const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::MINIMUM_RESERVATION_UNAVAILABLE, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status BufferAllocFailed(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::BUFFER_ALLOCATION_FAILED, fmt, std::forward<Args>(args)...);
     }
-    static Status Corruption(const Slice& msg, int16_t precise_code = 1,
-                             const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::CORRUPTION, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status InvalidArgument(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::INVALID_ARGUMENT, fmt, std::forward<Args>(args)...);
     }
-    static Status IOError(const Slice& msg, int16_t precise_code = 1, const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::IO_ERROR, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status MinimumReservationUnavailable(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::MINIMUM_RESERVATION_UNAVAILABLE, fmt,
+                        std::forward<Args>(args)...);
     }
-    static Status NotFound(const Slice& msg, int16_t precise_code = 1,
-                           const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::NOT_FOUND, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status Corruption(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::CORRUPTION, fmt, std::forward<Args>(args)...);
     }
-    static Status AlreadyExist(const Slice& msg, int16_t precise_code = 1,
-                               const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::ALREADY_EXIST, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status IOError(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::IO_ERROR, fmt, std::forward<Args>(args)...);
     }
-    static Status NotSupported(const Slice& msg, int16_t precise_code = 1,
-                               const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::NOT_IMPLEMENTED_ERROR, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status NotFound(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::NOT_FOUND, fmt, std::forward<Args>(args)...);
     }
-    static Status EndOfFile(const Slice& msg, int16_t precise_code = 1,
-                            const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::END_OF_FILE, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status AlreadyExist(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::ALREADY_EXIST, fmt, std::forward<Args>(args)...);
     }
-    static Status InternalError(const Slice& msg, int16_t precise_code = 1,
-                                const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::INTERNAL_ERROR, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status NotSupported(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::NOT_IMPLEMENTED_ERROR, fmt, std::forward<Args>(args)...);
     }
-    static Status RuntimeError(const Slice& msg, int16_t precise_code = 1,
-                               const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::RUNTIME_ERROR, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status EndOfFile(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::END_OF_FILE, fmt, std::forward<Args>(args)...);
     }
-    static Status Cancelled(const Slice& msg, int16_t precise_code = 1,
-                            const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::CANCELLED, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status InternalError(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::INTERNAL_ERROR, fmt, std::forward<Args>(args)...);
     }
 
-    static Status MemoryLimitExceeded(const Slice& msg, int16_t precise_code = 1,
-                                      const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::MEM_LIMIT_EXCEEDED, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status RuntimeError(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::RUNTIME_ERROR, fmt, std::forward<Args>(args)...);
+    }
+    template <typename... Args>
+    static Status Cancelled(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::CANCELLED, fmt, std::forward<Args>(args)...);
     }
 
-    static Status ThriftRpcError(const Slice& msg, int16_t precise_code = 1,
-                                 const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::THRIFT_RPC_ERROR, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status MemoryLimitExceeded(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::MEM_LIMIT_EXCEEDED, fmt, std::forward<Args>(args)...);
     }
 
-    static Status TimedOut(const Slice& msg, int16_t precise_code = 1,
-                           const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::TIMEOUT, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status ThriftRpcError(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::THRIFT_RPC_ERROR, fmt, std::forward<Args>(args)...);
     }
 
-    static Status TooManyTasks(const Slice& msg, int16_t precise_code = 1,
-                               const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::TOO_MANY_TASKS, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status TimedOut(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::TIMEOUT, fmt, std::forward<Args>(args)...);
     }
-    static Status ServiceUnavailable(const Slice& msg, int16_t precise_code = -1,
-                                     const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::SERVICE_UNAVAILABLE, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status TooManyTasks(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::TOO_MANY_TASKS, fmt, std::forward<Args>(args)...);
     }
-    static Status Uninitialized(const Slice& msg, int16_t precise_code = -1,
-                                const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::UNINITIALIZED, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status ServiceUnavailable(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::SERVICE_UNAVAILABLE, fmt, std::forward<Args>(args)...);
     }
-    static Status Aborted(const Slice& msg, int16_t precise_code = -1,
-                          const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::ABORTED, msg, precise_code, msg2);
+
+    template <typename... Args>
+    static Status Uninitialized(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::UNINITIALIZED, fmt, std::forward<Args>(args)...);
     }
 
-    static Status DataQualityError(const Slice& msg, int16_t precise_code = -1,
-                                   const Slice& msg2 = Slice()) {
-        return Status(TStatusCode::DATA_QUALITY_ERROR, msg, precise_code, msg2);
+    template <typename... Args>
+    static Status Aborted(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::ABORTED, fmt, std::forward<Args>(args)...);
     }
 
     template <typename... Args>
-    static Status OLAPInternalError(int16_t precise_code, const std::string& fmt, Args&&... args) {
-        return ConstructErrorStatus(precise_code, fmt::format(fmt, std::forward<Args>(args)...));
+    static Status DataQualityError(const std::string& fmt, Args&&... args) {
+        return ErrorFmt(TStatusCode::DATA_QUALITY_ERROR, fmt, std::forward<Args>(args)...);
     }
 
     // A wrapper for ErrorCode
@@ -535,7 +552,8 @@ private:
         }
     }
 
-    Status(TStatusCode::type code, const Slice& msg, int16_t precise_code, const Slice& msg2) {
+    Status(TStatusCode::type code, const Slice& msg, int16_t precise_code = 1,
+           const Slice& msg2 = Slice()) {
         assemble_state(code, msg, precise_code, msg2);
     }
 
diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp
index a31cc92aa1..5a841324fb 100644
--- a/be/src/env/env_posix.cpp
+++ b/be/src/env/env_posix.cpp
@@ -66,16 +66,19 @@ static Status io_error(const std::string& context, int err_number) {
     case ENAMETOOLONG:
     case ENOENT:
     case ENOTDIR:
-        return Status::NotFound(context, err_number, errno_to_string(err_number));
+        return Status::NotFound("{} (error {}) {}", context, err_number,
+                                errno_to_string(err_number));
     case EEXIST:
-        return Status::AlreadyExist(context, err_number, errno_to_string(err_number));
+        return Status::AlreadyExist("{} (error {}) {}", context, err_number,
+                                    errno_to_string(err_number));
     case EOPNOTSUPP:
     case EXDEV: // No cross FS links allowed
-        return Status::NotSupported(context, err_number, errno_to_string(err_number));
+        return Status::NotSupported("{} (error {}) {}", context, err_number,
+                                    errno_to_string(err_number));
     case EIO:
         LOG(ERROR) << "I/O error, context=" << context;
     }
-    return Status::IOError(context, err_number, errno_to_string(err_number));
+    return Status::IOError("{} (error {}) {}", context, err_number, errno_to_string(err_number));
 }
 
 static Status do_sync(int fd, const string& filename) {
@@ -100,7 +103,7 @@ static Status do_open(const string& filename, Env::OpenMode mode, int* fd) {
     case Env::MUST_EXIST:
         break;
     default:
-        return Status::NotSupported(strings::Substitute("Unknown create mode $0", mode));
+        return Status::NotSupported("Unknown create mode {}", mode);
     }
     int f;
     RETRY_ON_EINTR(f, open(filename.c_str(), flags, 0666));
@@ -137,8 +140,7 @@ static Status do_readv_at(int fd, const std::string& filename, uint64_t offset,
         }
 
         if (PREDICT_FALSE(r == 0)) {
-            return Status::EndOfFile(strings::Substitute("EOF trying to read $0 bytes at offset $1",
-                                                         bytes_req, offset));
+            return Status::EndOfFile("EOF trying to read {} bytes at offset {}", bytes_req, offset);
         }
 
         if (PREDICT_TRUE(r == rem)) {
@@ -244,9 +246,9 @@ public:
     Status read_all(std::string* content) const override {
         std::fstream fs(_filename.c_str(), std::fstream::in);
         if (!fs.is_open()) {
-            RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(strings::Substitute(
-                                                   "failed to open cluster id file $0", _filename)),
-                                           "open file failed");
+            RETURN_NOT_OK_STATUS_WITH_WARN(
+                    Status::IOError("failed to open cluster id file {}", _filename),
+                    "open file failed");
         }
         std::string data;
         fs >> data;
@@ -255,11 +257,11 @@ public:
             *content = data;
         } else {
             RETURN_NOT_OK_STATUS_WITH_WARN(
-                    Status::Corruption(strings::Substitute(
-                            "read_all from file $0 is corrupt. [eofbit=$1 failbit=$2 badbit=$3]",
+                    Status::Corruption(
+                            "read_all from file {} is corrupt. [eofbit={} failbit={} badbit={}]",
                             _filename, fs.rdstate() & std::fstream::eofbit,
                             fs.rdstate() & std::fstream::failbit,
-                            fs.rdstate() & std::fstream::badbit)),
+                            fs.rdstate() & std::fstream::badbit),
                     "read_all is error");
         }
         return Status::OK();
@@ -602,7 +604,7 @@ Status PosixEnv::create_dir_if_missing(const string& dirname, bool* created) {
 
 Status PosixEnv::create_dirs(const string& dirname) {
     if (dirname.empty()) {
-        return Status::InvalidArgument(strings::Substitute("Unknown primitive type($0)", dirname));
+        return Status::InvalidArgument("Unknown primitive type({})", dirname);
     }
 
     std::filesystem::path p(dirname);
@@ -629,7 +631,7 @@ Status PosixEnv::create_dirs(const string& dirname) {
                 // It's a symlink to a directory.
                 continue;
             } else {
-                return Status::IOError(partial_path + " exists but is not a directory");
+                return Status::IOError("{} exists but is not a directory", partial_path);
             }
         }
 
@@ -746,8 +748,7 @@ Status PosixEnv::get_space_info(const std::string& path, int64_t* capacity, int6
         *available = path_info.available;
     } catch (std::filesystem::filesystem_error& e) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
-                Status::IOError(strings::Substitute(
-                        "get path $0 available capacity failed, error=$1", path, e.what())),
+                Status::IOError("get path {} available capacity failed, error={}", path, e.what()),
                 "std::filesystem::space failed");
     }
     return Status::OK();
diff --git a/be/src/exec/assert_num_rows_node.cpp b/be/src/exec/assert_num_rows_node.cpp
index c91d7ef085..3766d83ad3 100644
--- a/be/src/exec/assert_num_rows_node.cpp
+++ b/be/src/exec/assert_num_rows_node.cpp
@@ -99,9 +99,8 @@ Status AssertNumRowsNode::get_next(RuntimeState* state, RowBatch* output_batch,
         };
         LOG(INFO) << "Expected " << to_string_lambda(_assertion) << " " << _desired_num_rows
                   << " to be returned by expression " << _subquery_string;
-        return Status::Cancelled(strings::Substitute(
-                "Expected $0 $1 to be returned by expression $2", to_string_lambda(_assertion),
-                _desired_num_rows, _subquery_string));
+        return Status::Cancelled("Expected {} {} to be returned by expression {}",
+                                 to_string_lambda(_assertion), _desired_num_rows, _subquery_string);
     }
     COUNTER_SET(_rows_returned_counter, _num_rows_returned);
     return Status::OK();
diff --git a/be/src/exec/base_scanner.cpp b/be/src/exec/base_scanner.cpp
index e8e4d21118..3ec5aba1cd 100644
--- a/be/src/exec/base_scanner.cpp
+++ b/be/src/exec/base_scanner.cpp
@@ -100,9 +100,8 @@ Status BaseScanner::init_expr_ctxes() {
     const TupleDescriptor* src_tuple_desc =
             _state->desc_tbl().get_tuple_descriptor(_params.src_tuple_id);
     if (src_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Unknown source tuple descriptor, tuple_id=" << _params.src_tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown source tuple descriptor, tuple_id={}",
+                                     _params.src_tuple_id);
     }
 
     std::map<SlotId, SlotDescriptor*> src_slot_desc_map;
@@ -112,9 +111,7 @@ Status BaseScanner::init_expr_ctxes() {
     for (auto slot_id : _params.src_slot_ids) {
         auto it = src_slot_desc_map.find(slot_id);
         if (it == std::end(src_slot_desc_map)) {
-            std::stringstream ss;
-            ss << "Unknown source slot descriptor, slot_id=" << slot_id;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Unknown source slot descriptor, slot_id={}", slot_id);
         }
         _src_slot_descs.emplace_back(it->second);
     }
@@ -147,9 +144,8 @@ Status BaseScanner::init_expr_ctxes() {
     // Construct dest slots information
     _dest_tuple_desc = _state->desc_tbl().get_tuple_descriptor(_params.dest_tuple_id);
     if (_dest_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Unknown dest tuple descriptor, tuple_id=" << _params.dest_tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown dest tuple descriptor, tuple_id={}",
+                                     _params.dest_tuple_id);
     }
 
     bool has_slot_id_map = _params.__isset.dest_sid_to_src_sid_without_trans;
@@ -159,10 +155,8 @@ Status BaseScanner::init_expr_ctxes() {
         }
         auto it = _params.expr_of_dest_slot.find(slot_desc->id());
         if (it == std::end(_params.expr_of_dest_slot)) {
-            std::stringstream ss;
-            ss << "No expr for dest slot, id=" << slot_desc->id()
-               << ", name=" << slot_desc->col_name();
-            return Status::InternalError(ss.str());
+            return Status::InternalError("No expr for dest slot, id={}, name={}", slot_desc->id(),
+                                         slot_desc->col_name());
         }
 
         if (_state->enable_vectorized_exec()) {
@@ -186,9 +180,7 @@ Status BaseScanner::init_expr_ctxes() {
             } else {
                 auto _src_slot_it = src_slot_desc_map.find(it->second);
                 if (_src_slot_it == std::end(src_slot_desc_map)) {
-                    std::stringstream ss;
-                    ss << "No src slot " << it->second << " in src slot descs";
-                    return Status::InternalError(ss.str());
+                    return Status::InternalError("No src slot {} in src slot descs", it->second);
                 }
                 _src_slot_descs_order_by_dest.emplace_back(_src_slot_it->second);
             }
diff --git a/be/src/exec/broker_scan_node.cpp b/be/src/exec/broker_scan_node.cpp
index fce8757e65..ed9e646397 100644
--- a/be/src/exec/broker_scan_node.cpp
+++ b/be/src/exec/broker_scan_node.cpp
@@ -68,18 +68,14 @@ Status BrokerScanNode::prepare(RuntimeState* state) {
     _runtime_state = state;
     _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
     if (_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Failed to get tuple descriptor, _tuple_id=" << _tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to get tuple descriptor, _tuple_id={}", _tuple_id);
     }
 
     // Initialize slots map
     for (auto slot : _tuple_desc->slots()) {
         auto pair = _slots_map.emplace(slot->col_name(), slot);
         if (!pair.second) {
-            std::stringstream ss;
-            ss << "Failed to insert slot, col_name=" << slot->col_name();
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Failed to insert slot, col_name={}", slot->col_name());
         }
     }
 
diff --git a/be/src/exec/broker_scanner.cpp b/be/src/exec/broker_scanner.cpp
index 29350645bb..aed2acd7b8 100644
--- a/be/src/exec/broker_scanner.cpp
+++ b/be/src/exec/broker_scanner.cpp
@@ -171,9 +171,8 @@ Status BrokerScanner::create_decompressor(TFileFormatType::type type) {
         compress_type = CompressType::DEFLATE;
         break;
     default: {
-        std::stringstream ss;
-        ss << "Unknown format type, cannot inference compress type, type=" << type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown format type, cannot inference compress type, type={}",
+                                     type);
     }
     }
     RETURN_IF_ERROR(Decompressor::create_decompressor(compress_type, &_cur_decompressor));
@@ -196,9 +195,7 @@ Status BrokerScanner::open_line_reader() {
     int64_t size = range.size;
     if (range.start_offset != 0) {
         if (range.format_type != TFileFormatType::FORMAT_CSV_PLAIN) {
-            std::stringstream ss;
-            ss << "For now we do not support split compressed file";
-            return Status::InternalError(ss.str());
+            return Status::InternalError("For now we do not support split compressed file");
         }
         size += 1;
         // not first range will always skip one line
@@ -226,9 +223,8 @@ Status BrokerScanner::open_line_reader() {
         _cur_line_reader = new PlainBinaryLineReader(_cur_file_reader.get());
         break;
     default: {
-        std::stringstream ss;
-        ss << "Unknown format type, cannot init line reader, type=" << range.format_type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown format type, cannot init line reader, type={}",
+                                     range.format_type);
     }
     }
 
diff --git a/be/src/exec/csv_scanner.cpp b/be/src/exec/csv_scanner.cpp
index 77a42243ec..59c480d650 100644
--- a/be/src/exec/csv_scanner.cpp
+++ b/be/src/exec/csv_scanner.cpp
@@ -68,7 +68,7 @@ Status CsvScanner::get_next_row(std::string* line_str, bool* eos) {
 
         _current_file = new std::ifstream(file_path, std::ifstream::in);
         if (!_current_file->is_open()) {
-            return Status::InternalError("Fail to read csv file: " + file_path);
+            return Status::InternalError("Fail to read csv file: {}", file_path);
         }
         ++_current_file_idx;
     }
diff --git a/be/src/exec/decompressor.cpp b/be/src/exec/decompressor.cpp
index 3d56ac3c0a..5d7dee008e 100644
--- a/be/src/exec/decompressor.cpp
+++ b/be/src/exec/decompressor.cpp
@@ -42,9 +42,7 @@ Status Decompressor::create_decompressor(CompressType type, Decompressor** decom
         break;
 #endif
     default:
-        std::stringstream ss;
-        ss << "Unknown compress type: " << type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown compress type: {}", type);
     }
 
     Status st = Status::OK();
@@ -77,9 +75,7 @@ Status GzipDecompressor::init() {
     int window_bits = _is_deflate ? WINDOW_BITS : (WINDOW_BITS | DETECT_CODEC);
     int ret = inflateInit2(&_z_strm, window_bits);
     if (ret < 0) {
-        std::stringstream ss;
-        ss << "Failed to init inflate. status code: " << ret;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to init inflate. status code: {}", ret);
     }
 
     return Status::OK();
@@ -123,14 +119,10 @@ Status GzipDecompressor::decompress(uint8_t* input, size_t input_len, size_t* in
             // reset _z_strm to continue decoding a subsequent gzip stream
             ret = inflateReset(&_z_strm);
             if (ret != Z_OK) {
-                std::stringstream ss;
-                ss << "Failed to inflateReset. return code: " << ret;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("Failed to inflateReset. return code: {}", ret);
             }
         } else if (ret != Z_OK) {
-            std::stringstream ss;
-            ss << "Failed to inflate. return code: " << ret;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Failed to inflate. return code: {}", ret);
         } else {
             // here ret must be Z_OK.
             // we continue if avail_out and avail_in > 0.
@@ -157,9 +149,7 @@ Status Bzip2Decompressor::init() {
     bzero(&_bz_strm, sizeof(_bz_strm));
     int ret = BZ2_bzDecompressInit(&_bz_strm, 0, 0);
     if (ret != BZ_OK) {
-        std::stringstream ss;
-        ss << "Failed to init bz2. status code: " << ret;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to init bz2. status code: {}", ret);
     }
 
     return Status::OK();
@@ -185,28 +175,22 @@ Status Bzip2Decompressor::decompress(uint8_t* input, size_t input_len, size_t* i
         if (ret == BZ_DATA_ERROR || ret == BZ_DATA_ERROR_MAGIC) {
             LOG(INFO) << "input_bytes_read: " << *input_bytes_read
                       << " decompressed_len: " << *decompressed_len;
-            std::stringstream ss;
-            ss << "Failed to bz2 decompress. status code: " << ret;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Failed to bz2 decompress. status code: {}", ret);
         } else if (ret == BZ_STREAM_END) {
             *stream_end = true;
             ret = BZ2_bzDecompressEnd(&_bz_strm);
             if (ret != BZ_OK) {
-                std::stringstream ss;
-                ss << "Failed to end bz2 after meet BZ_STREAM_END. status code: " << ret;
-                return Status::InternalError(ss.str());
+                return Status::InternalError(
+                        "Failed to end bz2 after meet BZ_STREAM_END. status code: {}", ret);
             }
 
             ret = BZ2_bzDecompressInit(&_bz_strm, 0, 0);
             if (ret != BZ_OK) {
-                std::stringstream ss;
-                ss << "Failed to init bz2 after meet BZ_STREAM_END. status code: " << ret;
-                return Status::InternalError(ss.str());
+                return Status::InternalError(
+                        "Failed to init bz2 after meet BZ_STREAM_END. status code: {}", ret);
             }
         } else if (ret != BZ_OK) {
-            std::stringstream ss;
-            ss << "Failed to bz2 decompress. status code: " << ret;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Failed to bz2 decompress. status code: {}", ret);
         } else {
             // continue
         }
@@ -260,26 +244,24 @@ Status Lz4FrameDecompressor::decompress(uint8_t* input, size_t input_len, size_t
         //       where lz4 header section is there.
 
         if (input_len < 15) {
-            std::stringstream ss;
-            ss << "Lz4 header size is between 7 and 15 bytes. "
-               << "but input size is only: " << input_len;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "Lz4 header size is between 7 and 15 bytes. "
+                    "but input size is only: {}",
+                    input_len);
         }
 
         LZ4F_frameInfo_t info;
         ret = LZ4F_getFrameInfo(_dctx, &info, (void*)src, &src_size);
         if (LZ4F_isError(ret)) {
-            std::stringstream ss;
-            ss << "LZ4F_getFrameInfo error: " << std::string(LZ4F_getErrorName(ret));
-            return Status::InternalError(ss.str());
+            return Status::InternalError("LZ4F_getFrameInfo error: {}",
+                                         std::string(LZ4F_getErrorName(ret)));
         }
 
         _expect_dec_buf_size = get_block_size(&info);
         if (_expect_dec_buf_size == -1) {
-            std::stringstream ss;
-            ss << "Impossible lz4 block size unless more block sizes are allowed"
-               << std::string(LZ4F_getErrorName(ret));
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "Impossible lz4 block size unless more block sizes are allowed {}",
+                    std::string(LZ4F_getErrorName(ret)));
         }
 
         *input_bytes_read = src_size;
@@ -295,9 +277,8 @@ Status Lz4FrameDecompressor::decompress(uint8_t* input, size_t input_len, size_t
     ret = LZ4F_decompress(_dctx, (void*)output, &output_len, (void*)src, &src_size,
                           /* LZ4F_decompressOptions_t */ nullptr);
     if (LZ4F_isError(ret)) {
-        std::stringstream ss;
-        ss << "Decompression error: " << std::string(LZ4F_getErrorName(ret));
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Decompression error: {}",
+                                     std::string(LZ4F_getErrorName(ret)));
     }
 
     // update
diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp
index 8f231e4692..cbf21d1651 100644
--- a/be/src/exec/es/es_predicate.cpp
+++ b/be/src/exec/es/es_predicate.cpp
@@ -331,9 +331,7 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) {
 
             _disjuncts.push_back(predicate);
         } else {
-            std::stringstream ss;
-            ss << "can not process function predicate[ " << fname << " ]";
-            return Status::InternalError(ss.str());
+            return Status::InternalError("can not process function predicate[ {} ]", fname);
         }
         return Status::OK();
     }
@@ -423,9 +421,8 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) {
     }
 
     // if go to here, report error
-    std::stringstream ss;
-    ss << "build disjuncts failed: node type " << conjunct->node_type() << " is not supported";
-    return Status::InternalError(ss.str());
+    return Status::InternalError("build disjuncts failed: node type {} is not supported",
+                                 conjunct->node_type());
 }
 
 const SlotDescriptor* EsPredicate::get_slot_desc(const SlotRef* slotRef) {
diff --git a/be/src/exec/es/es_scan_reader.cpp b/be/src/exec/es/es_scan_reader.cpp
index 5d67d995cf..8470350dd0 100644
--- a/be/src/exec/es/es_scan_reader.cpp
+++ b/be/src/exec/es/es_scan_reader.cpp
@@ -164,13 +164,13 @@ Status ESScanReader::get_next(bool* scan_eos, std::unique_ptr<ScrollParser>& scr
         if (status == 404) {
             LOG(WARNING) << "request scroll search failure 404["
                          << ", response: " << (response.empty() ? "empty response" : response);
-            return Status::InternalError("No search context found for " + _scroll_id);
+            return Status::InternalError("No search context found for {}", _scroll_id);
         }
         if (status != 200) {
             LOG(WARNING) << "request scroll search failure["
                          << "http status" << status
                          << ", response: " << (response.empty() ? "empty response" : response);
-            return Status::InternalError("request scroll search failure: " +
+            return Status::InternalError("request scroll search failure: {}",
                                          (response.empty() ? "empty response" : response));
         }
     }
diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp
index 36475d9cab..73f648b4a7 100644
--- a/be/src/exec/es/es_scroll_parser.cpp
+++ b/be/src/exec/es/es_scroll_parser.cpp
@@ -72,7 +72,7 @@ std::string json_value_to_string(const rapidjson::Value& value) {
 
 static const std::string ERROR_INVALID_COL_DATA =
         "Data source returned inconsistent column data. "
-        "Expected value of type $0 based on column metadata. This likely indicates a "
+        "Expected value of type {} based on column metadata. This likely indicates a "
         "problem with the data source library.";
 static const std::string ERROR_MEM_LIMIT_EXCEEDED =
         "DataSourceScanNode::$0() failed to allocate "
@@ -265,9 +265,7 @@ Status ScrollParser::parse(const std::string& scroll_result, bool exactly_once)
     _size = 0;
     _document_node.Parse(scroll_result.c_str(), scroll_result.length());
     if (_document_node.HasParseError()) {
-        std::stringstream ss;
-        ss << "Parsing json error, json is: " << scroll_result;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Parsing json error, json is: {}", scroll_result);
     }
 
     if (!exactly_once && !_document_node.HasMember(FIELD_SCROLL_ID)) {
@@ -343,9 +341,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
         if (slot_desc->col_name() == FIELD_ID) {
             // actually this branch will not be reached, this is guaranteed by Doris FE.
             if (pure_doc_value) {
-                std::stringstream ss;
-                ss << "obtain `_id` is not supported in doc_values mode";
-                return Status::RuntimeError(ss.str());
+                return Status::RuntimeError("obtain `_id` is not supported in doc_values mode");
             }
             tuple->set_not_null(slot_desc->null_indicator_offset());
             void* slot = tuple->get_slot(slot_desc->tuple_offset());
@@ -499,8 +495,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, Tuple* tuple,
             } else if (pure_doc_value && col.IsArray() && col[0].IsString()) {
                 is_nested_str = true;
             } else if (pure_doc_value && col.IsArray()) {
-                return Status::InternalError(
-                        strings::Substitute(ERROR_INVALID_COL_DATA, "BOOLEAN"));
+                return Status::InternalError(ERROR_INVALID_COL_DATA, "BOOLEAN");
             }
 
             const rapidjson::Value& str_col = is_nested_str ? col[0] : col;
@@ -605,9 +600,7 @@ Status ScrollParser::fill_columns(const TupleDescriptor* tuple_desc,
         if (slot_desc->col_name() == FIELD_ID) {
             // actually this branch will not be reached, this is guaranteed by Doris FE.
             if (pure_doc_value) {
-                std::stringstream ss;
-                ss << "obtain `_id` is not supported in doc_values mode";
-                return Status::RuntimeError(ss.str());
+                return Status::RuntimeError("obtain `_id` is not supported in doc_values mode");
             }
             // obj[FIELD_ID] must not be NULL
             std::string _id = obj[FIELD_ID].GetString();
@@ -727,8 +720,7 @@ Status ScrollParser::fill_columns(const TupleDescriptor* tuple_desc,
             } else if (pure_doc_value && col.IsArray() && col[0].IsString()) {
                 is_nested_str = true;
             } else if (pure_doc_value && col.IsArray()) {
-                return Status::InternalError(
-                        strings::Substitute(ERROR_INVALID_COL_DATA, "BOOLEAN"));
+                return Status::InternalError(ERROR_INVALID_COL_DATA, "BOOLEAN");
             }
 
             const rapidjson::Value& str_col = is_nested_str ? col[0] : col;
diff --git a/be/src/exec/es_http_scan_node.cpp b/be/src/exec/es_http_scan_node.cpp
index f31059cb3b..acdaeb772d 100644
--- a/be/src/exec/es_http_scan_node.cpp
+++ b/be/src/exec/es_http_scan_node.cpp
@@ -74,9 +74,7 @@ Status EsHttpScanNode::prepare(RuntimeState* state) {
     _runtime_state = state;
     _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
     if (_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Failed to get tuple descriptor, _tuple_id=" << _tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to get tuple descriptor, _tuple_id={}", _tuple_id);
     }
 
     // set up column name vector for ESScrollQueryBuilder
diff --git a/be/src/exec/es_http_scanner.cpp b/be/src/exec/es_http_scanner.cpp
index ca022be173..c6b7dac2bc 100644
--- a/be/src/exec/es_http_scanner.cpp
+++ b/be/src/exec/es_http_scanner.cpp
@@ -64,9 +64,7 @@ EsHttpScanner::~EsHttpScanner() {
 Status EsHttpScanner::open() {
     _tuple_desc = _state->desc_tbl().get_tuple_descriptor(_tuple_id);
     if (_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Unknown tuple descriptor, tuple_id=" << _tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown tuple descriptor, tuple_id={}", _tuple_id);
     }
 
     const std::string& host = _properties.at(ESScanReader::KEY_HOST_PORT);
diff --git a/be/src/exec/json_scanner.cpp b/be/src/exec/json_scanner.cpp
index a7e5263b98..2654070be9 100644
--- a/be/src/exec/json_scanner.cpp
+++ b/be/src/exec/json_scanner.cpp
@@ -292,12 +292,12 @@ Status JsonReader::_generate_json_paths(const std::string& jsonpath,
     rapidjson::Document jsonpaths_doc;
     if (!jsonpaths_doc.Parse(jsonpath.c_str(), jsonpath.length()).HasParseError()) {
         if (!jsonpaths_doc.IsArray()) {
-            return Status::InvalidArgument("Invalid json path: " + jsonpath);
+            return Status::InvalidArgument("Invalid json path: {}", jsonpath);
         } else {
             for (int i = 0; i < jsonpaths_doc.Size(); i++) {
                 const rapidjson::Value& path = jsonpaths_doc[i];
                 if (!path.IsString()) {
-                    return Status::InvalidArgument("Invalid json path: " + jsonpath);
+                    return Status::InvalidArgument("Invalid json path: {}", jsonpath);
                 }
                 std::vector<JsonPath> parsed_paths;
                 JsonFunctions::parse_json_paths(path.GetString(), &parsed_paths);
@@ -306,7 +306,7 @@ Status JsonReader::_generate_json_paths(const std::string& jsonpath,
             return Status::OK();
         }
     } else {
-        return Status::InvalidArgument("Invalid json path: " + jsonpath);
+        return Status::InvalidArgument("Invalid json path: {}", jsonpath);
     }
 }
 
diff --git a/be/src/exec/mysql_scan_node.cpp b/be/src/exec/mysql_scan_node.cpp
index 3e161c1df9..e28d7a1603 100644
--- a/be/src/exec/mysql_scan_node.cpp
+++ b/be/src/exec/mysql_scan_node.cpp
@@ -137,10 +137,8 @@ Status MysqlScanNode::write_text_slot(char* value, int value_length, SlotDescrip
                                       RuntimeState* state) {
     if (!_text_converter->write_slot(slot, _tuple, value, value_length, true, false,
                                      _tuple_pool.get())) {
-        std::stringstream ss;
-        ss << "Fail to convert mysql value:'" << value << "' to " << slot->type() << " on column:`"
-           << slot->col_name() + "`";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Fail to convert mysql value:'{}' to {} on column:`{}`", value,
+                                     slot->type(), slot->col_name());
     }
 
     return Status::OK();
@@ -212,10 +210,9 @@ Status MysqlScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* e
                 if (slot_desc->is_nullable()) {
                     _tuple->set_null(slot_desc->null_indicator_offset());
                 } else {
-                    std::stringstream ss;
-                    ss << "nonnull column contains nullptr. table=" << _table_name
-                       << ", column=" << slot_desc->col_name();
-                    return Status::InternalError(ss.str());
+                    return Status::InternalError(
+                            "nonnull column contains nullptr. table={}, column={}", _table_name,
+                            slot_desc->col_name());
                 }
             } else {
                 RETURN_IF_ERROR(write_text_slot(data[j], length[j], slot_desc, state));
diff --git a/be/src/exec/odbc_connector.cpp b/be/src/exec/odbc_connector.cpp
index 19defd6c37..9eaa654647 100644
--- a/be/src/exec/odbc_connector.cpp
+++ b/be/src/exec/odbc_connector.cpp
@@ -289,10 +289,8 @@ Status ODBCConnector::append(const std::string& table_name, RowBatch* batch,
                     break;
                 }
                 default: {
-                    fmt::memory_buffer err_out;
-                    fmt::format_to(err_out, "can't convert this type to mysql type. type = {}",
-                                   _output_expr_ctxs[j]->root()->type().type);
-                    return Status::InternalError(fmt::to_string(err_out));
+                    return Status::InternalError("can't convert this type to mysql type. type = {}",
+                                                 _output_expr_ctxs[j]->root()->type().type);
                 }
                 }
             }
diff --git a/be/src/exec/odbc_scan_node.cpp b/be/src/exec/odbc_scan_node.cpp
index 3d6f7f65e7..4e27864eda 100644
--- a/be/src/exec/odbc_scan_node.cpp
+++ b/be/src/exec/odbc_scan_node.cpp
@@ -196,16 +196,13 @@ Status OdbcScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eo
                 if (slot_desc->is_nullable()) {
                     _tuple->set_null(slot_desc->null_indicator_offset());
                 } else {
-                    std::stringstream ss;
-                    ss << "nonnull column contains nullptr. table=" << _table_name
-                       << ", column=" << slot_desc->col_name();
-                    return Status::InternalError(ss.str());
+                    return Status::InternalError(
+                            "nonnull column contains nullptr. table={}, column={}", _table_name,
+                            slot_desc->col_name());
                 }
             } else if (column_data.strlen_or_ind > column_data.buffer_length) {
-                std::stringstream ss;
-                ss << "nonnull column contains nullptr. table=" << _table_name
-                   << ", column=" << slot_desc->col_name();
-                return Status::InternalError(ss.str());
+                return Status::InternalError("nonnull column contains nullptr. table={}, column={}",
+                                             _table_name, slot_desc->col_name());
             } else {
                 RETURN_IF_ERROR(write_text_slot(static_cast<char*>(column_data.target_value_ptr),
                                                 column_data.strlen_or_ind, slot_desc, state));
diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp
index d78c21e996..e0e6a68549 100644
--- a/be/src/exec/olap_scanner.cpp
+++ b/be/src/exec/olap_scanner.cpp
@@ -317,11 +317,9 @@ Status OlapScanner::get_batch(RuntimeState* state, RowBatch* batch, bool* eof) {
             auto res = _tablet_reader->next_row_with_aggregation(&_read_row_cursor, mem_pool.get(),
                                                                  &tmp_object_pool, eof);
             if (!res.ok()) {
-                std::stringstream ss;
-                ss << "Internal Error: read storage fail. res=" << res
-                   << ", tablet=" << _tablet->full_name()
-                   << ", backend=" << BackendOptions::get_localhost();
-                return Status::InternalError(ss.str());
+                return Status::InternalError(
+                        "Internal Error: read storage fail. res={}, tablet={}, backend={}", res,
+                        _tablet->full_name(), BackendOptions::get_localhost());
             }
             // If we reach end of this scanner, break
             if (UNLIKELY(*eof)) {
diff --git a/be/src/exec/parquet_scanner.cpp b/be/src/exec/parquet_scanner.cpp
index 81d81656d3..776115427e 100644
--- a/be/src/exec/parquet_scanner.cpp
+++ b/be/src/exec/parquet_scanner.cpp
@@ -115,9 +115,8 @@ Status ParquetScanner::open_next_reader() {
             continue;
         } else {
             if (!status.ok()) {
-                std::stringstream ss;
-                ss << " file: " << range.path << " error:" << status.get_error_msg();
-                return Status::InternalError(ss.str());
+                return Status::InternalError("file: {}, error:{}", range.path,
+                                             status.get_error_msg());
             } else {
                 return status;
             }
diff --git a/be/src/exec/parquet_writer.cpp b/be/src/exec/parquet_writer.cpp
index 1f316a473e..627a37b4cc 100644
--- a/be/src/exec/parquet_writer.cpp
+++ b/be/src/exec/parquet_writer.cpp
@@ -226,10 +226,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             switch (_output_expr_ctxs[index]->root()->type().type) {
             case TYPE_BOOLEAN: {
                 if (_str_schema[index][1] != "boolean") {
-                    std::stringstream ss;
-                    ss << "project field type is boolean, but the definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is boolean, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::BoolWriter* col_writer =
@@ -246,11 +246,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             case TYPE_SMALLINT:
             case TYPE_INT: {
                 if (_str_schema[index][1] != "int32") {
-                    std::stringstream ss;
-                    ss << "project field type is tiny int/small int/int, should use int32, but the "
-                          "definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is tiny int/small int/int, should use int32,"
+                            " but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
 
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
@@ -266,11 +265,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             }
             case TYPE_BIGINT: {
                 if (_str_schema[index][1] != "int64") {
-                    std::stringstream ss;
-                    ss << "project field type is big int, should use int64, but the definition "
-                          "type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is big int, should use int64, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::Int64Writer* col_writer =
@@ -295,10 +293,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             }
             case TYPE_FLOAT: {
                 if (_str_schema[index][1] != "float") {
-                    std::stringstream ss;
-                    ss << "project field type is float, but the definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is float, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::FloatWriter* col_writer =
@@ -313,10 +311,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             }
             case TYPE_DOUBLE: {
                 if (_str_schema[index][1] != "double") {
-                    std::stringstream ss;
-                    ss << "project field type is double, but the definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is double, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::DoubleWriter* col_writer =
@@ -332,11 +330,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             case TYPE_DATETIME:
             case TYPE_DATE: {
                 if (_str_schema[index][1] != "int64") {
-                    std::stringstream ss;
-                    ss << "project field type is date/datetime, should use int64, but the "
-                          "definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is date/datetime, should use int64, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::Int64Writer* col_writer =
@@ -356,11 +353,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             case TYPE_OBJECT: {
                 if (_output_object_data) {
                     if (_str_schema[index][1] != "byte_array") {
-                        std::stringstream ss;
-                        ss << "project field type is hll/bitmap, should use byte_array, but the "
-                              "definition type of column "
-                           << _str_schema[index][2] << " is " << _str_schema[index][1];
-                        return Status::InvalidArgument(ss.str());
+                        return Status::InvalidArgument(
+                                "project field type is hll/bitmap, should use byte_array, "
+                                "but the definition type of column {} is {}",
+                                _str_schema[index][2], _str_schema[index][1]);
                     }
                     parquet::RowGroupWriter* rgWriter = get_rg_writer();
                     parquet::ByteArrayWriter* col_writer =
@@ -387,11 +383,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             case TYPE_VARCHAR:
             case TYPE_STRING: {
                 if (_str_schema[index][1] != "byte_array") {
-                    std::stringstream ss;
-                    ss << "project field type is char/varchar, should use byte_array, but the "
-                          "definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is char/varchar, should use byte_array, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::ByteArrayWriter* col_writer =
@@ -410,11 +405,10 @@ Status ParquetWriterWrapper::_write_one_row(TupleRow* row) {
             }
             case TYPE_DECIMALV2: {
                 if (_str_schema[index][1] != "byte_array") {
-                    std::stringstream ss;
-                    ss << "project field type is decimal v2, should use byte_array, but the "
-                          "definition type of column "
-                       << _str_schema[index][2] << " is " << _str_schema[index][1];
-                    return Status::InvalidArgument(ss.str());
+                    return Status::InvalidArgument(
+                            "project field type is decimal v2, should use byte_array, "
+                            "but the definition type of column {} is {}",
+                            _str_schema[index][2], _str_schema[index][1]);
                 }
                 parquet::RowGroupWriter* rgWriter = get_rg_writer();
                 parquet::ByteArrayWriter* col_writer =
diff --git a/be/src/exec/partitioned_hash_table.cc b/be/src/exec/partitioned_hash_table.cc
index 712f4fa4d2..ec4e12db73 100644
--- a/be/src/exec/partitioned_hash_table.cc
+++ b/be/src/exec/partitioned_hash_table.cc
@@ -102,9 +102,9 @@ Status PartitionedHashTableCtx::Init(ObjectPool* pool, RuntimeState* state, int
     scratch_row_ = reinterpret_cast<TupleRow*>(malloc(scratch_row_size));
     if (UNLIKELY(scratch_row_ == nullptr)) {
         return Status::InternalError(
-                Substitute("Failed to allocate $0 bytes for scratch row of "
-                           "PartitionedHashTableCtx.",
-                           scratch_row_size));
+                "Failed to allocate {} bytes for scratch row of "
+                "PartitionedHashTableCtx.",
+                scratch_row_size);
     }
 
     // TODO chenhao replace ExprContext with ScalarFnEvaluator
diff --git a/be/src/exec/plain_text_line_reader.cpp b/be/src/exec/plain_text_line_reader.cpp
index 825982c5b1..6d9d25841b 100644
--- a/be/src/exec/plain_text_line_reader.cpp
+++ b/be/src/exec/plain_text_line_reader.cpp
@@ -244,9 +244,8 @@ Status PlainTextLineReader::read_line(const uint8_t** ptr, size_t* size, bool* e
                 // LOG(INFO) << "after read file: _file_eof: " << _file_eof << " read_len: " << read_len;
                 if (_file_eof || read_len == 0) {
                     if (!_stream_end) {
-                        std::stringstream ss;
-                        ss << "Compressed file has been truncated, which is not allowed";
-                        return Status::InternalError(ss.str());
+                        return Status::InternalError(
+                                "Compressed file has been truncated, which is not allowed");
                     } else {
                         // last loop we meet stream end,
                         // and now we finished reading file, so we are finished
diff --git a/be/src/exec/read_write_util.h b/be/src/exec/read_write_util.h
index f160949cab..0b387e0fbb 100644
--- a/be/src/exec/read_write_util.h
+++ b/be/src/exec/read_write_util.h
@@ -187,9 +187,8 @@ inline bool ReadWriteUtil::read(uint8_t** buf, int* buf_len, T* val, Status* sta
     int val_len = sizeof(T);
 
     if (UNLIKELY(val_len > *buf_len)) {
-        std::stringstream ss;
-        ss << "Cannot read " << val_len << " bytes, buffer length is " << *buf_len;
-        *status = Status::InternalError(ss.str());
+        *status = Status::InternalError("Cannot read {} bytes, buffer length is {}", val_len,
+                                        *buf_len);
         return false;
     }
 
@@ -203,9 +202,8 @@ inline bool ReadWriteUtil::skip_bytes(uint8_t** buf, int* buf_len, int num_bytes
     DCHECK_GE(*buf_len, 0);
 
     if (UNLIKELY(num_bytes > *buf_len)) {
-        std::stringstream ss;
-        ss << "Cannot skip " << num_bytes << " bytes, buffer length is " << *buf_len;
-        *status = Status::InternalError(ss.str());
+        *status = Status::InternalError("Cannot skip {} bytes, buffer length is {}", num_bytes,
+                                        *buf_len);
         return false;
     }
 
diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index b346081b5d..a19410c5c3 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -53,9 +53,7 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) {
         for (auto& col : p_index.columns()) {
             auto it = slots_map.find(col);
             if (it == std::end(slots_map)) {
-                std::stringstream ss;
-                ss << "unknown index column, column=" << col;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("unknown index column, column={}", col);
             }
             index->slots.emplace_back(it->second);
         }
@@ -88,9 +86,7 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) {
         for (auto& col : t_index.columns) {
             auto it = slots_map.find(col);
             if (it == std::end(slots_map)) {
-                std::stringstream ss;
-                ss << "unknown index column, column=" << col;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("unknown index column, column={}", col);
             }
             index->slots.emplace_back(it->second);
         }
@@ -171,18 +167,15 @@ Status OlapTablePartitionParam::init() {
     if (_t_param.__isset.partition_column) {
         auto it = slots_map.find(_t_param.partition_column);
         if (it == std::end(slots_map)) {
-            std::stringstream ss;
-            ss << "partition column not found, column=" << _t_param.partition_column;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("partition column not found, column={}",
+                                         _t_param.partition_column);
         }
         _partition_slot_descs.push_back(it->second);
     } else if (_t_param.__isset.partition_columns) {
         for (auto& part_col : _t_param.partition_columns) {
             auto it = slots_map.find(part_col);
             if (it == std::end(slots_map)) {
-                std::stringstream ss;
-                ss << "partition column not found, column=" << part_col;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("partition column not found, column={}", part_col);
             }
             _partition_slot_descs.push_back(it->second);
         }
@@ -194,9 +187,7 @@ Status OlapTablePartitionParam::init() {
         for (auto& col : _t_param.distributed_columns) {
             auto it = slots_map.find(col);
             if (it == std::end(slots_map)) {
-                std::stringstream ss;
-                ss << "distributed column not found, columns=" << col;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("distributed column not found, columns={}", col);
             }
             _distributed_slot_descs.emplace_back(it->second);
         }
@@ -256,11 +247,10 @@ Status OlapTablePartitionParam::init() {
         part->num_buckets = t_part.num_buckets;
         auto num_indexes = _schema->indexes().size();
         if (t_part.indexes.size() != num_indexes) {
-            std::stringstream ss;
-            ss << "number of partition's index is not equal with schema's"
-               << ", num_part_indexes=" << t_part.indexes.size()
-               << ", num_schema_indexes=" << num_indexes;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "number of partition's index is not equal with schema's,"
+                    "num_part_indexes={}, num_schema_indexes={}",
+                    t_part.indexes.size(), num_indexes);
         }
         part->indexes = t_part.indexes;
         std::sort(part->indexes.begin(), part->indexes.end(),
@@ -274,7 +264,10 @@ Status OlapTablePartitionParam::init() {
                 ss << "partition's index is not equal with schema's"
                    << ", part_index=" << part->indexes[j].index_id
                    << ", schema_index=" << _schema->indexes()[j]->index_id;
-                return Status::InternalError(ss.str());
+                return Status::InternalError(
+                        "partition's index is not equal with schema's"
+                        ", part_index={}, schema_index={}",
+                        part->indexes[j].index_id, _schema->indexes()[j]->index_id);
             }
         }
         _partitions.emplace_back(part);
@@ -388,9 +381,8 @@ Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, T
         break;
     }
     default: {
-        std::stringstream ss;
-        ss << "unsupported partition column node type, type=" << t_expr.node_type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("unsupported partition column node type, type={}",
+                                     t_expr.node_type);
     }
     }
     return Status::OK();
@@ -437,7 +429,7 @@ Status VOlapTablePartitionParam::init() {
                                                const std::string& column_type) {
         auto it = std::find(slot_column_names.begin(), slot_column_names.end(), slot_name);
         if (it == slot_column_names.end()) {
-            return Status::InternalError(column_type + " column not found, column =" + slot_name);
+            return Status::InternalError("{} column not found, column ={}", column_type, slot_name);
         }
         locs.emplace_back(it - slot_column_names.begin());
         return Status::OK();
@@ -510,11 +502,10 @@ Status VOlapTablePartitionParam::init() {
         part->num_buckets = t_part.num_buckets;
         auto num_indexes = _schema->indexes().size();
         if (t_part.indexes.size() != num_indexes) {
-            std::stringstream ss;
-            ss << "number of partition's index is not equal with schema's"
-               << ", num_part_indexes=" << t_part.indexes.size()
-               << ", num_schema_indexes=" << num_indexes;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "number of partition's index is not equal with schema's"
+                    ", num_part_indexes={}, num_schema_indexes={}",
+                    t_part.indexes.size(), num_indexes);
         }
         part->indexes = t_part.indexes;
         std::sort(part->indexes.begin(), part->indexes.end(),
@@ -528,7 +519,10 @@ Status VOlapTablePartitionParam::init() {
                 ss << "partition's index is not equal with schema's"
                    << ", part_index=" << part->indexes[j].index_id
                    << ", schema_index=" << _schema->indexes()[j]->index_id;
-                return Status::InternalError(ss.str());
+                return Status::InternalError(
+                        "partition's index is not equal with schema's"
+                        ", part_index={}, schema_index={}",
+                        part->indexes[j].index_id, _schema->indexes()[j]->index_id);
             }
         }
         _partitions.emplace_back(part);
@@ -644,9 +638,8 @@ Status VOlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr,
         break;
     }
     default: {
-        std::stringstream ss;
-        ss << "unsupported partition column node type, type=" << t_expr.node_type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("unsupported partition column node type, type={}",
+                                     t_expr.node_type);
     }
     }
     part_key->second = column->size() - 1;
diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp
index fc3e976d7d..14f01f9327 100644
--- a/be/src/exec/tablet_sink.cpp
+++ b/be/src/exec/tablet_sink.cpp
@@ -76,10 +76,8 @@ Status NodeChannel::init(RuntimeState* state) {
     _state = state;
     auto node = _parent->_nodes_info->find_node(_node_id);
     if (node == nullptr) {
-        std::stringstream ss;
-        ss << "unknown node id, id=" << _node_id;
         _cancelled = true;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("unknown node id, id={}", _node_id);
     }
 
     _node_info = *node;
@@ -178,7 +176,9 @@ Status NodeChannel::open_wait() {
            << ", error_text=" << _open_closure->cntl.ErrorText();
         _cancelled = true;
         LOG(WARNING) << ss.str();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("failed to open tablet writer, error={}, error_text={}",
+                                     berror(_open_closure->cntl.ErrorCode()),
+                                     _open_closure->cntl.ErrorText());
     }
     Status status(_open_closure->result.status());
     if (_open_closure->unref()) {
@@ -266,7 +266,7 @@ Status NodeChannel::add_row(Tuple* input_tuple, int64_t tablet_id) {
     if (!st.ok()) {
         if (_cancelled) {
             std::lock_guard<SpinLock> l(_cancel_msg_lock);
-            return Status::InternalError("add row failed. " + _cancel_msg);
+            return Status::InternalError("add row failed. {}", _cancel_msg);
         } else {
             return st.clone_and_prepend("already stopped, can't add row. cancelled/eos: ");
         }
@@ -350,7 +350,7 @@ Status NodeChannel::close_wait(RuntimeState* state) {
     if (!st.ok()) {
         if (_cancelled) {
             std::lock_guard<SpinLock> l(_cancel_msg_lock);
-            return Status::InternalError("wait close failed. " + _cancel_msg);
+            return Status::InternalError("wait close failed. {}", _cancel_msg);
         } else {
             return st.clone_and_prepend(
                     "already stopped, skip waiting for close. cancelled/!eos: ");
diff --git a/be/src/exprs/agg_fn.cpp b/be/src/exprs/agg_fn.cpp
index a04ef1ba86..9e69ac8c6c 100644
--- a/be/src/exprs/agg_fn.cpp
+++ b/be/src/exprs/agg_fn.cpp
@@ -86,9 +86,7 @@ Status AggFn::init(const RowDescriptor& row_desc, RuntimeState* state) {
         (aggregate_fn.merge_fn_symbol.empty() && !aggregate_fn.is_analytic_only_fn)) {
         // This path is only for partially implemented builtins.
         DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::BUILTIN);
-        std::stringstream ss;
-        ss << "Function " << _fn.name.function_name << " is not implemented.";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Function {} is not implemented.", _fn.name.function_name);
     }
     if (_fn.binary_type == TFunctionBinaryType::NATIVE ||
         _fn.binary_type == TFunctionBinaryType::BUILTIN ||
@@ -152,7 +150,7 @@ Status AggFn::init(const RowDescriptor& row_desc, RuntimeState* state) {
                     std::make_unique<RPCFn>(state, _fn, RPCFn::AggregationStep::FINALIZE, true);
         }
     } else {
-        return Status::NotSupported(fmt::format("Not supported BinaryType: {}", _fn.binary_type));
+        return Status::NotSupported("Not supported BinaryType: {}", _fn.binary_type);
     }
     return Status::OK();
 }
diff --git a/be/src/exprs/agg_fn_evaluator.cpp b/be/src/exprs/agg_fn_evaluator.cpp
index 9c9d6f4eeb..6ef80aa1ff 100644
--- a/be/src/exprs/agg_fn_evaluator.cpp
+++ b/be/src/exprs/agg_fn_evaluator.cpp
@@ -194,9 +194,7 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M
         (!_is_analytic_fn && _fn.aggregate_fn.merge_fn_symbol.empty())) {
         // This path is only for partially implemented builtins.
         DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::BUILTIN);
-        std::stringstream ss;
-        ss << "Function " << _fn.name.function_name << " is not implemented.";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Function {} is not implemented.", _fn.name.function_name);
     }
 
     // Load the function pointers.
diff --git a/be/src/exprs/block_bloom_filter_impl.cc b/be/src/exprs/block_bloom_filter_impl.cc
index 1cdab38605..e553694c5a 100644
--- a/be/src/exprs/block_bloom_filter_impl.cc
+++ b/be/src/exprs/block_bloom_filter_impl.cc
@@ -60,8 +60,8 @@ Status BlockBloomFilter::init_internal(const int log_space_bytes, uint32_t hash_
     // Since we use 32 bits in the arguments of Insert() and Find(), _log_num_buckets
     // must be limited.
     if (_log_num_buckets > 32) {
-        return Status::InvalidArgument(
-                fmt::format("Bloom filter too large. log_space_bytes: {}", log_space_bytes));
+        return Status::InvalidArgument("Bloom filter too large. log_space_bytes: {}",
+                                       log_space_bytes);
     }
     // Don't use _log_num_buckets if it will lead to undefined behavior by a shift
     // that is too large.
@@ -93,9 +93,9 @@ Status BlockBloomFilter::init_from_directory(int log_space_bytes, const Slice& d
     DCHECK(_directory);
 
     if (directory_size() != directory.size) {
-        return Status::InvalidArgument(fmt::format(
+        return Status::InvalidArgument(
                 "Mismatch in BlockBloomFilter source directory size {} and expected size {}",
-                directory.size, directory_size()));
+                directory.size, directory_size());
     }
     memcpy(_directory, directory.data, directory.size);
     _always_false = always_false;
@@ -183,7 +183,7 @@ void BlockBloomFilter::or_equal_array_internal(size_t n, const uint8_t* __restri
 Status BlockBloomFilter::or_equal_array(size_t n, const uint8_t* __restrict__ in,
                                         uint8_t* __restrict__ out) {
     if ((n % kBucketByteSize) != 0) {
-        return Status::InvalidArgument(fmt::format("Input size {} not a multiple of 32-bytes", n));
+        return Status::InvalidArgument("Input size {} not a multiple of 32-bytes", n);
     }
 
     or_equal_array_internal(n, in, out);
@@ -232,9 +232,8 @@ Status BlockBloomFilter::merge(const BlockBloomFilter& other) {
         return Status::OK();
     }
     if (directory_size() != other.directory_size()) {
-        return Status::InvalidArgument(
-                fmt::format("Directory size don't match. this: {}, other: {}", directory_size(),
-                            other.directory_size()));
+        return Status::InvalidArgument("Directory size don't match. this: {}, other: {}",
+                                       directory_size(), other.directory_size());
     }
     if (other.always_false()) {
         // Nothing to do.
diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp
index 4325e5cecb..37f8bc4c31 100644
--- a/be/src/exprs/expr.cpp
+++ b/be/src/exprs/expr.cpp
@@ -410,9 +410,7 @@ Status Expr::create_expr(ObjectPool* pool, const TExprNode& texpr_node, Expr** e
     }
 
     default:
-        std::stringstream os;
-        os << "Unknown expr node type: " << texpr_node.node_type;
-        return Status::InternalError(os.str());
+        return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
     }
 }
 
diff --git a/be/src/exprs/rpc_fn.cpp b/be/src/exprs/rpc_fn.cpp
index 797968b775..17f7d45ab0 100644
--- a/be/src/exprs/rpc_fn.cpp
+++ b/be/src/exprs/rpc_fn.cpp
@@ -749,17 +749,16 @@ Status RPCFn::vec_call(FunctionContext* context, vectorized::Block& block,
     brpc::Controller cntl;
     _client->fn_call(&cntl, &request, &response, nullptr);
     if (cntl.Failed()) {
-        return Status::InternalError(
-                fmt::format("call to rpc function {} failed: {}", _signature, cntl.ErrorText())
-                        .c_str());
+        return Status::InternalError("call to rpc function {} failed: {}", _signature,
+                                     cntl.ErrorText());
     }
     if (!response.has_status() || response.result_size() == 0) {
-        return Status::InternalError(fmt::format(
-                "call rpc function {} failed: status or result is not set.", _signature));
+        return Status::InternalError("call rpc function {} failed: status or result is not set.",
+                                     _signature);
     }
     if (response.status().status_code() != 0) {
-        return Status::InternalError(fmt::format("call to rpc function {} failed: {}", _signature,
-                                                 response.status().DebugString()));
+        return Status::InternalError("call to rpc function {} failed: {}", _signature,
+                                     response.status().DebugString());
     }
     convert_to_block(block, response.result(0), result);
     return Status::OK();
diff --git a/be/src/exprs/rpc_fn_call.cpp b/be/src/exprs/rpc_fn_call.cpp
index 05074f3f1c..1af4365b02 100644
--- a/be/src/exprs/rpc_fn_call.cpp
+++ b/be/src/exprs/rpc_fn_call.cpp
@@ -46,8 +46,8 @@ Status RPCFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, ExprCo
 
     _rpc_fn = std::make_unique<RPCFn>(state, _fn, id, false);
     if (!_rpc_fn->avliable()) {
-        return Status::InternalError(
-                fmt::format("rpc env init error: {}/{}", _fn.hdfs_location, _fn.scalar_fn.symbol));
+        return Status::InternalError("rpc env init error: {}/{}", _fn.hdfs_location,
+                                     _fn.scalar_fn.symbol);
     }
     return Status::OK();
 }
diff --git a/be/src/exprs/scalar_fn_call.cpp b/be/src/exprs/scalar_fn_call.cpp
index 302bcad0c2..07f62a81fc 100644
--- a/be/src/exprs/scalar_fn_call.cpp
+++ b/be/src/exprs/scalar_fn_call.cpp
@@ -51,9 +51,7 @@ Status ScalarFnCall::prepare(RuntimeState* state, const RowDescriptor& desc, Exp
         // Having the failure in the BE (rather than during analysis) allows for
         // better FE testing.
         DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::BUILTIN);
-        std::stringstream ss;
-        ss << "Function " << _fn.name.function_name << " is not implemented.";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Function {} is not implemented.", _fn.name.function_name);
     }
 
     FunctionContext::TypeDesc return_type = AnyValUtil::column_type_to_type_desc(_type);
diff --git a/be/src/exprs/slot_ref.cpp b/be/src/exprs/slot_ref.cpp
index f4c28dcbd8..59e33c7254 100644
--- a/be/src/exprs/slot_ref.cpp
+++ b/be/src/exprs/slot_ref.cpp
@@ -60,15 +60,12 @@ SlotRef::SlotRef(const TypeDescriptor& type, int offset)
 
 Status SlotRef::prepare(const SlotDescriptor* slot_desc, const RowDescriptor& row_desc) {
     if (!slot_desc->is_materialized()) {
-        std::stringstream error;
-        error << "reference to non-materialized slot. slot_id: " << _slot_id;
-        return Status::InternalError(error.str());
+        return Status::InternalError("reference to non-materialized slot. slot_id: {}", _slot_id);
     }
     _tuple_idx = row_desc.get_tuple_idx(slot_desc->parent());
     if (_tuple_idx == RowDescriptor::INVALID_IDX) {
-        return Status::InternalError(
-                strings::Substitute("failed to get tuple idx with tuple id: $0, slot id: $1",
-                                    slot_desc->parent(), _slot_id));
+        return Status::InternalError("failed to get tuple idx with tuple id: {}, slot id: {}",
+                                     slot_desc->parent(), _slot_id);
     }
     _tuple_is_nullable = row_desc.tuple_is_nullable(_tuple_idx);
     _slot_offset = slot_desc->tuple_offset();
@@ -86,23 +83,19 @@ Status SlotRef::prepare(RuntimeState* state, const RowDescriptor& row_desc, Expr
     const SlotDescriptor* slot_desc = state->desc_tbl().get_slot_descriptor(_slot_id);
     if (slot_desc == nullptr) {
         // TODO: create macro MAKE_ERROR() that returns a stream
-        std::stringstream error;
-        error << "couldn't resolve slot descriptor " << _slot_id;
-        return Status::InternalError(error.str());
+        return Status::InternalError("couldn't resolve slot descriptor {}", _slot_id);
     }
 
     if (!slot_desc->is_materialized()) {
-        std::stringstream error;
-        error << "reference to non-materialized slot. slot_id: " << _slot_id;
-        return Status::InternalError(error.str());
+        return Status::InternalError("reference to non-materialized slot. slot_id: {}", _slot_id);
     }
 
     // TODO(marcel): get from runtime state
     _tuple_idx = row_desc.get_tuple_idx(slot_desc->parent());
     if (_tuple_idx == RowDescriptor::INVALID_IDX) {
-        return Status::InternalError(strings::Substitute(
-                "failed to get tuple idx when prepare with tuple id: $0, slot id: $1",
-                slot_desc->parent(), _slot_id));
+        return Status::InternalError(
+                "failed to get tuple idx when prepare with tuple id: {}, slot id: {}",
+                slot_desc->parent(), _slot_id);
     }
     DCHECK(_tuple_idx != RowDescriptor::INVALID_IDX);
     _tuple_is_nullable = row_desc.tuple_is_nullable(_tuple_idx);
diff --git a/be/src/exprs/table_function/table_function.h b/be/src/exprs/table_function/table_function.h
index 901572996b..93e6220383 100644
--- a/be/src/exprs/table_function/table_function.h
+++ b/be/src/exprs/table_function/table_function.h
@@ -43,25 +43,22 @@ public:
     virtual Status open() { return Status::OK(); }
 
     virtual Status process(TupleRow* tuple_row) {
-        return Status::NotSupported(fmt::format("table function {} not supported now.", _fn_name));
+        return Status::NotSupported("table function {} not supported now.", _fn_name);
     }
 
     // only used for vectorized.
     virtual Status process_init(vectorized::Block* block) {
-        return Status::NotSupported(
-                fmt::format("vectorized table function {} not supported now.", _fn_name));
+        return Status::NotSupported("vectorized table function {} not supported now.", _fn_name);
     }
 
     // only used for vectorized.
     virtual Status process_row(size_t row_idx) {
-        return Status::NotSupported(
-                fmt::format("vectorized table function {} not supported now.", _fn_name));
+        return Status::NotSupported("vectorized table function {} not supported now.", _fn_name);
     }
 
     // only used for vectorized.
     virtual Status process_close() {
-        return Status::NotSupported(
-                fmt::format("vectorized table function {} not supported now.", _fn_name));
+        return Status::NotSupported("vectorized table function {} not supported now.", _fn_name);
     }
 
     virtual Status reset() = 0;
diff --git a/be/src/exprs/table_function/table_function_factory.cpp b/be/src/exprs/table_function/table_function_factory.cpp
index 2415627ccd..7b14e011dd 100644
--- a/be/src/exprs/table_function/table_function_factory.cpp
+++ b/be/src/exprs/table_function/table_function_factory.cpp
@@ -101,8 +101,8 @@ Status TableFunctionFactory::get_fn(const std::string& fn_name_raw, bool is_vect
         return Status::OK();
     }
 
-    return Status::NotSupported(std::string(is_vectorized ? "vectorized " : "") +
-                                "table function " + fn_name_raw + " not support");
+    return Status::NotSupported("{}table function {} not support",
+                                std::string(is_vectorized ? "vectorized " : ""), fn_name_raw);
 }
 
 } // namespace doris
diff --git a/be/src/http/action/compaction_action.cpp b/be/src/http/action/compaction_action.cpp
index 87fbae2d8b..1e9ab8dde3 100644
--- a/be/src/http/action/compaction_action.cpp
+++ b/be/src/http/action/compaction_action.cpp
@@ -52,7 +52,7 @@ Status CompactionAction::_check_param(HttpRequest* req, uint64_t* tablet_id) {
     try {
         *tablet_id = std::stoull(req_tablet_id);
     } catch (const std::exception& e) {
-        return Status::InternalError(strings::Substitute("convert tablet_id failed, $0", e.what()));
+        return Status::InternalError("convert tablet_id failed, {}", e.what());
     }
 
     return Status::OK();
@@ -65,7 +65,7 @@ Status CompactionAction::_handle_show_compaction(HttpRequest* req, std::string*
 
     TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
     if (tablet == nullptr) {
-        return Status::NotFound(strings::Substitute("Tablet not found. tablet_id=$0", tablet_id));
+        return Status::NotFound("Tablet not found. tablet_id={}", tablet_id);
     }
 
     tablet->get_compaction_status(json_result);
@@ -82,14 +82,13 @@ Status CompactionAction::_handle_run_compaction(HttpRequest* req, std::string* j
     std::string compaction_type = req->param(PARAM_COMPACTION_TYPE);
     if (compaction_type != PARAM_COMPACTION_BASE &&
         compaction_type != PARAM_COMPACTION_CUMULATIVE) {
-        return Status::NotSupported(
-                strings::Substitute("The compaction type '$0' is not supported", compaction_type));
+        return Status::NotSupported("The compaction type '{}' is not supported", compaction_type);
     }
 
     // 2. fetch the tablet by tablet_id
     TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
     if (tablet == nullptr) {
-        return Status::NotFound(strings::Substitute("Tablet not found. tablet_id=$0", tablet_id));
+        return Status::NotFound("Tablet not found. tablet_id={}", tablet_id);
     }
 
     // 3. execute compaction task
@@ -145,7 +144,7 @@ Status CompactionAction::_handle_run_status_compaction(HttpRequest* req, std::st
         TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
         if (tablet == nullptr) {
             LOG(WARNING) << "invalid argument.tablet_id:" << tablet_id;
-            return Status::InternalError(strings::Substitute("fail to get $0", tablet_id));
+            return Status::InternalError("fail to get {}", tablet_id);
         }
 
         std::string json_template = R"({
diff --git a/be/src/http/action/download_action.cpp b/be/src/http/action/download_action.cpp
index a3816a6217..10cd6c1998 100644
--- a/be/src/http/action/download_action.cpp
+++ b/be/src/http/action/download_action.cpp
@@ -145,7 +145,7 @@ Status DownloadAction::check_path_is_allowed(const std::string& file_path) {
 
     std::string canonical_file_path;
     RETURN_WITH_WARN_IF_ERROR(FileUtils::canonicalize(file_path, &canonical_file_path),
-                              Status::InternalError("file path is invalid: " + file_path),
+                              Status::InternalError("file path is invalid: {}", file_path),
                               "file path is invalid: " + file_path);
 
     for (auto& allow_path : _allow_paths) {
@@ -154,7 +154,7 @@ Status DownloadAction::check_path_is_allowed(const std::string& file_path) {
         }
     }
 
-    return Status::InternalError("file path is not allowed: " + canonical_file_path);
+    return Status::InternalError("file path is not allowed: {}", canonical_file_path);
 }
 
 Status DownloadAction::check_log_path_is_allowed(const std::string& file_path) {
@@ -162,14 +162,14 @@ Status DownloadAction::check_log_path_is_allowed(const std::string& file_path) {
 
     std::string canonical_file_path;
     RETURN_WITH_WARN_IF_ERROR(FileUtils::canonicalize(file_path, &canonical_file_path),
-                              Status::InternalError("file path is invalid: " + file_path),
+                              Status::InternalError("file path is invalid: {}", file_path),
                               "file path is invalid: " + file_path);
 
     if (FileSystemUtil::contain_path(_error_log_root_dir, canonical_file_path)) {
         return Status::OK();
     }
 
-    return Status::InternalError("file path is not allowed: " + file_path);
+    return Status::InternalError("file path is not allowed: {}", file_path);
 }
 
 } // end namespace doris
diff --git a/be/src/http/action/meta_action.cpp b/be/src/http/action/meta_action.cpp
index 4855ef7cf6..7531dd6a93 100644
--- a/be/src/http/action/meta_action.cpp
+++ b/be/src/http/action/meta_action.cpp
@@ -52,7 +52,7 @@ Status MetaAction::_handle_header(HttpRequest* req, std::string* json_meta) {
     } catch (const std::exception& e) {
         LOG(WARNING) << "invalid argument.tablet_id:" << req_tablet_id
                      << ", enable_byte_to_base64: " << req_enable_base64;
-        return Status::InternalError(strings::Substitute("convert failed, $0", e.what()));
+        return Status::InternalError("convert failed, {}", e.what());
     }
 
     TabletSharedPtr tablet = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id);
diff --git a/be/src/http/action/stream_load.cpp b/be/src/http/action/stream_load.cpp
index 28bfafedf6..6f59403070 100644
--- a/be/src/http/action/stream_load.cpp
+++ b/be/src/http/action/stream_load.cpp
@@ -280,9 +280,8 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ct
     }
     ctx->format = parse_format(format_str, http_req->header(HTTP_COMPRESS_TYPE));
     if (ctx->format == TFileFormatType::FORMAT_UNKNOWN) {
-        std::stringstream ss;
-        ss << "unknown data format, format=" << http_req->header(HTTP_FORMAT_KEY);
-        return Status::InternalError(ss.str());
+        return Status::InternalError("unknown data format, format={}",
+                                     http_req->header(HTTP_FORMAT_KEY));
     }
 
     if (ctx->two_phase_commit && config::disable_stream_load_2pc) {
@@ -304,18 +303,16 @@ Status StreamLoadAction::_on_header(HttpRequest* http_req, StreamLoadContext* ct
         // json max body size
         if ((ctx->format == TFileFormatType::FORMAT_JSON) &&
             (ctx->body_bytes > json_max_body_bytes) && !read_json_by_line) {
-            std::stringstream ss;
-            ss << "The size of this batch exceed the max size [" << json_max_body_bytes
-               << "]  of json type data "
-               << " data [ " << ctx->body_bytes << " ]. Split the file, or use 'read_json_by_line'";
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "The size of this batch exceed the max size [{}]  of json type data "
+                    " data [ {} ]. Split the file, or use 'read_json_by_line'",
+                    json_max_body_bytes, ctx->body_bytes);
         }
         // csv max body size
         else if (ctx->body_bytes > csv_max_body_bytes) {
             LOG(WARNING) << "body exceed max size." << ctx->brief();
-            std::stringstream ss;
-            ss << "body exceed max size: " << csv_max_body_bytes << ", data: " << ctx->body_bytes;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("body exceed max size: {}, data: {}", csv_max_body_bytes,
+                                         ctx->body_bytes);
         }
     } else {
 #ifndef BE_TEST
@@ -541,7 +538,7 @@ Status StreamLoadAction::_process_put(HttpRequest* http_req, StreamLoadContext*
         if (merge_type_map.find(merge_type_str) != merge_type_map.end()) {
             merge_type = merge_type_map.find(merge_type_str)->second;
         } else {
-            return Status::InvalidArgument("Invalid merge type " + merge_type_str);
+            return Status::InvalidArgument("Invalid merge type {}", merge_type_str);
         }
         if (merge_type == TMergeType::MERGE && http_req->header(HTTP_DELETE_CONDITION).empty()) {
             return Status::InvalidArgument("Excepted DELETE ON clause when merge type is MERGE.");
diff --git a/be/src/http/action/stream_load_2pc.cpp b/be/src/http/action/stream_load_2pc.cpp
index 7aa5655e52..914d1e97fb 100644
--- a/be/src/http/action/stream_load_2pc.cpp
+++ b/be/src/http/action/stream_load_2pc.cpp
@@ -55,7 +55,7 @@ void StreamLoad2PCAction::handle(HttpRequest* req) {
     try {
         ctx->txn_id = std::stoull(req_txn_id);
     } catch (const std::exception& e) {
-        status = Status::InternalError("convert txn_id [" + req_txn_id + "] failed");
+        status = Status::InternalError("convert txn_id [{}] failed", req_txn_id);
         status_result = status.to_json();
         HttpChannel::send_reply(req, HttpStatus::OK, status_result);
         return;
diff --git a/be/src/http/action/tablet_migration_action.cpp b/be/src/http/action/tablet_migration_action.cpp
index 5f7a9e218e..0ac59a80d7 100644
--- a/be/src/http/action/tablet_migration_action.cpp
+++ b/be/src/http/action/tablet_migration_action.cpp
@@ -66,10 +66,10 @@ void TabletMigrationAction::handle(HttpRequest* req) {
                         std::map<MigrationTask, std::string>::iterator it_task =
                                 _migration_tasks.find(current_task);
                         if (it_task != _migration_tasks.end()) {
-                            status = Status::AlreadyExist(strings::Substitute(
+                            status = Status::AlreadyExist(
                                     "There is a migration task for this tablet already exists. "
-                                    "dest_disk is $0 .",
-                                    (it_task->first)._dest_disk));
+                                    "dest_disk is {} .",
+                                    (it_task->first)._dest_disk);
                             break;
                         }
                         _migration_tasks[current_task] = "submitted";
@@ -174,7 +174,7 @@ Status TabletMigrationAction::_check_param(HttpRequest* req, int64_t& tablet_id,
     } catch (const std::exception& e) {
         LOG(WARNING) << "invalid argument.tablet_id:" << req_tablet_id
                      << ", schema_hash:" << req_schema_hash;
-        return Status::InternalError(strings::Substitute("Convert failed, $0", e.what()));
+        return Status::InternalError("Convert failed, {}", e.what());
     }
     dest_disk = req->param("disk");
     goal = req->param("goal");
diff --git a/be/src/http/action/tablets_distribution_action.cpp b/be/src/http/action/tablets_distribution_action.cpp
index 30254665ea..6bb7b5ab6f 100644
--- a/be/src/http/action/tablets_distribution_action.cpp
+++ b/be/src/http/action/tablets_distribution_action.cpp
@@ -50,8 +50,7 @@ void TabletsDistributionAction::handle(HttpRequest* req) {
                 partition_id = std::stoull(req_partition_id);
             } catch (const std::exception& e) {
                 LOG(WARNING) << "invalid argument. partition_id:" << req_partition_id;
-                Status status = Status::InternalError(
-                        strings::Substitute("invalid argument: partition_id"));
+                Status status = Status::InternalError("invalid argument: {}", req_partition_id);
                 std::string status_result = status.to_json();
                 HttpChannel::send_reply(req, HttpStatus::INTERNAL_SERVER_ERROR, status_result);
                 return;
diff --git a/be/src/http/ev_http_server.cpp b/be/src/http/ev_http_server.cpp
index 69041efb70..2d9031383a 100644
--- a/be/src/http/ev_http_server.cpp
+++ b/be/src/http/ev_http_server.cpp
@@ -143,9 +143,7 @@ Status EvHttpServer::_bind() {
     butil::EndPoint point;
     auto res = butil::hostname2endpoint(_host.c_str(), _port, &point);
     if (res < 0) {
-        std::stringstream ss;
-        ss << "convert address failed, host=" << _host << ", port=" << _port;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("convert address failed, host={}, port={}", _host, _port);
     }
     _server_fd = butil::tcp_listen(point);
     if (_server_fd < 0) {
diff --git a/be/src/http/http_client.h b/be/src/http/http_client.h
index 824230bfc3..b9605afb0f 100644
--- a/be/src/http/http_client.h
+++ b/be/src/http/http_client.h
@@ -106,8 +106,7 @@ public:
             *length = cl;
             return Status::OK();
         }
-        return Status::InternalError(
-                fmt::format("failed to get content length. err code: {}", code));
+        return Status::InternalError("failed to get content length. err code: {}", code);
     }
 
     long get_http_status() const {
diff --git a/be/src/io/buffered_reader.cpp b/be/src/io/buffered_reader.cpp
index 453ed4b97b..f00c07f59f 100644
--- a/be/src/io/buffered_reader.cpp
+++ b/be/src/io/buffered_reader.cpp
@@ -48,9 +48,7 @@ BufferedReader::~BufferedReader() {
 
 Status BufferedReader::open() {
     if (!_reader) {
-        std::stringstream ss;
-        ss << "Open buffered reader failed, reader is null";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Open buffered reader failed, reader is null");
     }
 
     // the macro ADD_XXX is idempotent.
diff --git a/be/src/io/hdfs_file_reader.cpp b/be/src/io/hdfs_file_reader.cpp
index e993ec5b17..0112e880aa 100644
--- a/be/src/io/hdfs_file_reader.cpp
+++ b/be/src/io/hdfs_file_reader.cpp
@@ -63,10 +63,8 @@ Status HdfsFileReader::connect() {
     }
     _hdfs_fs = hdfsBuilderConnect(_builder.get());
     if (_hdfs_fs == nullptr) {
-        std::stringstream ss;
-        ss << "connect to hdfs failed. namenode address:" << _namenode
-           << ", error: " << hdfsGetLastError();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("connect to hdfs failed. namenode address:{}, error: {}",
+                                     _namenode, hdfsGetLastError());
     }
     return Status::OK();
 }
@@ -88,11 +86,9 @@ Status HdfsFileReader::open() {
     RETURN_IF_ERROR(connect());
     _hdfs_file = hdfsOpenFile(_hdfs_fs, _path.c_str(), O_RDONLY, 0, 0, 0);
     if (_hdfs_file == nullptr) {
-        std::stringstream ss;
-        ss << "open file failed. "
-           << "(BE: " << BackendOptions::get_localhost() << ")"
-           << " namenode:" << _namenode << ", path:" << _path << ", err: " << hdfsGetLastError();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("open file failed. (BE: {}) namenode:{}, path:{}, err: {}",
+                                     BackendOptions::get_localhost(), _namenode, _path,
+                                     hdfsGetLastError());
     }
     VLOG_NOTICE << "open file, namenode:" << _namenode << ", path:" << _path;
     return seek(_current_offset);
@@ -147,23 +143,17 @@ Status HdfsFileReader::readat(int64_t position, int64_t nbytes, int64_t* bytes_r
     if (position != _current_offset) {
         int ret = hdfsSeek(_hdfs_fs, _hdfs_file, position);
         if (ret != 0) { // check fseek return value
-            std::stringstream ss;
-            ss << "hdfsSeek failed. "
-               << "(BE: " << BackendOptions::get_localhost() << ")" << _namenode << _path
-               << ", err: " << hdfsGetLastError();
-            ;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("hdfsSeek failed.(BE: {}) namenode:{}, path:{}, err: {}",
+                                         BackendOptions::get_localhost(), _namenode, _path,
+                                         hdfsGetLastError());
         }
     }
 
     *bytes_read = hdfsRead(_hdfs_fs, _hdfs_file, out, nbytes);
     if (*bytes_read < 0) {
-        std::stringstream ss;
-        ss << "Read hdfs file failed. "
-           << "(BE: " << BackendOptions::get_localhost() << ")" << _namenode << _path
-           << ", err: " << hdfsGetLastError();
-        ;
-        return Status::InternalError(ss.str());
+        return Status::InternalError(
+                "Read hdfs file failed. (BE: {}) namenode:{}, path:{}, err: {}",
+                BackendOptions::get_localhost(), _namenode, _path, hdfsGetLastError());
     }
     _current_offset += *bytes_read; // save offset with file
     return Status::OK();
@@ -198,11 +188,8 @@ int64_t HdfsFileReader::size() {
 Status HdfsFileReader::seek(int64_t position) {
     int res = hdfsSeek(_hdfs_fs, _hdfs_file, position);
     if (res != 0) {
-        std::stringstream ss;
-        ss << "Seek to offset failed. "
-           << "(BE: " << BackendOptions::get_localhost() << ")"
-           << " offset=" << position << ", err: " << hdfsGetLastError();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Seek to offset failed. (BE: {}) offset={}, err: {}",
+                                     BackendOptions::get_localhost(), position, hdfsGetLastError());
     }
     return Status::OK();
 }
diff --git a/be/src/io/hdfs_writer.cpp b/be/src/io/hdfs_writer.cpp
index 10b7955cbe..51c9fed7d7 100644
--- a/be/src/io/hdfs_writer.cpp
+++ b/be/src/io/hdfs_writer.cpp
@@ -53,7 +53,7 @@ Status HDFSWriter::open() {
     int exists = hdfsExists(_hdfs_fs, _path.c_str());
     if (exists == 0) {
         // the path already exists
-        return Status::AlreadyExist(_path + " already exists.");
+        return Status::AlreadyExist("{} already exists.", _path);
     }
 
     std::filesystem::path hdfs_path(_path);
@@ -141,10 +141,8 @@ Status HDFSWriter::_connect() {
     }
     _hdfs_fs = hdfsBuilderConnect(_builder.get());
     if (_hdfs_fs == nullptr) {
-        std::stringstream ss;
-        ss << "connect to hdfs failed. namenode address:" << _namenode << ", error"
-           << hdfsGetLastError();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("connect to hdfs failed. namenode address:{}, error {}",
+                                     _namenode, hdfsGetLastError());
     }
     return Status::OK();
 }
diff --git a/be/src/io/local_file_writer.cpp b/be/src/io/local_file_writer.cpp
index e73dc96ebe..b6c6dfcefa 100644
--- a/be/src/io/local_file_writer.cpp
+++ b/be/src/io/local_file_writer.cpp
@@ -35,19 +35,16 @@ Status LocalFileWriter::open() {
 
     _fp = fopen(_path.c_str(), "w+");
     if (_fp == nullptr) {
-        std::stringstream ss;
-        ss << "Open file failed. path=" << _path << ", errno= " << errno
-           << ", description=" << get_str_err_msg();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Open file failed. path={}, errno= {}, description={}", _path,
+                                     errno, get_str_err_msg());
     }
 
     if (_start_offset != 0) {
         int success = fseek(_fp, _start_offset, SEEK_SET);
         if (success != 0) {
-            std::stringstream ss;
-            ss << "Seek to start_offset failed. offset=" << _start_offset << ", errno= " << errno
-               << ", description=" << get_str_err_msg();
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "Seek to start_offset failed. offset={}, errno= {}, description={}",
+                    _start_offset, errno, get_str_err_msg());
         }
     }
 
@@ -57,11 +54,9 @@ Status LocalFileWriter::open() {
 Status LocalFileWriter::write(const uint8_t* buf, size_t buf_len, size_t* written_len) {
     size_t bytes_written = fwrite(buf, 1, buf_len, _fp);
     if (bytes_written < buf_len) {
-        std::stringstream error_msg;
-        error_msg << "fail to write to file. "
-                  << " len=" << buf_len << ", path=" << _path << ", failed with errno=" << errno
-                  << ", description=" << get_str_err_msg();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError(
+                "fail to write to file. len={}, path={}, failed with errno={}, description={}",
+                buf_len, _path, errno, get_str_err_msg());
     }
 
     *written_len = bytes_written;
@@ -83,8 +78,8 @@ Status LocalFileWriter::_check_file_path(const std::string& file_path) {
     // Doris is not responsible for ensuring the correctness of the path.
     // This is just to prevent overwriting the existing file.
     if (FileUtils::check_exist(file_path)) {
-        return Status::InternalError("File already exists: " + file_path +
-                                     ". Host: " + BackendOptions::get_localhost());
+        return Status::InternalError("File already exists: {}. Host: {}", file_path,
+                                     BackendOptions::get_localhost());
     }
 
     return Status::OK();
diff --git a/be/src/io/s3_reader.cpp b/be/src/io/s3_reader.cpp
index a1c6b8cd2c..62da0f8e23 100644
--- a/be/src/io/s3_reader.cpp
+++ b/be/src/io/s3_reader.cpp
@@ -52,7 +52,7 @@ S3Reader::~S3Reader() {}
 Status S3Reader::open() {
     CHECK_S3_CLIENT(_client);
     if (!_uri.parse()) {
-        return Status::InvalidArgument("s3 uri is invalid: " + _path);
+        return Status::InvalidArgument("s3 uri is invalid: {}", _path);
     }
     Aws::S3::Model::HeadObjectRequest request;
     request.WithBucket(_uri.get_bucket()).WithKey(_uri.get_key());
@@ -61,12 +61,11 @@ Status S3Reader::open() {
         _file_size = response.GetResult().GetContentLength();
         return Status::OK();
     } else if (response.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) {
-        return Status::NotFound(_path + " not exists!");
+        return Status::NotFound("{} not exists!", _path);
     } else {
-        std::stringstream out;
-        out << "Error: [" << response.GetError().GetExceptionName() << ":"
-            << response.GetError().GetMessage() << "] at " << BackendOptions::get_localhost();
-        return Status::InternalError(out.str());
+        return Status::InternalError("Error: [{}:{}] at {}", response.GetError().GetExceptionName(),
+                                     response.GetError().GetMessage(),
+                                     BackendOptions::get_localhost());
     }
 }
 
diff --git a/be/src/io/s3_writer.cpp b/be/src/io/s3_writer.cpp
index c7cf4abe19..ac9bbf997e 100644
--- a/be/src/io/s3_writer.cpp
+++ b/be/src/io/s3_writer.cpp
@@ -61,20 +61,19 @@ S3Writer::~S3Writer() {
 Status S3Writer::open() {
     CHECK_S3_CLIENT(_client);
     if (!_uri.parse()) {
-        return Status::InvalidArgument("s3 uri is invalid: " + _path);
+        return Status::InvalidArgument("s3 uri is invalid: {}", _path);
     }
     Aws::S3::Model::HeadObjectRequest request;
     request.WithBucket(_uri.get_bucket()).WithKey(_uri.get_key());
     Aws::S3::Model::HeadObjectOutcome response = _client->HeadObject(request);
     if (response.IsSuccess()) {
-        return Status::AlreadyExist(_path + " already exists.");
+        return Status::AlreadyExist("{} already exists.", _path);
     } else if (response.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) {
         return Status::OK();
     } else {
-        std::stringstream out;
-        out << "Error: [" << response.GetError().GetExceptionName() << ":"
-            << response.GetError().GetMessage() << "] at " << BackendOptions::get_localhost();
-        return Status::InternalError(out.str());
+        return Status::InternalError("Error: [{}:{}] at {}", response.GetError().GetExceptionName(),
+                                     response.GetError().GetMessage(),
+                                     BackendOptions::get_localhost());
     }
 }
 
@@ -86,16 +85,16 @@ Status S3Writer::write(const uint8_t* buf, size_t buf_len, size_t* written_len)
     if (!_temp_file) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
                 Status::BufferAllocFailed(
-                        fmt::format("The internal temporary file is not writable for {}. at {}",
-                                    strerror(errno), BackendOptions::get_localhost())),
+                        "The internal temporary file is not writable for {}. at {}",
+                        strerror(errno), BackendOptions::get_localhost()),
                 "write temp file error");
     }
     _temp_file->write(reinterpret_cast<const char*>(buf), buf_len);
     if (!_temp_file->good()) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
                 Status::BufferAllocFailed(
-                        fmt::format("Could not append to the internal temporary file for {}. at {}",
-                                    strerror(errno), BackendOptions::get_localhost())),
+                        "Could not append to the internal temporary file for {}. at {}",
+                        strerror(errno), BackendOptions::get_localhost()),
                 "write temp file error");
     }
     *written_len = buf_len;
@@ -114,8 +113,8 @@ Status S3Writer::_sync() {
     if (!_temp_file) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
                 Status::BufferAllocFailed(
-                        fmt::format("The internal temporary file is not writable for {}. at {}",
-                                    strerror(errno), BackendOptions::get_localhost())),
+                        "The internal temporary file is not writable for {}. at {}",
+                        strerror(errno), BackendOptions::get_localhost()),
                 "write temp file error");
     }
     CHECK_S3_CLIENT(_client);
@@ -131,10 +130,9 @@ Status S3Writer::_sync() {
     if (response.IsSuccess()) {
         return Status::OK();
     } else {
-        std::stringstream out;
-        out << "Error: [" << response.GetError().GetExceptionName() << ":"
-            << response.GetError().GetMessage() << "] at " << BackendOptions::get_localhost();
-        return Status::InternalError(out.str());
+        return Status::InternalError("Error: [{}:{}] at {}", response.GetError().GetExceptionName(),
+                                     response.GetError().GetMessage(),
+                                     BackendOptions::get_localhost());
     }
 }
 } // end namespace doris
diff --git a/be/src/olap/column_vector.cpp b/be/src/olap/column_vector.cpp
index 14817c85cc..15957ffd2c 100644
--- a/be/src/olap/column_vector.cpp
+++ b/be/src/olap/column_vector.cpp
@@ -127,7 +127,7 @@ Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, const T
                                                                                 is_nullable));
             break;
         default:
-            return Status::NotSupported("unsupported type for ColumnVectorBatch: " +
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: {}",
                                         std::to_string(type_info->type()));
         }
         RETURN_IF_ERROR(local->resize(init_capacity));
@@ -161,7 +161,7 @@ Status ColumnVectorBatch::create(size_t init_capacity, bool is_nullable, const T
             return Status::OK();
         }
         default:
-            return Status::NotSupported("unsupported type for ColumnVectorBatch: " +
+            return Status::NotSupported("unsupported type for ColumnVectorBatch: {}",
                                         std::to_string(type_info->type()));
         }
     }
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index 13f4648690..e3bf8f4ed5 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -96,9 +96,9 @@ DataDir::~DataDir() {
 
 Status DataDir::init() {
     if (!Env::Default()->path_exists(_path_desc.filepath).ok()) {
-        RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(strings::Substitute(
-                                               "opendir failed, path=$0", _path_desc.filepath)),
-                                       "check file exist failed");
+        RETURN_NOT_OK_STATUS_WITH_WARN(
+                Status::IOError("opendir failed, path={}", _path_desc.filepath),
+                "check file exist failed");
     }
 
     RETURN_NOT_OK_STATUS_WITH_WARN(update_capacity(), "update_capacity failed");
@@ -163,9 +163,9 @@ Status DataDir::_init_capacity() {
         _capacity_bytes = disk_capacity;
     } else if (_capacity_bytes > disk_capacity) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
-                Status::InvalidArgument(strings::Substitute(
-                        "root path $0's capacity $1 should not larger than disk capacity $2",
-                        _path_desc.filepath, _capacity_bytes, disk_capacity)),
+                Status::InvalidArgument(
+                        "root path {}'s capacity {} should not larger than disk capacity {}",
+                        _path_desc.filepath, _capacity_bytes, disk_capacity),
                 "init capacity failed");
     }
 
@@ -173,9 +173,9 @@ Status DataDir::_init_capacity() {
     Status exist_status = Env::Default()->path_exists(data_path);
     if (!exist_status.ok() &&
         (!exist_status.is_not_found() || !Env::Default()->create_dirs(data_path).ok())) {
-        RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(strings::Substitute(
-                                               "failed to create data root path $0", data_path)),
-                                       "create_dirs failed");
+        RETURN_NOT_OK_STATUS_WITH_WARN(
+                Status::IOError("failed to create data root path {}", data_path),
+                "create_dirs failed");
     }
 
     return Status::OK();
@@ -196,8 +196,7 @@ Status DataDir::_init_meta() {
     Status res = _meta->init();
     if (!res.ok()) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
-                Status::IOError(
-                        strings::Substitute("open rocksdb failed, path=$0", _path_desc.filepath)),
+                Status::IOError("open rocksdb failed, path={}", _path_desc.filepath),
                 "init OlapMeta failed");
     }
     return Status::OK();
diff --git a/be/src/olap/fs/remote_block_manager.cpp b/be/src/olap/fs/remote_block_manager.cpp
index 481f0ac2d6..9f9042551d 100644
--- a/be/src/olap/fs/remote_block_manager.cpp
+++ b/be/src/olap/fs/remote_block_manager.cpp
@@ -121,11 +121,11 @@ RemoteWritableBlock::RemoteWritableBlock(RemoteBlockManager* block_manager,
 RemoteWritableBlock::~RemoteWritableBlock() {}
 
 Status RemoteWritableBlock::close() {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 Status RemoteWritableBlock::abort() {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 BlockManager* RemoteWritableBlock::block_manager() const {
@@ -146,15 +146,15 @@ Status RemoteWritableBlock::append(const Slice& data) {
 }
 
 Status RemoteWritableBlock::appendv(const Slice* data, size_t data_cnt) {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 Status RemoteWritableBlock::flush_data_async() {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 Status RemoteWritableBlock::finalize() {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 size_t RemoteWritableBlock::bytes_appended() const {
@@ -166,7 +166,7 @@ WritableBlock::State RemoteWritableBlock::state() const {
 }
 
 Status RemoteWritableBlock::_close(SyncMode mode) {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 ////////////////////////////////////////////////////////////
@@ -227,7 +227,7 @@ RemoteReadableBlock::RemoteReadableBlock(
 RemoteReadableBlock::~RemoteReadableBlock() {}
 
 Status RemoteReadableBlock::close() {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 BlockManager* RemoteReadableBlock::block_manager() const {
@@ -244,7 +244,7 @@ const FilePathDesc& RemoteReadableBlock::path_desc() const {
 }
 
 Status RemoteReadableBlock::size(uint64_t* sz) const {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 Status RemoteReadableBlock::read(uint64_t offset, Slice result) const {
@@ -252,7 +252,7 @@ Status RemoteReadableBlock::read(uint64_t offset, Slice result) const {
 }
 
 Status RemoteReadableBlock::readv(uint64_t offset, const Slice* results, size_t res_cnt) const {
-    return Status::IOError("invalid function", 0, "");
+    return Status::IOError("invalid function");
 }
 
 } // namespace internal
@@ -275,9 +275,8 @@ Status RemoteBlockManager::open() {
 Status RemoteBlockManager::create_block(const CreateBlockOptions& opts,
                                         std::unique_ptr<WritableBlock>* block) {
     if (_opts.read_only) {
-        std::stringstream ss;
-        ss << "create_block failed. remote block is readonly: " << opts.path_desc.debug_string();
-        return Status::NotSupported(ss.str());
+        return Status::NotSupported("create_block failed. remote block is readonly: {}",
+                                    opts.path_desc.debug_string());
     }
 
     shared_ptr<WritableFile> local_writer;
diff --git a/be/src/olap/key_coder.h b/be/src/olap/key_coder.h
index 7f452350d4..b25aa20dd8 100644
--- a/be/src/olap/key_coder.h
+++ b/be/src/olap/key_coder.h
@@ -120,8 +120,8 @@ public:
     static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr,
                                    MemPool* pool) {
         if (encoded_key->size < sizeof(UnsignedCppType)) {
-            return Status::InvalidArgument(Substitute("Key too short, need=$0 vs real=$1",
-                                                      sizeof(UnsignedCppType), encoded_key->size));
+            return Status::InvalidArgument("Key too short, need={} vs real={}",
+                                           sizeof(UnsignedCppType), encoded_key->size);
         }
         UnsignedCppType unsigned_val;
         memcpy(&unsigned_val, encoded_key->data, sizeof(UnsignedCppType));
@@ -158,8 +158,8 @@ public:
     static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr,
                                    MemPool* pool) {
         if (encoded_key->size < sizeof(UnsignedCppType)) {
-            return Status::InvalidArgument(Substitute("Key too short, need=$0 vs real=$1",
-                                                      sizeof(UnsignedCppType), encoded_key->size));
+            return Status::InvalidArgument("Key too short, need={} vs real={}",
+                                           sizeof(UnsignedCppType), encoded_key->size);
         }
         UnsignedCppType unsigned_val;
         memcpy(&unsigned_val, encoded_key->data, sizeof(UnsignedCppType));
@@ -249,8 +249,8 @@ public:
     static Status decode_ascending(Slice* encoded_key, size_t index_size, uint8_t* cell_ptr,
                                    MemPool* pool) {
         if (encoded_key->size < index_size) {
-            return Status::InvalidArgument(
-                    Substitute("Key too short, need=$0 vs real=$1", index_size, encoded_key->size));
+            return Status::InvalidArgument("Key too short, need={} vs real={}", index_size,
+                                           encoded_key->size);
         }
         Slice* slice = (Slice*)cell_ptr;
         slice->data = (char*)pool->allocate(index_size);
diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp
index 5088bd8775..e08ef5ca5b 100644
--- a/be/src/olap/olap_server.cpp
+++ b/be/src/olap/olap_server.cpp
@@ -650,9 +650,9 @@ Status StorageEngine::_submit_compaction_task(TabletSharedPtr tablet,
                                               CompactionType compaction_type) {
     bool already_exist = _push_tablet_into_submitted_compaction(tablet, compaction_type);
     if (already_exist) {
-        return Status::AlreadyExist(strings::Substitute(
-                "compaction task has already been submitted, tablet_id=$0, compaction_type=$1.",
-                tablet->tablet_id(), compaction_type));
+        return Status::AlreadyExist(
+                "compaction task has already been submitted, tablet_id={}, compaction_type={}.",
+                tablet->tablet_id(), compaction_type);
     }
     int64_t permits = 0;
     Status st = tablet->prepare_compaction_and_calculate_permits(compaction_type, tablet, &permits);
@@ -677,9 +677,9 @@ Status StorageEngine::_submit_compaction_task(TabletSharedPtr tablet,
             tablet->reset_compaction(compaction_type);
             _pop_tablet_from_submitted_compaction(tablet, compaction_type);
             return Status::InternalError(
-                    strings::Substitute("failed to submit compaction task to thread pool, "
-                                        "tablet_id=$0, compaction_type=$1.",
-                                        tablet->tablet_id(), compaction_type));
+                    "failed to submit compaction task to thread pool, "
+                    "tablet_id={}, compaction_type={}.",
+                    tablet->tablet_id(), compaction_type);
         }
         return Status::OK();
     } else {
@@ -688,11 +688,11 @@ Status StorageEngine::_submit_compaction_task(TabletSharedPtr tablet,
         _pop_tablet_from_submitted_compaction(tablet, compaction_type);
         if (!st.ok()) {
             return Status::InternalError(
-                    strings::Substitute("failed to prepare compaction task and calculate permits, "
-                                        "tablet_id=$0, compaction_type=$1, "
-                                        "permit=$2, current_permit=$3, status=$4",
-                                        tablet->tablet_id(), compaction_type, permits,
-                                        _permit_limiter.usage(), st.get_error_msg()));
+                    "failed to prepare compaction task and calculate permits, "
+                    "tablet_id={}, compaction_type={}, "
+                    "permit={}, current_permit={}, status={}",
+                    tablet->tablet_id(), compaction_type, permits, _permit_limiter.usage(),
+                    st.get_error_msg());
         }
         return st;
     }
diff --git a/be/src/olap/row_block2.cpp b/be/src/olap/row_block2.cpp
index 393fe40adf..b850a50899 100644
--- a/be/src/olap/row_block2.cpp
+++ b/be/src/olap/row_block2.cpp
@@ -199,8 +199,8 @@ Status RowBlockV2::_copy_data_to_column(int cid,
                 if (LIKELY(slice->size <= limit)) {
                     column_string->insert_data(slice->data, slice->size);
                 } else {
-                    return Status::NotSupported(fmt::format(
-                            "Not support string len over than {} in vec engine.", limit));
+                    return Status::NotSupported(
+                            "Not support string len over than {} in vec engine.", limit);
                 }
             } else {
                 column_string->insert_default();
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
index 6fb1258604..d07dc74e3a 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
@@ -86,8 +86,8 @@ Status BinaryDictPageBuilder::add(const uint8_t* vals, size_t* count) {
                 if (src->size > 0) {
                     char* item_mem = (char*)_pool.allocate(src->size);
                     if (item_mem == nullptr) {
-                        return Status::MemoryAllocFailed(
-                                strings::Substitute("memory allocate failed, size:$0", src->size));
+                        return Status::MemoryAllocFailed("memory allocate failed, size:{}",
+                                                         src->size);
                     }
                     dict_item.relocate(item_mem);
                 }
@@ -191,8 +191,8 @@ BinaryDictPageDecoder::BinaryDictPageDecoder(Slice data, const PageDecoderOption
 Status BinaryDictPageDecoder::init() {
     CHECK(!_parsed);
     if (_data.size < BINARY_DICT_PAGE_HEADER_SIZE) {
-        return Status::Corruption(strings::Substitute("invalid data size:$0, header size:$1",
-                                                      _data.size, BINARY_DICT_PAGE_HEADER_SIZE));
+        return Status::Corruption("invalid data size:{}, header size:{}", _data.size,
+                                  BINARY_DICT_PAGE_HEADER_SIZE);
     }
     size_t type = decode_fixed32_le((const uint8_t*)&_data.data[0]);
     _encoding_type = static_cast<EncodingTypePB>(type);
@@ -209,7 +209,7 @@ Status BinaryDictPageDecoder::init() {
         _data_page_decoder.reset(new BinaryPlainPageDecoder<OLAP_FIELD_TYPE_INT>(_data, _options));
     } else {
         LOG(WARNING) << "invalid encoding type:" << _encoding_type;
-        return Status::Corruption(strings::Substitute("invalid encoding type:$0", _encoding_type));
+        return Status::Corruption("invalid encoding type:{}", _encoding_type);
     }
 
     RETURN_IF_ERROR(_data_page_decoder->init());
@@ -333,8 +333,7 @@ Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) {
     out = reinterpret_cast<Slice*>(dst->data());
     char* destination = (char*)dst->column_block()->pool()->allocate(mem_size);
     if (destination == nullptr) {
-        return Status::MemoryAllocFailed(
-                strings::Substitute("memory allocate failed, size:$0", mem_size));
+        return Status::MemoryAllocFailed("memory allocate failed, size:{}", mem_size);
     }
     for (int i = 0; i < len; ++i) {
         out->relocate(destination);
diff --git a/be/src/olap/rowset/segment_v2/binary_plain_page.h b/be/src/olap/rowset/segment_v2/binary_plain_page.h
index e4531fe00a..1242fd9b75 100644
--- a/be/src/olap/rowset/segment_v2/binary_plain_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_plain_page.h
@@ -177,11 +177,10 @@ public:
         CHECK(!_parsed);
 
         if (_data.size < sizeof(uint32_t)) {
-            std::stringstream ss;
-            ss << "file corruption: not enough bytes for trailer in BinaryPlainPageDecoder ."
-                  "invalid data size:"
-               << _data.size << ", trailer size:" << sizeof(uint32_t);
-            return Status::Corruption(ss.str());
+            return Status::Corruption(
+                    "file corruption: not enough bytes for trailer in BinaryPlainPageDecoder ."
+                    "invalid data size:{}, trailer size:{}",
+                    _data.size, sizeof(uint32_t));
         }
 
         // Decode trailer
@@ -230,8 +229,7 @@ public:
         out = reinterpret_cast<Slice*>(dst->data());
         char* destination = (char*)dst->column_block()->pool()->allocate(mem_size);
         if (destination == nullptr) {
-            return Status::MemoryAllocFailed(
-                    strings::Substitute("memory allocate failed, size:$0", mem_size));
+            return Status::MemoryAllocFailed("memory allocate failed, size:{}", mem_size);
         }
         for (int i = 0; i < max_fetch; ++i) {
             out->relocate(destination);
diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
index 2eebd1cfe7..5b8d4d3090 100644
--- a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
@@ -123,8 +123,7 @@ Status BinaryPrefixPageDecoder::_read_next_value() {
     uint32_t non_shared_len;
     auto data_ptr = _decode_value_lengths(_next_ptr, &shared_len, &non_shared_len);
     if (data_ptr == nullptr) {
-        return Status::Corruption(
-                strings::Substitute("Failed to decode value at position $0", _cur_pos));
+        return Status::Corruption("Failed to decode value at position {}", _cur_pos);
     }
     _current_value.resize(shared_len);
     _current_value.append(data_ptr, non_shared_len);
@@ -217,8 +216,7 @@ Status BinaryPrefixPageDecoder::_read_next_value_to_output(Slice prev, MemPool*
     uint32_t non_shared_len;
     auto data_ptr = _decode_value_lengths(_next_ptr, &shared_len, &non_shared_len);
     if (data_ptr == nullptr) {
-        return Status::Corruption(
-                strings::Substitute("Failed to decode value at position $0", _cur_pos));
+        return Status::Corruption("Failed to decode value at position {}", _cur_pos);
     }
 
     output->size = shared_len + non_shared_len;
@@ -237,8 +235,7 @@ Status BinaryPrefixPageDecoder::_copy_current_to_output(MemPool* mem_pool, Slice
     if (output->size > 0) {
         output->data = (char*)mem_pool->allocate(output->size);
         if (output->data == nullptr) {
-            return Status::MemoryAllocFailed(
-                    strings::Substitute("failed to allocate $0 bytes", output->size));
+            return Status::MemoryAllocFailed("failed to allocate {} bytes", output->size);
         }
         memcpy(output->data, _current_value.data(), output->size);
     }
diff --git a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
index 9308c432b4..2c696f291f 100644
--- a/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/bitmap_index_writer.cpp
@@ -233,7 +233,7 @@ Status BitmapIndexWriter::create(const TypeInfo* type_info,
         res->reset(new BitmapIndexWriterImpl<OLAP_FIELD_TYPE_BOOL>(type_info));
         break;
     default:
-        return Status::NotSupported("unsupported type for bitmap index: " + std::to_string(type));
+        return Status::NotSupported("unsupported type for bitmap index: {}", std::to_string(type));
     }
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/bitshuffle_page.h b/be/src/olap/rowset/segment_v2/bitshuffle_page.h
index d0b22f663b..02163a6a7f 100644
--- a/be/src/olap/rowset/segment_v2/bitshuffle_page.h
+++ b/be/src/olap/rowset/segment_v2/bitshuffle_page.h
@@ -243,21 +243,18 @@ inline Status parse_bit_shuffle_header(const Slice& data, size_t& num_elements,
                                        size_t& compressed_size, size_t& num_element_after_padding,
                                        int& size_of_element) {
     if (data.size < BITSHUFFLE_PAGE_HEADER_SIZE) {
-        std::stringstream ss;
-        ss << "file corruption: invalid data size:" << data.size
-           << ", header size:" << BITSHUFFLE_PAGE_HEADER_SIZE;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("file corruption: invalid data size:{}, header size:{}",
+                                     data.size, BITSHUFFLE_PAGE_HEADER_SIZE);
     }
 
     num_elements = decode_fixed32_le((const uint8_t*)&data[0]);
     compressed_size = decode_fixed32_le((const uint8_t*)&data[4]);
     num_element_after_padding = decode_fixed32_le((const uint8_t*)&data[8]);
     if (num_element_after_padding != ALIGN_UP(num_elements, 8)) {
-        std::stringstream ss;
-        ss << "num of element information corrupted,"
-           << " _num_element_after_padding:" << num_element_after_padding
-           << ", _num_elements:" << num_elements;
-        return Status::InternalError(ss.str());
+        return Status::InternalError(
+                "num of element information corrupted,"
+                " _num_element_after_padding:{}, _num_elements:{}",
+                num_element_after_padding, num_elements);
     }
     size_of_element = decode_fixed32_le((const uint8_t*)&data[12]);
     switch (size_of_element) {
@@ -270,9 +267,7 @@ inline Status parse_bit_shuffle_header(const Slice& data, size_t& num_elements,
     case 16:
         break;
     default:
-        std::stringstream ss;
-        ss << "invalid size_of_elem:" << size_of_element;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("invalid size_of_elem:{}", size_of_element);
     }
     return Status::OK();
 }
@@ -306,16 +301,13 @@ public:
 
         // Currently, only the UINT32 block encoder supports expanding size:
         if (UNLIKELY(Type != OLAP_FIELD_TYPE_UNSIGNED_INT && _size_of_element != SIZE_OF_TYPE)) {
-            std::stringstream ss;
-            ss << "invalid size info. size of element:" << _size_of_element
-               << ", SIZE_OF_TYPE:" << SIZE_OF_TYPE << ", type:" << Type;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "invalid size info. size of element:{}, SIZE_OF_TYPE:{}, type:{}",
+                    _size_of_element, SIZE_OF_TYPE, Type);
         }
         if (UNLIKELY(_size_of_element > SIZE_OF_TYPE)) {
-            std::stringstream ss;
-            ss << "invalid size info. size of element:" << _size_of_element
-               << ", SIZE_OF_TYPE:" << SIZE_OF_TYPE;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("invalid size info. size of element:{}, SIZE_OF_TYPE:{}",
+                                         _size_of_element, SIZE_OF_TYPE);
         }
         _parsed = true;
         return Status::OK();
diff --git a/be/src/olap/rowset/segment_v2/bitshuffle_page_pre_decoder.h b/be/src/olap/rowset/segment_v2/bitshuffle_page_pre_decoder.h
index 8ca5695a38..e762f90d39 100644
--- a/be/src/olap/rowset/segment_v2/bitshuffle_page_pre_decoder.h
+++ b/be/src/olap/rowset/segment_v2/bitshuffle_page_pre_decoder.h
@@ -57,10 +57,9 @@ struct BitShufflePagePreDecoder : public DataPagePreDecoder {
                                                  num_element_after_padding, size_of_element));
 
         if (compressed_size != data.size) {
-            std::stringstream ss;
-            ss << "Size information unmatched, compressed_size:" << compressed_size
-               << ", num_elements:" << num_elements << ", data size:" << data.size;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "Size information unmatched, compressed_size:{}, num_elements:{}, data size:{}",
+                    compressed_size, num_elements, data.size);
         }
 
         Slice decoded_slice;
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.cpp b/be/src/olap/rowset/segment_v2/bloom_filter.cpp
index d7396b44c3..523f00260f 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter.cpp
+++ b/be/src/olap/rowset/segment_v2/bloom_filter.cpp
@@ -31,8 +31,7 @@ Status BloomFilter::create(BloomFilterAlgorithmPB algorithm, std::unique_ptr<Blo
     if (algorithm == BLOCK_BLOOM_FILTER) {
         bf->reset(new BlockSplitBloomFilter());
     } else {
-        return Status::InternalError(
-                strings::Substitute("invalid bloom filter algorithm:$0", algorithm));
+        return Status::InternalError("invalid bloom filter algorithm:{}", algorithm);
     }
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter.h b/be/src/olap/rowset/segment_v2/bloom_filter.h
index 4c015f0743..8d068bf732 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter.h
+++ b/be/src/olap/rowset/segment_v2/bloom_filter.h
@@ -66,7 +66,7 @@ public:
         if (strategy == HASH_MURMUR3_X64_64) {
             _hash_func = murmur_hash3_x64_64;
         } else {
-            return Status::InvalidArgument(strings::Substitute("invalid strategy:$0", strategy));
+            return Status::InvalidArgument("invalid strategy:{}", strategy);
         }
         _num_bytes = filter_size;
         DCHECK((_num_bytes & (_num_bytes - 1)) == 0);
@@ -86,10 +86,10 @@ public:
         if (strategy == HASH_MURMUR3_X64_64) {
             _hash_func = murmur_hash3_x64_64;
         } else {
-            return Status::InvalidArgument(strings::Substitute("invalid strategy:$0", strategy));
+            return Status::InvalidArgument("invalid strategy:{}", strategy);
         }
         if (size == 0) {
-            return Status::InvalidArgument(strings::Substitute("invalid size:$0", size));
+            return Status::InvalidArgument("invalid size:{}", size);
         }
         _data = new char[size];
         memcpy(_data, buf, size);
diff --git a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
index 5b3156078e..d105170a09 100644
--- a/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/bloom_filter_index_writer.cpp
@@ -213,7 +213,7 @@ Status BloomFilterIndexWriter::create(const BloomFilterOptions& bf_options,
         res->reset(new BloomFilterIndexWriterImpl<OLAP_FIELD_TYPE_DECIMAL>(bf_options, type_info));
         break;
     default:
-        return Status::NotSupported("unsupported type for bitmap index: " + std::to_string(type));
+        return Status::NotSupported("unsupported type for bitmap index: {}", std::to_string(type));
     }
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp
index 9087c7b246..ce0ea63d7d 100644
--- a/be/src/olap/rowset/segment_v2/column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/column_reader.cpp
@@ -85,7 +85,7 @@ Status ColumnReader::create(const ColumnReaderOptions& opts, const ColumnMetaPB&
             return Status::OK();
         }
         default:
-            return Status::NotSupported("unsupported type for ColumnReader: " +
+            return Status::NotSupported("unsupported type for ColumnReader: {}",
                                         std::to_string(type));
         }
     }
@@ -104,8 +104,7 @@ ColumnReader::~ColumnReader() = default;
 Status ColumnReader::init() {
     _type_info = get_type_info(&_meta);
     if (_type_info == nullptr) {
-        return Status::NotSupported(
-                strings::Substitute("unsupported typeinfo, type=$0", _meta.type()));
+        return Status::NotSupported("unsupported typeinfo, type={}", _meta.type());
     }
     RETURN_IF_ERROR(EncodingInfo::get(_type_info.get(), _meta.encoding(), &_encoding_info));
 
@@ -125,17 +124,15 @@ Status ColumnReader::init() {
             _bf_index_meta = &index_meta.bloom_filter_index();
             break;
         default:
-            return Status::Corruption(
-                    strings::Substitute("Bad file $0: invalid column index type $1",
-                                        _path_desc.filepath, index_meta.type()));
+            return Status::Corruption("Bad file {}: invalid column index type {}",
+                                      _path_desc.filepath, index_meta.type());
         }
     }
     // ArrayColumnWriter writes a single empty array and flushes. In this scenario,
     // the item writer doesn't write any data and the corresponding ordinal index is empty.
     if (_ordinal_index_meta == nullptr && !is_empty()) {
-        return Status::Corruption(
-                strings::Substitute("Bad file $0: missing ordinal index for column $1",
-                                    _path_desc.filepath, _meta.column_id()));
+        return Status::Corruption("Bad file {}: missing ordinal index for column {}",
+                                  _path_desc.filepath, _meta.column_id());
     }
     return Status::OK();
 }
@@ -341,7 +338,7 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat
     RETURN_IF_ERROR(_ensure_index_loaded());
     *iter = _ordinal_index->seek_at_or_before(ordinal);
     if (!iter->valid()) {
-        return Status::NotFound(strings::Substitute("Failed to seek to ordinal $0, ", ordinal));
+        return Status::NotFound("Failed to seek to ordinal {}, ", ordinal);
     }
     return Status::OK();
 }
@@ -374,7 +371,7 @@ Status ColumnReader::new_iterator(ColumnIterator** iterator) {
             return Status::OK();
         }
         default:
-            return Status::NotSupported("unsupported type to create iterator: " +
+            return Status::NotSupported("unsupported type to create iterator: {}",
                                         std::to_string(type));
         }
     }
diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp
index c96ccd9a61..ef5acb8276 100644
--- a/be/src/olap/rowset/segment_v2/column_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/column_writer.cpp
@@ -171,7 +171,7 @@ Status ColumnWriter::create(const ColumnWriterOptions& opts, const TabletColumn*
             return Status::OK();
         }
         default:
-            return Status::NotSupported("unsupported type for ColumnWriter: " +
+            return Status::NotSupported("unsupported type for ColumnWriter: {}",
                                         std::to_string(field->type()));
         }
     }
@@ -276,9 +276,8 @@ Status ScalarColumnWriter::init() {
     opts.data_page_size = _opts.data_page_size;
     RETURN_IF_ERROR(_encoding_info->create_page_builder(opts, &page_builder));
     if (page_builder == nullptr) {
-        return Status::NotSupported(
-                strings::Substitute("Failed to create page builder for type $0 and encoding $1",
-                                    get_field()->type(), _opts.meta->encoding()));
+        return Status::NotSupported("Failed to create page builder for type {} and encoding {}",
+                                    get_field()->type(), _opts.meta->encoding());
     }
     // should store more concrete encoding type instead of DEFAULT_ENCODING
     // because the default encoding of a data type can be changed in the future
diff --git a/be/src/olap/rowset/segment_v2/encoding_info.cpp b/be/src/olap/rowset/segment_v2/encoding_info.cpp
index 4f9e2d5d14..555886611d 100644
--- a/be/src/olap/rowset/segment_v2/encoding_info.cpp
+++ b/be/src/olap/rowset/segment_v2/encoding_info.cpp
@@ -316,9 +316,8 @@ Status EncodingInfoResolver::get(FieldType data_type, EncodingTypePB encoding_ty
     auto key = std::make_pair(data_type, encoding_type);
     auto it = _encoding_map.find(key);
     if (it == std::end(_encoding_map)) {
-        return Status::InternalError(
-                strings::Substitute("fail to find valid type encoding, type:$0, encoding:$1",
-                                    data_type, encoding_type));
+        return Status::InternalError("fail to find valid type encoding, type:{}, encoding:{}",
+                                     data_type, encoding_type);
     }
     *out = it->second;
     return Status::OK();
diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
index 682f620ea8..7e3140fb9a 100644
--- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp
@@ -33,8 +33,7 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) {
 
     _type_info = get_scalar_type_info((FieldType)_meta.data_type());
     if (_type_info == nullptr) {
-        return Status::NotSupported(
-                strings::Substitute("unsupported typeinfo, type=$0", _meta.data_type()));
+        return Status::NotSupported("unsupported typeinfo, type={}", _meta.data_type());
     }
     RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info));
     _value_key_coder = get_key_coder(_type_info->type());
diff --git a/be/src/olap/rowset/segment_v2/page_io.cpp b/be/src/olap/rowset/segment_v2/page_io.cpp
index 1c649aa44c..fba66470a9 100644
--- a/be/src/olap/rowset/segment_v2/page_io.cpp
+++ b/be/src/olap/rowset/segment_v2/page_io.cpp
@@ -137,7 +137,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle*
     // every page contains 4 bytes footer length and 4 bytes checksum
     const uint32_t page_size = opts.page_pointer.size;
     if (page_size < 8) {
-        return Status::Corruption(strings::Substitute("Bad page: too small size ($0)", page_size));
+        return Status::Corruption("Bad page: too small size ({})", page_size);
     }
 
     // hold compressed page at first, reset to decompressed page later
@@ -153,8 +153,8 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle*
         uint32_t expect = decode_fixed32_le((uint8_t*)page_slice.data + page_slice.size - 4);
         uint32_t actual = crc32c::Value(page_slice.data, page_slice.size - 4);
         if (expect != actual) {
-            return Status::Corruption(strings::Substitute(
-                    "Bad page: checksum mismatch (actual=$0 vs expect=$1)", actual, expect));
+            return Status::Corruption("Bad page: checksum mismatch (actual={} vs expect={})",
+                                      actual, expect);
         }
     }
 
@@ -180,9 +180,9 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, PageHandle*
         Slice decompressed_body(decompressed_page.get(), footer->uncompressed_size());
         RETURN_IF_ERROR(opts.codec->decompress(compressed_body, &decompressed_body));
         if (decompressed_body.size != footer->uncompressed_size()) {
-            return Status::Corruption(strings::Substitute(
-                    "Bad page: record uncompressed size=$0 vs real decompressed size=$1",
-                    footer->uncompressed_size(), decompressed_body.size));
+            return Status::Corruption(
+                    "Bad page: record uncompressed size={} vs real decompressed size={}",
+                    footer->uncompressed_size(), decompressed_body.size);
         }
         // append footer and footer size
         memcpy(decompressed_body.data + decompressed_body.size, page_slice.data + body_size,
diff --git a/be/src/olap/rowset/segment_v2/plain_page.h b/be/src/olap/rowset/segment_v2/plain_page.h
index cd7db008ce..a38a3d7dc9 100644
--- a/be/src/olap/rowset/segment_v2/plain_page.h
+++ b/be/src/olap/rowset/segment_v2/plain_page.h
@@ -111,19 +111,16 @@ public:
         CHECK(!_parsed);
 
         if (_data.size < PLAIN_PAGE_HEADER_SIZE) {
-            std::stringstream ss;
-            ss << "file corruption: not enough bytes for header in PlainPageDecoder ."
-                  "invalid data size:"
-               << _data.size << ", header size:" << PLAIN_PAGE_HEADER_SIZE;
-            return Status::InternalError(ss.str());
+            return Status::InternalError(
+                    "file corruption: not enough bytes for header in PlainPageDecoder ."
+                    "invalid data size:{}, header size:{}",
+                    _data.size, PLAIN_PAGE_HEADER_SIZE);
         }
 
         _num_elems = decode_fixed32_le((const uint8_t*)&_data[0]);
 
         if (_data.size != PLAIN_PAGE_HEADER_SIZE + _num_elems * SIZE_OF_TYPE) {
-            std::stringstream ss;
-            ss << "file corruption: unexpected data size.";
-            return Status::InternalError(ss.str());
+            return Status::InternalError("file corruption: unexpected data size.");
         }
 
         _parsed = true;
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp
index 7970d0da5b..039e2cc100 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -107,8 +107,8 @@ Status Segment::_parse_footer() {
     RETURN_IF_ERROR(rblock->size(&file_size));
 
     if (file_size < 12) {
-        return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < 12",
-                                                      _path_desc.filepath, file_size));
+        return Status::Corruption("Bad segment file {}: file size {} < 12", _path_desc.filepath,
+                                  file_size);
     }
 
     uint8_t fixed_buf[12];
@@ -116,16 +116,15 @@ Status Segment::_parse_footer() {
 
     // validate magic number
     if (memcmp(fixed_buf + 8, k_segment_magic, k_segment_magic_length) != 0) {
-        return Status::Corruption(strings::Substitute("Bad segment file $0: magic number not match",
-                                                      _path_desc.filepath));
+        return Status::Corruption("Bad segment file {}: magic number not match",
+                                  _path_desc.filepath);
     }
 
     // read footer PB
     uint32_t footer_length = decode_fixed32_le(fixed_buf);
     if (file_size < 12 + footer_length) {
-        return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < $2",
-                                                      _path_desc.filepath, file_size,
-                                                      12 + footer_length));
+        return Status::Corruption("Bad segment file {}: file size {} < {}", _path_desc.filepath,
+                                  file_size, 12 + footer_length);
     }
     _mem_tracker->consume(footer_length);
 
@@ -137,15 +136,15 @@ Status Segment::_parse_footer() {
     uint32_t expect_checksum = decode_fixed32_le(fixed_buf + 4);
     uint32_t actual_checksum = crc32c::Value(footer_buf.data(), footer_buf.size());
     if (actual_checksum != expect_checksum) {
-        return Status::Corruption(strings::Substitute(
-                "Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2",
-                _path_desc.filepath, actual_checksum, expect_checksum));
+        return Status::Corruption(
+                "Bad segment file {}: footer checksum not match, actual={} vs expect={}",
+                _path_desc.filepath, actual_checksum, expect_checksum);
     }
 
     // deserialize footer PB
     if (!_footer.ParseFromString(footer_buf)) {
-        return Status::Corruption(strings::Substitute(
-                "Bad segment file $0: failed to parse SegmentFooterPB", _path_desc.filepath));
+        return Status::Corruption("Bad segment file {}: failed to parse SegmentFooterPB",
+                                  _path_desc.filepath);
     }
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index 2a60172ee8..a719f341b9 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -239,8 +239,7 @@ uint64_t SegmentWriter::estimate_segment_size() {
 Status SegmentWriter::finalize(uint64_t* segment_file_size, uint64_t* index_size) {
     // check disk capacity
     if (_data_dir != nullptr && _data_dir->reach_capacity_limit((int64_t)estimate_segment_size())) {
-        return Status::InternalError(
-                fmt::format("disk {} exceed capacity limit.", _data_dir->path_hash()));
+        return Status::InternalError("disk {} exceed capacity limit.", _data_dir->path_hash());
     }
     for (auto& column_writer : _column_writers) {
         RETURN_IF_ERROR(column_writer->finish());
diff --git a/be/src/olap/short_key_index.cpp b/be/src/olap/short_key_index.cpp
index dd940c8a22..19055b3b12 100644
--- a/be/src/olap/short_key_index.cpp
+++ b/be/src/olap/short_key_index.cpp
@@ -56,9 +56,8 @@ Status ShortKeyIndexDecoder::parse(const Slice& body, const segment_v2::ShortKey
 
     // check if body size match footer's information
     if (body.size != (_footer.key_bytes() + _footer.offset_bytes())) {
-        return Status::Corruption(strings::Substitute("Index size not match, need=$0, real=$1",
-                                                      _footer.key_bytes() + _footer.offset_bytes(),
-                                                      body.size));
+        return Status::Corruption("Index size not match, need={}, real={}",
+                                  _footer.key_bytes() + _footer.offset_bytes(), body.size);
     }
 
     // set index buffer
diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp
index 5a75a4f88d..fd6aba53cb 100644
--- a/be/src/olap/storage_engine.cpp
+++ b/be/src/olap/storage_engine.cpp
@@ -244,7 +244,7 @@ Status StorageEngine::_init_store_map() {
         for (auto store : tmp_stores) {
             delete store;
         }
-        return Status::InternalError(strings::Substitute("init path failed, error=$0", error_msg));
+        return Status::InternalError("init path failed, error={}", error_msg);
     }
 
     for (auto store : tmp_stores) {
@@ -274,8 +274,8 @@ Status StorageEngine::_init_stream_load_recorder(const std::string& stream_load_
     auto st = _stream_load_recorder->init();
     if (!st.ok()) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
-                Status::IOError(Substitute("open StreamLoadRecorder rocksdb failed, path=$0",
-                                           stream_load_record_path)),
+                Status::IOError("open StreamLoadRecorder rocksdb failed, path={}",
+                                stream_load_record_path),
                 "init StreamLoadRecorder failed");
     }
     return Status::OK();
@@ -307,9 +307,8 @@ Status StorageEngine::_judge_and_update_effective_cluster_id(int32_t cluster_id)
     } else {
         if (cluster_id != _effective_cluster_id) {
             RETURN_NOT_OK_STATUS_WITH_WARN(
-                    Status::Corruption(
-                            strings::Substitute("multiple cluster ids is not equal. one=$0, other=",
-                                                _effective_cluster_id, cluster_id)),
+                    Status::Corruption("multiple cluster ids is not equal. one={}, other={}",
+                                       _effective_cluster_id, cluster_id),
                     "cluster id not equal");
         }
     }
@@ -455,9 +454,8 @@ Status StorageEngine::_check_all_root_path_cluster_id() {
             cluster_id = tmp_cluster_id;
         } else {
             RETURN_NOT_OK_STATUS_WITH_WARN(
-                    Status::Corruption(strings::Substitute(
-                            "multiple cluster ids is not equal. one=$0, other=", cluster_id,
-                            tmp_cluster_id)),
+                    Status::Corruption("multiple cluster ids is not equal. one={}, other={}",
+                                       cluster_id, tmp_cluster_id),
                     "cluster id not equal");
         }
     }
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 2db820b975..da29ce7253 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -1438,8 +1438,7 @@ Status Tablet::prepare_compaction_and_calculate_permits(CompactionType compactio
             *permits = 0;
             if (res.precise_code() != OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION) {
                 DorisMetrics::instance()->cumulative_compaction_request_failed->increment(1);
-                return Status::InternalError(
-                        fmt::format("prepare cumulative compaction with err: {}", res));
+                return Status::InternalError("prepare cumulative compaction with err: {}", res);
             }
             // return OK if OLAP_ERR_CUMULATIVE_NO_SUITABLE_VERSION, so that we don't need to
             // print too much useless logs.
@@ -1468,8 +1467,7 @@ Status Tablet::prepare_compaction_and_calculate_permits(CompactionType compactio
             *permits = 0;
             if (res.precise_code() != OLAP_ERR_BE_NO_SUITABLE_VERSION) {
                 DorisMetrics::instance()->base_compaction_request_failed->increment(1);
-                return Status::InternalError(
-                        fmt::format("prepare base compaction with err: {}", res));
+                return Status::InternalError("prepare base compaction with err: {}", res);
             }
             // return OK if OLAP_ERR_BE_NO_SUITABLE_VERSION, so that we don't need to
             // print too much useless logs.
diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp
index 2fb942e0ef..ac0b358393 100644
--- a/be/src/olap/task/engine_batch_load_task.cpp
+++ b/be/src/olap/task/engine_batch_load_task.cpp
@@ -109,8 +109,7 @@ Status EngineBatchLoadTask::_init() {
         LOG(WARNING) << "get tables failed. "
                      << "tablet_id: " << _push_req.tablet_id
                      << ", schema_hash: " << _push_req.schema_hash;
-        return Status::InvalidArgument(
-                fmt::format("Could not find tablet {}", _push_req.tablet_id));
+        return Status::InvalidArgument("Could not find tablet {}", _push_req.tablet_id);
     }
 
     // check disk capacity
@@ -159,7 +158,7 @@ Status EngineBatchLoadTask::_get_tmp_file_dir(const string& root_path, string* d
         std::filesystem::create_directories(*download_path, ec);
 
         if (ec) {
-            status = Status::IOError("Create download dir failed " + *download_path);
+            status = Status::IOError("Create download dir failed {}", *download_path);
             LOG(WARNING) << "create download dir failed.path: " << *download_path
                          << ", error code: " << ec;
         }
diff --git a/be/src/runtime/buffered_tuple_stream3.cc b/be/src/runtime/buffered_tuple_stream3.cc
index a2c1e8db02..8c05608831 100644
--- a/be/src/runtime/buffered_tuple_stream3.cc
+++ b/be/src/runtime/buffered_tuple_stream3.cc
@@ -668,9 +668,9 @@ Status BufferedTupleStream3::GetRows(std::unique_ptr<RowBatch>* batch, bool* got
     if (num_rows() > numeric_limits<int>::max()) {
         // RowBatch::num_rows_ is a 32-bit int, avoid an overflow.
         return Status::InternalError(
-                Substitute("Trying to read $0 rows into in-memory batch failed. Limit "
-                           "is $1",
-                           num_rows(), numeric_limits<int>::max()));
+                "Trying to read {} rows into in-memory batch failed. Limit "
+                "is {}",
+                num_rows(), numeric_limits<int>::max());
     }
     RETURN_IF_ERROR(PinStream(got_rows));
     if (!*got_rows) return Status::OK();
diff --git a/be/src/runtime/bufferpool/system_allocator.cc b/be/src/runtime/bufferpool/system_allocator.cc
index 3fa69e981e..d4846f595c 100644
--- a/be/src/runtime/bufferpool/system_allocator.cc
+++ b/be/src/runtime/bufferpool/system_allocator.cc
@@ -128,9 +128,8 @@ Status SystemAllocator::AllocateViaMalloc(int64_t len, uint8_t** buffer_mem) {
     if (rc == 0 && *buffer_mem == nullptr && len != 0) rc = ENOMEM;
 #endif
     if (rc != 0) {
-        std::stringstream ss;
-        ss << "posix_memalign() failed to allocate buffer: " << get_str_err_msg();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("posix_memalign() failed to allocate buffer: {}",
+                                     get_str_err_msg());
     }
     if (use_huge_pages) {
 #ifdef MADV_HUGEPAGE
diff --git a/be/src/runtime/collection_value.cpp b/be/src/runtime/collection_value.cpp
index d8581d8ac5..c354c9f1a2 100644
--- a/be/src/runtime/collection_value.cpp
+++ b/be/src/runtime/collection_value.cpp
@@ -394,7 +394,7 @@ Status type_check(PrimitiveType type) {
     case TYPE_ARRAY:
         break;
     default:
-        return Status::InvalidArgument(fmt::format("Type not implemented: {}", type));
+        return Status::InvalidArgument("Type not implemented: {}", type);
     }
     return Status::OK();
 }
diff --git a/be/src/runtime/disk_io_mgr.cc b/be/src/runtime/disk_io_mgr.cc
index 892ce1512c..90a25b0d7a 100644
--- a/be/src/runtime/disk_io_mgr.cc
+++ b/be/src/runtime/disk_io_mgr.cc
@@ -649,10 +649,8 @@ Status DiskIoMgr::read(RequestContext* reader, ScanRange* range, BufferDescripto
     *buffer = nullptr;
 
     if (range->len() > _max_buffer_size) {
-        stringstream error_msg;
-        error_msg << "Cannot perform sync read larger than " << _max_buffer_size << ". Request was "
-                  << range->len();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Cannot perform sync read larger than {}. Request was {}",
+                                     _max_buffer_size, range->len());
     }
 
     vector<DiskIoMgr::ScanRange*> ranges;
@@ -1136,9 +1134,7 @@ void DiskIoMgr::write(RequestContext* writer_context, WriteRange* write_range) {
 
         int success = fclose(file_handle);
         if (ret_status.ok() && success != 0) {
-            stringstream error_msg;
-            error_msg << "fclose(" << write_range->_file << ") failed";
-            ret_status = Status::InternalError(error_msg.str());
+            ret_status = Status::InternalError("fclose({}) failed", write_range->_file);
         }
     }
 
@@ -1149,19 +1145,16 @@ Status DiskIoMgr::write_range_helper(FILE* file_handle, WriteRange* write_range)
     // Seek to the correct offset and perform the write.
     int success = fseek(file_handle, write_range->offset(), SEEK_SET);
     if (success != 0) {
-        stringstream error_msg;
-        error_msg << "fseek(" << write_range->_file << ", " << write_range->offset()
-                  << " SEEK_SET) failed with errno=" << errno
-                  << " description=" << get_str_err_msg();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("fseek({}, {} SEEK_SET) failed with errno={} description={}",
+                                     write_range->_file, write_range->offset(), errno,
+                                     get_str_err_msg());
     }
 
     int64_t bytes_written = fwrite(write_range->_data, 1, write_range->_len, file_handle);
     if (bytes_written < write_range->_len) {
-        stringstream error_msg;
-        error_msg << "fwrite(buffer, 1, " << write_range->_len << ", " << write_range->_file
-                  << ") failed with errno=" << errno << " description=" << get_str_err_msg();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError(
+                "fwrite(buffer, 1, {}, {}) failed with errno={} description={}", write_range->_len,
+                write_range->_file, errno, get_str_err_msg());
     }
 
     return Status::OK();
diff --git a/be/src/runtime/disk_io_mgr_scan_range.cc b/be/src/runtime/disk_io_mgr_scan_range.cc
index 466d872582..82962f4b3d 100644
--- a/be/src/runtime/disk_io_mgr_scan_range.cc
+++ b/be/src/runtime/disk_io_mgr_scan_range.cc
@@ -297,17 +297,14 @@ Status DiskIoMgr::ScanRange::open() {
     _local_file = fopen(file(), "r");
     if (_local_file == nullptr) {
         string error_msg = get_str_err_msg();
-        stringstream ss;
-        ss << "Could not open file: " << _file << ": " << error_msg;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Could not open file: {}: {}", _file, error_msg);
     }
     if (fseek(_local_file, _offset, SEEK_SET) == -1) {
         fclose(_local_file);
         _local_file = nullptr;
         string error_msg = get_str_err_msg();
-        stringstream ss;
-        ss << "Could not seek to " << _offset << " for file: " << _file << ": " << error_msg;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Could not seek to {} for file: {}: {}", _offset, _file,
+                                     error_msg);
     }
     // }
     return Status::OK();
@@ -416,10 +413,8 @@ Status DiskIoMgr::ScanRange::read(char* buffer, int64_t* bytes_read, bool* eosr)
     if (*bytes_read < bytes_to_read) {
         if (ferror(_local_file) != 0) {
             string error_msg = get_str_err_msg();
-            stringstream ss;
-            ss << "Error reading from " << _file << " at byte offset: " << (_offset + _bytes_read)
-               << ": " << error_msg;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Error reading from {} at byte offset: {}: {}", _file,
+                                         (_offset + _bytes_read), error_msg);
         } else {
             // On Linux, we should only get partial reads from block devices on error or eof.
             DCHECK(feof(_local_file) != 0);
diff --git a/be/src/runtime/dpp_sink_internal.cpp b/be/src/runtime/dpp_sink_internal.cpp
index a35297a8ae..dece00cb9b 100644
--- a/be/src/runtime/dpp_sink_internal.cpp
+++ b/be/src/runtime/dpp_sink_internal.cpp
@@ -89,9 +89,7 @@ Status PartRangeKey::from_thrift(ObjectPool* pool, const TPartitionKey& t_key, P
         key->_key = pool->add(new DateTimeValue());
         DateTimeValue* datetime = reinterpret_cast<DateTimeValue*>(key->_key);
         if (!(datetime->from_date_str(t_key.key.c_str(), t_key.key.length()))) {
-            std::stringstream error_msg;
-            error_msg << "Fail to convert date string:" << t_key.key;
-            return Status::InternalError(error_msg.str());
+            return Status::InternalError("Fail to convert date string:{}", t_key.key);
         }
         datetime->cast_to_date();
         break;
@@ -101,9 +99,7 @@ Status PartRangeKey::from_thrift(ObjectPool* pool, const TPartitionKey& t_key, P
         key->_key = pool->add(new DateTimeValue());
         DateTimeValue* datetime = reinterpret_cast<DateTimeValue*>(key->_key);
         if (!(datetime->from_date_str(t_key.key.c_str(), t_key.key.length()))) {
-            std::stringstream error_msg;
-            error_msg << "Fail to convert datetime string:" << t_key.key;
-            return Status::InternalError(error_msg.str());
+            return Status::InternalError("Fail to convert datetime string:{}", t_key.key);
         }
         datetime->to_datetime();
         break;
@@ -114,9 +110,7 @@ Status PartRangeKey::from_thrift(ObjectPool* pool, const TPartitionKey& t_key, P
         break;
     }
     if (parse_result != StringParser::PARSE_SUCCESS) {
-        std::stringstream error_msg;
-        error_msg << "Fail to convert string:" << t_key.key;
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Fail to convert string:{}", t_key.key);
     }
 
     return Status::OK();
diff --git a/be/src/runtime/dpp_writer.cpp b/be/src/runtime/dpp_writer.cpp
index 00a746872d..56971c2bf4 100644
--- a/be/src/runtime/dpp_writer.cpp
+++ b/be/src/runtime/dpp_writer.cpp
@@ -186,9 +186,7 @@ Status DppWriter::append_one_row(TupleRow* row) {
             // write len first
             uint16_t len = str_val->len;
             if (len != str_val->len) {
-                std::stringstream ss;
-                ss << "length of string is overflow.len=" << str_val->len;
-                return Status::InternalError(ss.str());
+                return Status::InternalError("length of string is overflow.len={}", str_val->len);
             }
             append_to_buf(&len, 2);
             // passing a nullptr pointer to memcpy may be core/
diff --git a/be/src/runtime/export_task_mgr.cpp b/be/src/runtime/export_task_mgr.cpp
index 2b1a59fd99..e17df358d6 100644
--- a/be/src/runtime/export_task_mgr.cpp
+++ b/be/src/runtime/export_task_mgr.cpp
@@ -97,9 +97,7 @@ Status ExportTaskMgr::erase_task(const TUniqueId& id) {
     std::lock_guard<std::mutex> l(_lock);
     auto it = _running_tasks.find(id);
     if (it != _running_tasks.end()) {
-        std::stringstream ss;
-        ss << "Task(" << id << ") is running, can not be deleted.";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Task({}) is running, can not be deleted.", id);
     }
     _success_tasks.erase(id);
     _failed_tasks.erase(id);
@@ -127,9 +125,7 @@ Status ExportTaskMgr::finish_task(const TUniqueId& id, const Status& status,
     std::lock_guard<std::mutex> l(_lock);
     auto it = _running_tasks.find(id);
     if (it == _running_tasks.end()) {
-        std::stringstream ss;
-        ss << "Unknown task id(" << id << ").";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unknown task id({}).", id);
     }
     _running_tasks.erase(it);
 
diff --git a/be/src/runtime/file_result_writer.cpp b/be/src/runtime/file_result_writer.cpp
index 8f4e0c0d03..8e7ee645b7 100644
--- a/be/src/runtime/file_result_writer.cpp
+++ b/be/src/runtime/file_result_writer.cpp
@@ -120,8 +120,7 @@ Status FileResultWriter::_create_file_writer(const std::string& file_name) {
                                                    _output_object_data);
         break;
     default:
-        return Status::InternalError(
-                strings::Substitute("unsupported file format: $0", _file_opts->file_format));
+        return Status::InternalError("unsupported file format: {}", _file_opts->file_format);
     }
     LOG(INFO) << "create file for exporting query result. file name: " << file_name
               << ". query id: " << print_id(_state->query_id())
@@ -143,8 +142,8 @@ Status FileResultWriter::_get_next_file_name(std::string* file_name) {
         // Doris is not responsible for ensuring the correctness of the path.
         // This is just to prevent overwriting the existing file.
         if (FileUtils::check_exist(*file_name)) {
-            return Status::InternalError("File already exists: " + *file_name +
-                                         ". Host: " + BackendOptions::get_localhost());
+            return Status::InternalError("File already exists: {}. Host: {}", *file_name,
+                                         BackendOptions::get_localhost());
         }
     }
 
diff --git a/be/src/runtime/fragment_mgr.cpp b/be/src/runtime/fragment_mgr.cpp
index 686ece037e..29a5087635 100644
--- a/be/src/runtime/fragment_mgr.cpp
+++ b/be/src/runtime/fragment_mgr.cpp
@@ -545,9 +545,9 @@ Status FragmentMgr::start_query_execution(const PExecPlanFragmentStartRequest* r
     auto search = _fragments_ctx_map.find(query_id);
     if (search == _fragments_ctx_map.end()) {
         return Status::InternalError(
-                strings::Substitute("Failed to get query fragments context. Query may be "
-                                    "timeout or be cancelled. host: ",
-                                    BackendOptions::get_localhost()));
+                "Failed to get query fragments context. Query may be "
+                "timeout or be cancelled. host: {}",
+                BackendOptions::get_localhost());
     }
     search->second->set_ready_to_execute();
     return Status::OK();
@@ -595,9 +595,9 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, Fi
         auto search = _fragments_ctx_map.find(params.params.query_id);
         if (search == _fragments_ctx_map.end()) {
             return Status::InternalError(
-                    strings::Substitute("Failed to get query fragments context. Query may be "
-                                        "timeout or be cancelled. host: ",
-                                        BackendOptions::get_localhost()));
+                    "Failed to get query fragments context. Query may be "
+                    "timeout or be cancelled. host: {}",
+                    BackendOptions::get_localhost());
         }
         fragments_ctx = search->second;
     } else {
@@ -666,9 +666,8 @@ Status FragmentMgr::exec_plan_fragment(const TExecPlanFragmentParams& params, Fi
             _fragment_map.erase(params.params.fragment_instance_id);
         }
         exec_state->cancel_before_execute();
-        return Status::InternalError(
-                strings::Substitute("Put planfragment to thread pool failed. err = $0, BE: $1",
-                                    st.get_error_msg(), BackendOptions::get_localhost()));
+        return Status::InternalError("Put planfragment to thread pool failed. err = {}, BE: {}",
+                                     st.get_error_msg(), BackendOptions::get_localhost());
     }
 
     return Status::OK();
@@ -873,7 +872,7 @@ Status FragmentMgr::apply_filter(const PPublishFilterRequest* request, const cha
         auto iter = _fragment_map.find(tfragment_instance_id);
         if (iter == _fragment_map.end()) {
             VLOG_CRITICAL << "unknown.... fragment-id:" << fragment_instance_id;
-            return Status::InvalidArgument("fragment-id: " + fragment_instance_id.to_string());
+            return Status::InvalidArgument("fragment-id: {}", fragment_instance_id.to_string());
         }
         fragment_state = iter->second;
     }
diff --git a/be/src/runtime/load_channel_mgr.h b/be/src/runtime/load_channel_mgr.h
index 39d7ed5b2b..77a046e8a8 100644
--- a/be/src/runtime/load_channel_mgr.h
+++ b/be/src/runtime/load_channel_mgr.h
@@ -105,8 +105,8 @@ Status LoadChannelMgr::_get_load_channel(std::shared_ptr<LoadChannel>& channel,
                 return Status::OK();
             }
         }
-        return Status::InternalError(strings::Substitute(
-                "fail to add batch in load channel. unknown load_id=$0", load_id.to_string()));
+        return Status::InternalError("fail to add batch in load channel. unknown load_id={}",
+                                     load_id.to_string());
     }
     channel = it->second;
     return Status::OK();
diff --git a/be/src/runtime/mem_tracker.h b/be/src/runtime/mem_tracker.h
index 684f23ec9a..e69f6c80ad 100644
--- a/be/src/runtime/mem_tracker.h
+++ b/be/src/runtime/mem_tracker.h
@@ -113,9 +113,9 @@ public:
 
     Status check_sys_mem_info(int64_t bytes) {
         if (MemInfo::initialized() && MemInfo::current_mem() + bytes >= MemInfo::mem_limit()) {
-            return Status::MemoryLimitExceeded(fmt::format(
+            return Status::MemoryLimitExceeded(
                     "{}: TryConsume failed, bytes={} process whole consumption={}  mem limit={}",
-                    _label, bytes, MemInfo::current_mem(), MemInfo::mem_limit()));
+                    _label, bytes, MemInfo::current_mem(), MemInfo::mem_limit());
         }
         return Status::OK();
     }
@@ -474,8 +474,8 @@ private:
     inline Status try_gc_memory(int64_t bytes) {
         if (UNLIKELY(gc_memory(_limit - bytes))) {
             return Status::MemoryLimitExceeded(
-                    fmt::format("label={} TryConsume failed size={}, used={}, limit={}", label(),
-                                bytes, _consumption->current_value(), _limit));
+                    "label={} TryConsume failed size={}, used={}, limit={}", label(), bytes,
+                    _consumption->current_value(), _limit);
         }
         VLOG_NOTICE << "GC succeeded, TryConsume bytes=" << bytes
                     << " consumption=" << _consumption->current_value() << " limit=" << _limit;
diff --git a/be/src/runtime/memory/chunk_allocator.cpp b/be/src/runtime/memory/chunk_allocator.cpp
index 872b6c37f8..a07a967ed4 100644
--- a/be/src/runtime/memory/chunk_allocator.cpp
+++ b/be/src/runtime/memory/chunk_allocator.cpp
@@ -191,8 +191,7 @@ Status ChunkAllocator::allocate(size_t size, Chunk* chunk, MemTracker* tracker,
     if (chunk->data == nullptr) {
         // allocate fails, return this part of the memory to the parameter tracker.
         reset_tracker->transfer_to(_mem_tracker.get(), size);
-        return Status::MemoryAllocFailed(
-                fmt::format("ChunkAllocator failed to allocate chunk {} bytes", size));
+        return Status::MemoryAllocFailed("ChunkAllocator failed to allocate chunk {} bytes", size);
     }
     return Status::OK();
 }
diff --git a/be/src/runtime/mysql_table_writer.cpp b/be/src/runtime/mysql_table_writer.cpp
index def4845a4f..3f921db2a4 100644
--- a/be/src/runtime/mysql_table_writer.cpp
+++ b/be/src/runtime/mysql_table_writer.cpp
@@ -146,10 +146,8 @@ Status MysqlTableWriter::insert_row(TupleRow* row) {
         }
 
         default: {
-            std::stringstream err_ss;
-            err_ss << "can't convert this type to mysql type. type = "
-                   << _output_expr_ctxs[i]->root()->type();
-            return Status::InternalError(err_ss.str());
+            return Status::InternalError("can't convert this type to mysql type. type = {}",
+                                         _output_expr_ctxs[i]->root()->type());
         }
         }
     }
diff --git a/be/src/runtime/routine_load/data_consumer.cpp b/be/src/runtime/routine_load/data_consumer.cpp
index 5573955fd1..417d37c77c 100644
--- a/be/src/runtime/routine_load/data_consumer.cpp
+++ b/be/src/runtime/routine_load/data_consumer.cpp
@@ -94,10 +94,8 @@ Status KafkaDataConsumer::init(StreamLoadContext* ctx) {
             std::string file_path;
             Status st = ctx->exec_env()->small_file_mgr()->get_file(file_id, parts[2], &file_path);
             if (!st.ok()) {
-                std::stringstream ss;
-                ss << "PAUSE: failed to get file for config: " << item.first
-                   << ", error: " << st.get_error_msg();
-                return Status::InternalError(ss.str());
+                return Status::InternalError("PAUSE: failed to get file for config: {}, error: {}",
+                                             item.first, st.get_error_msg());
             }
             RETURN_IF_ERROR(set_conf(item.first, file_path));
         } else {
@@ -406,9 +404,7 @@ Status KafkaDataConsumer::commit(std::vector<RdKafka::TopicPartition*>& offset)
     // Commit failure has no effect on Doris, subsequent tasks will continue to commit the new offset
     RdKafka::ErrorCode err = _k_consumer->commitAsync(offset);
     if (err != RdKafka::ERR_NO_ERROR) {
-        std::stringstream ss;
-        ss << "failed to commit kafka offset : " << RdKafka::err2str(err);
-        return Status::InternalError(ss.str());
+        return Status::InternalError("failed to commit kafka offset : {}", RdKafka::err2str(err));
     }
     return Status::OK();
 }
diff --git a/be/src/runtime/routine_load/data_consumer_pool.cpp b/be/src/runtime/routine_load/data_consumer_pool.cpp
index a134697c00..31f666d3ab 100644
--- a/be/src/runtime/routine_load/data_consumer_pool.cpp
+++ b/be/src/runtime/routine_load/data_consumer_pool.cpp
@@ -47,9 +47,7 @@ Status DataConsumerPool::get_consumer(StreamLoadContext* ctx, std::shared_ptr<Da
         consumer = std::make_shared<KafkaDataConsumer>(ctx);
         break;
     default:
-        std::stringstream ss;
-        ss << "PAUSE: unknown routine load task type: " << ctx->load_type;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("PAUSE: unknown routine load task type: {}", ctx->load_type);
     }
 
     // init the consumer
diff --git a/be/src/runtime/routine_load/routine_load_task_executor.cpp b/be/src/runtime/routine_load/routine_load_task_executor.cpp
index 076cbe33d6..66a75a8c4e 100644
--- a/be/src/runtime/routine_load/routine_load_task_executor.cpp
+++ b/be/src/runtime/routine_load/routine_load_task_executor.cpp
@@ -161,7 +161,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
                   << ", job id: " << task.job_id
                   << ", queue size: " << _thread_pool.get_queue_size()
                   << ", current tasks num: " << _task_map.size();
-        return Status::TooManyTasks(UniqueId(task.id).to_string() + "_" +
+        return Status::TooManyTasks("{}_{}", UniqueId(task.id).to_string(),
                                     BackendOptions::get_localhost());
     }
 
diff --git a/be/src/runtime/row_batch.cpp b/be/src/runtime/row_batch.cpp
index 74a79a6db6..42aad8da09 100644
--- a/be/src/runtime/row_batch.cpp
+++ b/be/src/runtime/row_batch.cpp
@@ -311,8 +311,8 @@ Status RowBatch::serialize(PRowBatch* output_batch, size_t* uncompressed_size,
     *compressed_size = pb_size;
     if (!allow_transfer_large_data && pb_size > std::numeric_limits<int32_t>::max()) {
         // the protobuf has a hard limit of 2GB for serialized data.
-        return Status::InternalError(fmt::format(
-                "The rowbatch is large than 2GB({}), can not send by Protobuf.", pb_size));
+        return Status::InternalError(
+                "The rowbatch is large than 2GB({}), can not send by Protobuf.", pb_size);
     }
     return Status::OK();
 }
diff --git a/be/src/runtime/small_file_mgr.cpp b/be/src/runtime/small_file_mgr.cpp
index aa9d65ab01..504d25a889 100644
--- a/be/src/runtime/small_file_mgr.cpp
+++ b/be/src/runtime/small_file_mgr.cpp
@@ -77,20 +77,19 @@ Status SmallFileMgr::_load_single_file(const std::string& path, const std::strin
     // file_id.md5
     std::vector<std::string> parts = strings::Split(file_name, ".");
     if (parts.size() != 2) {
-        return Status::InternalError("Not a valid file name: " + file_name);
+        return Status::InternalError("Not a valid file name: {}", file_name);
     }
     int64_t file_id = std::stol(parts[0]);
     std::string md5 = parts[1];
 
     if (_file_cache.find(file_id) != _file_cache.end()) {
-        return Status::InternalError(
-                fmt::format("File with same id is already been loaded: {}", file_id));
+        return Status::InternalError("File with same id is already been loaded: {}", file_id);
     }
 
     std::string file_md5;
     RETURN_IF_ERROR(FileUtils::md5sum(path + "/" + file_name, &file_md5));
     if (file_md5 != md5) {
-        return Status::InternalError("Invalid md5 of file: " + file_name);
+        return Status::InternalError("Invalid md5 of file: {}", file_name);
     }
 
     CacheEntry entry;
@@ -112,9 +111,8 @@ Status SmallFileMgr::get_file(int64_t file_id, const std::string& md5, std::stri
         if (!st.ok()) {
             // check file failed, we should remove this cache and download it from FE again
             if (remove(entry.path.c_str()) != 0) {
-                std::stringstream ss;
-                ss << "failed to remove file: " << file_id << ", err: " << std::strerror(errno);
-                return Status::InternalError(ss.str());
+                return Status::InternalError("failed to remove file: {}, err: {}", file_id,
+                                             std::strerror(errno));
             }
             _file_cache.erase(it);
         } else {
diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp
index 2c0178c872..0545e7de8a 100644
--- a/be/src/runtime/snapshot_loader.cpp
+++ b/be/src/runtime/snapshot_loader.cpp
@@ -507,7 +507,7 @@ Status SnapshotLoader::_get_tablet_id_and_schema_hash_from_file_path(const std::
     // we try to extract tablet_id from path
     size_t pos = src_path.find_last_of("/");
     if (pos == std::string::npos || pos == src_path.length() - 1) {
-        return Status::InternalError("failed to get tablet id from path: " + src_path);
+        return Status::InternalError("failed to get tablet id from path: {}", src_path);
     }
 
     std::string schema_hash_str = src_path.substr(pos + 1);
@@ -518,7 +518,7 @@ Status SnapshotLoader::_get_tablet_id_and_schema_hash_from_file_path(const std::
     // skip schema hash part
     size_t pos2 = src_path.find_last_of("/", pos - 1);
     if (pos2 == std::string::npos) {
-        return Status::InternalError("failed to get tablet id from path: " + src_path);
+        return Status::InternalError("failed to get tablet id from path: {}", src_path);
     }
 
     std::string tablet_str = src_path.substr(pos2 + 1, pos - pos2);
@@ -581,7 +581,7 @@ Status SnapshotLoader::_replace_tablet_id(const std::string& file_name, int64_t
         *new_file_name = file_name;
         return Status::OK();
     } else {
-        return Status::InternalError("invalid tablet file name: " + file_name);
+        return Status::InternalError("invalid tablet file name: {}", file_name);
     }
 }
 
@@ -591,7 +591,7 @@ Status SnapshotLoader::_get_tablet_id_from_remote_path(const std::string& remote
     // bos://xxx/../__tbl_10004/__part_10003/__idx_10004/__10005
     size_t pos = remote_path.find_last_of("_");
     if (pos == std::string::npos) {
-        return Status::InternalError("invalid remove file path: " + remote_path);
+        return Status::InternalError("invalid remove file path: {}", remote_path);
     }
 
     std::string tablet_id_str = remote_path.substr(pos + 1);
diff --git a/be/src/runtime/stream_load/stream_load_pipe.h b/be/src/runtime/stream_load/stream_load_pipe.h
index 716a145f61..dd86c24dd2 100644
--- a/be/src/runtime/stream_load/stream_load_pipe.h
+++ b/be/src/runtime/stream_load/stream_load_pipe.h
@@ -91,9 +91,7 @@ public:
     // Otherwise, this should be a stream load task that needs to read the specified amount of data.
     Status read_one_message(std::unique_ptr<uint8_t[]>* data, int64_t* length) override {
         if (_total_length < -1) {
-            std::stringstream ss;
-            ss << "invalid, _total_length is: " << _total_length;
-            return Status::InternalError(ss.str());
+            return Status::InternalError("invalid, _total_length is: {}", _total_length);
         } else if (_total_length == 0) {
             // no data
             *length = 0;
@@ -123,7 +121,7 @@ public:
             }
             // cancelled
             if (_cancelled) {
-                return Status::InternalError("cancelled: " + _cancelled_reason);
+                return Status::InternalError("cancelled: {}", _cancelled_reason);
             }
             // finished
             if (_buf_queue.empty()) {
@@ -198,7 +196,7 @@ private:
         }
         // cancelled
         if (_cancelled) {
-            return Status::InternalError("cancelled: " + _cancelled_reason);
+            return Status::InternalError("cancelled: {}", _cancelled_reason);
         }
         // finished
         if (_buf_queue.empty()) {
@@ -238,7 +236,7 @@ private:
                 }
             }
             if (_cancelled) {
-                return Status::InternalError("cancelled: " + _cancelled_reason);
+                return Status::InternalError("cancelled: {}", _cancelled_reason);
             }
             _buf_queue.push_back(buf);
             if (_use_proto) {
diff --git a/be/src/runtime/stream_load/stream_load_recorder.cpp b/be/src/runtime/stream_load/stream_load_recorder.cpp
index 46d0225e81..e1d082ed18 100644
--- a/be/src/runtime/stream_load/stream_load_recorder.cpp
+++ b/be/src/runtime/stream_load/stream_load_recorder.cpp
@@ -58,7 +58,7 @@ Status StreamLoadRecorder::init() {
             rocksdb::DBWithTTL::Open(options, db_path, column_families, &_handles, &_db, ttls);
     if (!s.ok() || _db == nullptr) {
         LOG(WARNING) << "rocks db open failed, reason:" << s.ToString();
-        return Status::InternalError("Stream load record rocksdb open failed, reason: " +
+        return Status::InternalError("Stream load record rocksdb open failed, reason: {}",
                                      s.ToString());
     }
     return Status::OK();
@@ -71,7 +71,7 @@ Status StreamLoadRecorder::put(const std::string& key, const std::string& value)
     rocksdb::Status s = _db->Put(write_options, handle, rocksdb::Slice(key), rocksdb::Slice(value));
     if (!s.ok()) {
         LOG(WARNING) << "rocks db put key:" << key << " failed, reason:" << s.ToString();
-        return Status::InternalError("Stream load record rocksdb put failed, reason: " +
+        return Status::InternalError("Stream load record rocksdb put failed, reason: {}",
                                      s.ToString());
     }
 
diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp
index 2e3945bb80..9733ff0bc7 100644
--- a/be/src/runtime/tablets_channel.cpp
+++ b/be/src/runtime/tablets_channel.cpp
@@ -201,8 +201,8 @@ Status TabletsChannel::reduce_mem_usage(int64_t mem_limit) {
     for (int i = 0; i < counter; i++) {
         Status st = writers[i]->wait_flush();
         if (!st.ok()) {
-            return Status::InternalError(fmt::format(
-                    "failed to reduce mem consumption by flushing memtable. err: {}", st));
+            return Status::InternalError(
+                    "failed to reduce mem consumption by flushing memtable. err: {}", st);
         }
     }
     return Status::OK();
diff --git a/be/src/runtime/tablets_channel.h b/be/src/runtime/tablets_channel.h
index 7c3bb3300c..5ad1b47027 100644
--- a/be/src/runtime/tablets_channel.h
+++ b/be/src/runtime/tablets_channel.h
@@ -154,10 +154,9 @@ template <typename Request>
 Status TabletsChannel::_get_current_seq(int64_t& cur_seq, const Request& request) {
     std::lock_guard<std::mutex> l(_lock);
     if (_state != kOpened) {
-        return _state == kFinished
-                       ? _close_status
-                       : Status::InternalError(strings::Substitute("TabletsChannel $0 state: $1",
-                                                                   _key.to_string(), _state));
+        return _state == kFinished ? _close_status
+                                   : Status::InternalError("TabletsChannel {} state: {}",
+                                                           _key.to_string(), _state);
     }
     cur_seq = _next_seqs[request.sender_id()];
     // check packet
@@ -215,8 +214,8 @@ Status TabletsChannel::add_batch(const TabletWriterAddRequest& request,
     for (const auto& tablet_to_rowidxs_it : tablet_to_rowidxs) {
         auto tablet_writer_it = _tablet_writers.find(tablet_to_rowidxs_it.first);
         if (tablet_writer_it == _tablet_writers.end()) {
-            return Status::InternalError(strings::Substitute(
-                    "unknown tablet to append data, tablet=$0", tablet_to_rowidxs_it.first));
+            return Status::InternalError("unknown tablet to append data, tablet={}",
+                                         tablet_to_rowidxs_it.first);
         }
 
         Status st = tablet_writer_it->second->write(&send_data, tablet_to_rowidxs_it.second);
diff --git a/be/src/runtime/tmp_file_mgr.cc b/be/src/runtime/tmp_file_mgr.cc
index 16e3649bd1..639d411b13 100644
--- a/be/src/runtime/tmp_file_mgr.cc
+++ b/be/src/runtime/tmp_file_mgr.cc
@@ -138,9 +138,7 @@ Status TmpFileMgr::get_file(const DeviceId& device_id, const TUniqueId& query_id
     DCHECK_GE(device_id, 0);
     DCHECK_LT(device_id, _tmp_dirs.size());
     if (is_blacklisted(device_id)) {
-        std::stringstream error_msg;
-        error_msg << "path is blacklist. path: " << _tmp_dirs[device_id].path();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("path is blacklist. path: {}", _tmp_dirs[device_id].path());
     }
 
     // Generate the full file path.
@@ -224,9 +222,7 @@ Status TmpFileMgr::File::allocate_space(int64_t write_size, int64_t* offset) {
     Status status;
     if (_mgr->is_blacklisted(_device_id)) {
         _blacklisted = true;
-        std::stringstream error_msg;
-        error_msg << "path is blacklist. path: " << _path;
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("path is blacklist. path: {}", _path);
     }
     if (_current_size == 0) {
         // First call to AllocateSpace. Create the file.
diff --git a/be/src/tools/meta_tool.cpp b/be/src/tools/meta_tool.cpp
index f4e5e6087c..376f059759 100644
--- a/be/src/tools/meta_tool.cpp
+++ b/be/src/tools/meta_tool.cpp
@@ -258,8 +258,7 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer)
     RETURN_IF_ERROR(input_file->size(&file_size));
 
     if (file_size < 12) {
-        return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < 12",
-                                                      file_name, file_size));
+        return Status::Corruption("Bad segment file {}: file size {} < 12", file_name, file_size);
     }
 
     uint8_t fixed_buf[12];
@@ -270,15 +269,14 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer)
     const char* k_segment_magic = "D0R1";
     const uint32_t k_segment_magic_length = 4;
     if (memcmp(fixed_buf + 8, k_segment_magic, k_segment_magic_length) != 0) {
-        return Status::Corruption(
-                strings::Substitute("Bad segment file $0: magic number not match", file_name));
+        return Status::Corruption("Bad segment file {}: magic number not match", file_name);
     }
 
     // read footer PB
     uint32_t footer_length = doris::decode_fixed32_le(fixed_buf);
     if (file_size < 12 + footer_length) {
-        return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < $2",
-                                                      file_name, file_size, 12 + footer_length));
+        return Status::Corruption("Bad segment file {}: file size {} < {}", file_name, file_size,
+                                  12 + footer_length);
     }
     std::string footer_buf;
     footer_buf.resize(footer_length);
@@ -289,15 +287,15 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer)
     uint32_t expect_checksum = doris::decode_fixed32_le(fixed_buf + 4);
     uint32_t actual_checksum = doris::crc32c::Value(footer_buf.data(), footer_buf.size());
     if (actual_checksum != expect_checksum) {
-        return Status::Corruption(strings::Substitute(
-                "Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", file_name,
-                actual_checksum, expect_checksum));
+        return Status::Corruption(
+                "Bad segment file {}: footer checksum not match, actual={} vs expect={}", file_name,
+                actual_checksum, expect_checksum);
     }
 
     // deserialize footer PB
     if (!footer->ParseFromString(footer_buf)) {
-        return Status::Corruption(strings::Substitute(
-                "Bad segment file $0: failed to parse SegmentFooterPB", file_name));
+        return Status::Corruption("Bad segment file {}: failed to parse SegmentFooterPB",
+                                  file_name);
     }
     return Status::OK();
 }
diff --git a/be/src/util/array_parser.h b/be/src/util/array_parser.h
index bfb214a51e..7bcf122c34 100644
--- a/be/src/util/array_parser.h
+++ b/be/src/util/array_parser.h
@@ -229,8 +229,8 @@ private:
             break;
         }
         default:
-            return Status::RuntimeError("Failed to parse json to type (" +
-                                        std::to_string(type_desc.type) + ").");
+            return Status::RuntimeError("Failed to parse json to type ({}).",
+                                        std::to_string(type_desc.type));
         }
         return Status::OK();
     }
diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp
index fe834289d7..01ce8d4597 100644
--- a/be/src/util/arrow/row_batch.cpp
+++ b/be/src/util/arrow/row_batch.cpp
@@ -87,8 +87,7 @@ Status convert_to_arrow_type(const TypeDescriptor& type, std::shared_ptr<arrow::
         *result = arrow::boolean();
         break;
     default:
-        return Status::InvalidArgument(
-                strings::Substitute("Unknown primitive type($0)", type.type));
+        return Status::InvalidArgument("Unknown primitive type({})", type.type);
     }
     return Status::OK();
 }
@@ -138,7 +137,7 @@ Status convert_to_doris_type(const arrow::DataType& type, TSlotDescriptorBuilder
         builder->type(TYPE_BOOLEAN);
         break;
     default:
-        return Status::InvalidArgument(strings::Substitute("Unknown arrow type id($0)", type.id()));
+        return Status::InvalidArgument("Unknown arrow type id({})", type.id());
     }
     return Status::OK();
 }
@@ -462,51 +461,40 @@ Status serialize_record_batch(const arrow::RecordBatch& record_batch, std::strin
     int64_t capacity;
     arrow::Status a_st = arrow::ipc::GetRecordBatchSize(record_batch, &capacity);
     if (!a_st.ok()) {
-        std::stringstream msg;
-        msg << "GetRecordBatchSize failure, reason: " << a_st.ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("GetRecordBatchSize failure, reason: {}", a_st.ToString());
     }
     auto sink_res = arrow::io::BufferOutputStream::Create(capacity, arrow::default_memory_pool());
     if (!sink_res.ok()) {
-        std::stringstream msg;
-        msg << "create BufferOutputStream failure, reason: " << sink_res.status().ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("create BufferOutputStream failure, reason: {}",
+                                     sink_res.status().ToString());
     }
     std::shared_ptr<arrow::io::BufferOutputStream> sink = sink_res.ValueOrDie();
     // create RecordBatch Writer
     auto res = arrow::ipc::MakeStreamWriter(sink.get(), record_batch.schema());
     if (!res.ok()) {
-        std::stringstream msg;
-        msg << "open RecordBatchStreamWriter failure, reason: " << res.status().ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("open RecordBatchStreamWriter failure, reason: {}",
+                                     res.status().ToString());
     }
     // write RecordBatch to memory buffer outputstream
     std::shared_ptr<arrow::ipc::RecordBatchWriter> record_batch_writer = res.ValueOrDie();
     a_st = record_batch_writer->WriteRecordBatch(record_batch);
     if (!a_st.ok()) {
-        std::stringstream msg;
-        msg << "write record batch failure, reason: " << a_st.ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("write record batch failure, reason: {}", a_st.ToString());
     }
     a_st = record_batch_writer->Close();
     if (!a_st.ok()) {
-        std::stringstream msg;
-        msg << "Close failed, reason: " << a_st.ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("Close failed, reason: {}", a_st.ToString());
     }
     auto finish_res = sink->Finish();
     if (!finish_res.ok()) {
-        std::stringstream msg;
-        msg << "allocate result buffer failure, reason: " << finish_res.status().ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("allocate result buffer failure, reason: {}",
+                                     finish_res.status().ToString());
     }
     *result = finish_res.ValueOrDie()->ToString();
     // close the sink
     a_st = sink->Close();
     if (!a_st.ok()) {
-        std::stringstream msg;
-        msg << "Close failed, reason: " << a_st.ToString();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("Close failed, reason: {}", a_st.ToString());
     }
     return Status::OK();
 }
diff --git a/be/src/util/arrow/row_block.cpp b/be/src/util/arrow/row_block.cpp
index 1ed04cd910..680b724974 100644
--- a/be/src/util/arrow/row_block.cpp
+++ b/be/src/util/arrow/row_block.cpp
@@ -60,7 +60,7 @@ Status convert_to_arrow_type(FieldType type, std::shared_ptr<arrow::DataType>* r
         *result = arrow::float64();
         break;
     default:
-        return Status::InvalidArgument(strings::Substitute("Unknown FieldType($0)", type));
+        return Status::InvalidArgument("Unknown FieldType({})", type);
     }
     return Status::OK();
 }
@@ -106,7 +106,7 @@ Status convert_to_type_name(const arrow::DataType& type, std::string* name) {
         *name = "DOUBLE";
         break;
     default:
-        return Status::InvalidArgument(strings::Substitute("Unknown arrow type id($0)", type.id()));
+        return Status::InvalidArgument("Unknown arrow type id({})", type.id());
     }
     return Status::OK();
 }
diff --git a/be/src/util/block_compression.cpp b/be/src/util/block_compression.cpp
index 01c3734948..4279080b21 100644
--- a/be/src/util/block_compression.cpp
+++ b/be/src/util/block_compression.cpp
@@ -61,8 +61,8 @@ public:
         auto compressed_len =
                 LZ4_compress_default(input.data, output->data, input.size, output->size);
         if (compressed_len == 0) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Output buffer's capacity is not enough, size=$0", output->size));
+            return Status::InvalidArgument("Output buffer's capacity is not enough, size={}",
+                                           output->size);
         }
         output->size = compressed_len;
         return Status::OK();
@@ -72,8 +72,7 @@ public:
         auto decompressed_len =
                 LZ4_decompress_safe(input.data, output->data, input.size, output->size);
         if (decompressed_len < 0) {
-            return Status::InvalidArgument(
-                    strings::Substitute("fail to do LZ4 decompress, error=$0", decompressed_len));
+            return Status::InvalidArgument("fail to do LZ4 decompress, error={}", decompressed_len);
         }
         output->size = decompressed_len;
         return Status::OK();
@@ -145,23 +144,23 @@ private:
                      Slice* output) const {
         auto wbytes = LZ4F_compressBegin(ctx, output->data, output->size, &_s_preferences);
         if (LZ4F_isError(wbytes)) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Fail to do LZ4F compress begin, res=$0", LZ4F_getErrorName(wbytes)));
+            return Status::InvalidArgument("Fail to do LZ4F compress begin, res={}",
+                                           LZ4F_getErrorName(wbytes));
         }
         size_t offset = wbytes;
         for (auto input : inputs) {
             wbytes = LZ4F_compressUpdate(ctx, output->data + offset, output->size - offset,
                                          input.data, input.size, nullptr);
             if (LZ4F_isError(wbytes)) {
-                return Status::InvalidArgument(strings::Substitute(
-                        "Fail to do LZ4F compress update, res=$0", LZ4F_getErrorName(wbytes)));
+                return Status::InvalidArgument("Fail to do LZ4F compress update, res={}",
+                                               LZ4F_getErrorName(wbytes));
             }
             offset += wbytes;
         }
         wbytes = LZ4F_compressEnd(ctx, output->data + offset, output->size - offset, nullptr);
         if (LZ4F_isError(wbytes)) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Fail to do LZ4F compress end, res=$0", LZ4F_getErrorName(wbytes)));
+            return Status::InvalidArgument("Fail to do LZ4F compress end, res={}",
+                                           LZ4F_getErrorName(wbytes));
         }
         offset += wbytes;
         output->size = offset;
@@ -175,16 +174,16 @@ private:
         auto lres =
                 LZ4F_decompress(ctx, output->data, &output->size, input.data, &input_size, nullptr);
         if (LZ4F_isError(lres)) {
-            return Status::InvalidArgument(strings::Substitute("Fail to do LZ4F decompress, res=$0",
-                                                               LZ4F_getErrorName(lres)));
+            return Status::InvalidArgument("Fail to do LZ4F decompress, res={}",
+                                           LZ4F_getErrorName(lres));
         } else if (input_size != input.size) {
             return Status::InvalidArgument(
                     strings::Substitute("Fail to do LZ4F decompress: trailing data left in "
                                         "compressed data, read=$0 vs given=$1",
                                         input_size, input.size));
         } else if (lres != 0) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Fail to do LZ4F decompress: expect more compressed data, expect=$0", lres));
+            return Status::InvalidArgument(
+                    "Fail to do LZ4F decompress: expect more compressed data, expect={}", lres);
         }
         return Status::OK();
     }
@@ -317,8 +316,7 @@ public:
     Status compress(const Slice& input, Slice* output) const override {
         auto zres = ::compress((Bytef*)output->data, &output->size, (Bytef*)input.data, input.size);
         if (zres != Z_OK) {
-            return Status::InvalidArgument(
-                    strings::Substitute("Fail to do ZLib compress, error=$0", zError(zres)));
+            return Status::InvalidArgument("Fail to do ZLib compress, error={}", zError(zres));
         }
         return Status::OK();
     }
@@ -330,8 +328,8 @@ public:
         zstrm.opaque = Z_NULL;
         auto zres = deflateInit(&zstrm, Z_DEFAULT_COMPRESSION);
         if (zres != Z_OK) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Fail to do ZLib stream compress, error=$0, res=$1", zError(zres), zres));
+            return Status::InvalidArgument("Fail to do ZLib stream compress, error={}, res={}",
+                                           zError(zres), zres);
         }
         // we assume that output is e
         zstrm.next_out = (Bytef*)output->data;
@@ -346,16 +344,16 @@ public:
 
             zres = deflate(&zstrm, flush);
             if (zres != Z_OK && zres != Z_STREAM_END) {
-                return Status::InvalidArgument(strings::Substitute(
-                        "Fail to do ZLib stream compress, error=$0, res=$1", zError(zres), zres));
+                return Status::InvalidArgument("Fail to do ZLib stream compress, error={}, res={}",
+                                               zError(zres), zres);
             }
         }
 
         output->size = zstrm.total_out;
         zres = deflateEnd(&zstrm);
         if (zres != Z_OK) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Fail to do deflateEnd on ZLib stream, error=$0, res=$1", zError(zres), zres));
+            return Status::InvalidArgument("Fail to do deflateEnd on ZLib stream, error={}, res={}",
+                                           zError(zres), zres);
         }
         return Status::OK();
     }
@@ -365,8 +363,7 @@ public:
         auto zres =
                 ::uncompress2((Bytef*)output->data, &output->size, (Bytef*)input.data, &input_size);
         if (zres != Z_OK) {
-            return Status::InvalidArgument(
-                    strings::Substitute("Fail to do ZLib decompress, error=$0", zError(zres)));
+            return Status::InvalidArgument("Fail to do ZLib decompress, error={}", zError(zres));
         }
         return Status::OK();
     }
@@ -424,22 +421,20 @@ public:
         // reset ctx to start new compress session
         auto ret = ZSTD_CCtx_reset(ctx_c, ZSTD_reset_session_only);
         if (ZSTD_isError(ret)) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "ZSTD_CCtx_reset error: $0", ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+            return Status::InvalidArgument("ZSTD_CCtx_reset error: {}",
+                                           ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
         }
         // set compression level to default 3
         ret = ZSTD_CCtx_setParameter(ctx_c, ZSTD_c_compressionLevel, ZSTD_CLEVEL_DEFAULT);
         if (ZSTD_isError(ret)) {
-            return Status::InvalidArgument(
-                    strings::Substitute("ZSTD_CCtx_setParameter compression level error: $0",
-                                        ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+            return Status::InvalidArgument("ZSTD_CCtx_setParameter compression level error: {}",
+                                           ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
         }
         // set checksum flag to 1
         ret = ZSTD_CCtx_setParameter(ctx_c, ZSTD_c_checksumFlag, 1);
         if (ZSTD_isError(ret)) {
-            return Status::InvalidArgument(
-                    strings::Substitute("ZSTD_CCtx_setParameter checksumFlag error: $0",
-                                        ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+            return Status::InvalidArgument("ZSTD_CCtx_setParameter checksumFlag error: {}",
+                                           ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
         }
 
         ZSTD_outBuffer out_buf = {output->data, output->size, 0};
@@ -456,15 +451,13 @@ public:
                 auto ret = ZSTD_compressStream2(ctx_c, &out_buf, &in_buf, mode);
 
                 if (ZSTD_isError(ret)) {
-                    return Status::InvalidArgument(
-                            strings::Substitute("ZSTD_compressStream2 error: $0",
-                                                ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+                    return Status::InvalidArgument("ZSTD_compressStream2 error: {}",
+                                                   ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
                 }
 
                 // ret is ZSTD hint for needed output buffer size
                 if (ret > 0 && out_buf.pos == out_buf.size) {
-                    return Status::InvalidArgument(
-                            strings::Substitute("ZSTD_compressStream2 output buffer full"));
+                    return Status::InvalidArgument("ZSTD_compressStream2 output buffer full");
                 }
 
                 finished = last_input ? (ret == 0) : (in_buf.pos == inputs[i].size);
@@ -485,8 +478,8 @@ public:
         // reset ctx to start a new decompress session
         auto ret = ZSTD_DCtx_reset(ctx_d, ZSTD_reset_session_only);
         if (ZSTD_isError(ret)) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "ZSTD_DCtx_reset error: $0", ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+            return Status::InvalidArgument("ZSTD_DCtx_reset error: {}",
+                                           ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
         }
 
         ZSTD_inBuffer in_buf = {input.data, input.size, 0};
@@ -497,15 +490,13 @@ public:
             auto ret = ZSTD_decompressStream(ctx_d, &out_buf, &in_buf);
 
             if (ZSTD_isError(ret)) {
-                return Status::InvalidArgument(
-                        strings::Substitute("ZSTD_decompressStream error: $0",
-                                            ZSTD_getErrorString(ZSTD_getErrorCode(ret))));
+                return Status::InvalidArgument("ZSTD_decompressStream error: {}",
+                                               ZSTD_getErrorString(ZSTD_getErrorCode(ret)));
             }
 
             // ret is ZSTD hint for needed output buffer size
             if (ret > 0 && out_buf.pos == out_buf.size) {
-                return Status::InvalidArgument(
-                        strings::Substitute("ZSTD_decompressStream output buffer full"));
+                return Status::InvalidArgument("ZSTD_decompressStream output buffer full");
             }
         }
 
@@ -544,7 +535,7 @@ Status get_block_compression_codec(segment_v2::CompressionTypePB type,
         ptr = new ZstdBlockCompression();
         break;
     default:
-        return Status::NotFound(strings::Substitute("unknown compression type($0)", type));
+        return Status::NotFound("unknown compression type({})", type);
     }
 
     if (!ptr) return Status::NotFound("Failed to create compression codec");
diff --git a/be/src/util/broker_storage_backend.cpp b/be/src/util/broker_storage_backend.cpp
index fc653bcb2c..0471c5794d 100644
--- a/be/src/util/broker_storage_backend.cpp
+++ b/be/src/util/broker_storage_backend.cpp
@@ -64,7 +64,7 @@ Status BrokerStorageBackend::download(const std::string& remote, const std::stri
     Status ost =
             file_handler.open_with_mode(local, O_CREAT | O_TRUNC | O_WRONLY, S_IRUSR | S_IWUSR);
     if (!ost.ok()) {
-        return Status::InternalError("failed to open file: " + local);
+        return Status::InternalError("failed to open file: {}", local);
     }
 
     // 4. read remote and write to local
@@ -85,7 +85,7 @@ Status BrokerStorageBackend::download(const std::string& remote, const std::stri
         if (read_len > 0) {
             ost = file_handler.pwrite(read_buf, read_len, write_offset);
             if (!ost.ok()) {
-                return Status::InternalError("failed to write file: " + local);
+                return Status::InternalError("failed to write file: {}", local);
             }
 
             write_offset += read_len;
@@ -105,12 +105,12 @@ Status BrokerStorageBackend::upload(const std::string& local, const std::string&
     FileHandler file_handler;
     Status ost = file_handler.open(local, O_RDONLY);
     if (!ost.ok()) {
-        return Status::InternalError("failed to open file: " + local);
+        return Status::InternalError("failed to open file: {}", local);
     }
 
     size_t file_len = file_handler.length();
     if (file_len == -1) {
-        return Status::InternalError("failed to get length of file: " + local);
+        return Status::InternalError("failed to get length of file: {}", local);
     }
 
     // NOTICE: broker writer must be closed before calling rename
@@ -128,7 +128,7 @@ Status BrokerStorageBackend::upload(const std::string& local, const std::string&
         size_t read_len = left_len > buf_sz ? buf_sz : left_len;
         ost = file_handler.pread(read_buf, read_len, read_offset);
         if (!ost.ok()) {
-            return Status::InternalError("failed to read file: " + local);
+            return Status::InternalError("failed to read file: {}", local);
         }
         // write through broker
         size_t write_len = 0;
@@ -376,7 +376,7 @@ Status BrokerStorageBackend::exist(const std::string& path) {
             LOG(WARNING) << ss.str();
             return Status::InternalError(ss.str());
         } else if (!check_rep.isPathExist) {
-            return Status::NotFound(path + " not exists!");
+            return Status::NotFound("{} not exists!", path);
         } else {
             return Status::OK();
         }
diff --git a/be/src/util/cgroup_util.cpp b/be/src/util/cgroup_util.cpp
index 91d5c909e7..ab7e164f0a 100644
--- a/be/src/util/cgroup_util.cpp
+++ b/be/src/util/cgroup_util.cpp
@@ -43,11 +43,9 @@ Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
     string line;
     while (true) {
         if (proc_cgroups.fail()) {
-            return Status::IOError(
-                    strings::Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg()));
+            return Status::IOError("Error reading /proc/self/cgroup: {}", get_str_err_msg());
         } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) {
-            return Status::NotFound(strings::Substitute(
-                    "Could not find subsystem $0 in /proc/self/cgroup", subsystem));
+            return Status::NotFound("Could not find subsystem {} in /proc/self/cgroup", subsystem);
         }
         // The line format looks like this:
         // 4:memory:/user.slice
@@ -61,9 +59,9 @@ Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
         // ":" in the path does not appear to be escaped - bail in the unusual case that
         // we get too many tokens.
         if (fields.size() != 3) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'",
-                    fields.size(), line));
+            return Status::InvalidArgument(
+                    "Could not parse line from /proc/self/cgroup - had {} > 3 tokens: '{}'",
+                    fields.size(), line);
         }
         std::vector<string> subsystems = Split(fields[1], ",");
         auto it = std::find(subsystems.begin(), subsystems.end(), subsystem);
@@ -77,8 +75,7 @@ Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) {
 static Status unescape_path(const string& escaped, string* unescaped) {
     string err;
     if (!CUnescape(escaped, unescaped, &err)) {
-        return Status::InvalidArgument(
-                strings::Substitute("Could not unescape path '$0': $1", escaped, err));
+        return Status::InvalidArgument("Could not unescape path '{}': {}", escaped, err);
     }
     return Status::OK();
 }
@@ -88,16 +85,14 @@ static Status read_cgroup_value(const string& limit_file_path, int64_t* val) {
     string line;
     getline(limit_file, line);
     if (limit_file.fail() || limit_file.bad()) {
-        return Status::IOError(
-                strings::Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg()));
+        return Status::IOError("Error reading {}: {}", limit_file_path, get_str_err_msg());
     }
     StringParser::ParseResult pr;
     // Parse into an int64_t If it overflows, returning the max value of int64_t is ok because that
     // is effectively unlimited.
     *val = StringParser::string_to_int<int64_t>(line.c_str(), line.size(), &pr);
     if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) {
-        return Status::InvalidArgument(
-                strings::Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line));
+        return Status::InvalidArgument("Failed to parse {} as int64: '{}'", limit_file_path, line);
     }
     return Status::OK();
 }
@@ -107,11 +102,10 @@ Status CGroupUtil::find_cgroup_mounts(const string& subsystem, pair<string, stri
     string line;
     while (true) {
         if (mountinfo.fail() || mountinfo.bad()) {
-            return Status::IOError(strings::Substitute("Error reading /proc/self/mountinfo: $0",
-                                                       get_str_err_msg()));
+            return Status::IOError("Error reading /proc/self/mountinfo: {}", get_str_err_msg());
         } else if (mountinfo.eof()) {
-            return Status::NotFound(strings::Substitute(
-                    "Could not find subsystem $0 in /proc/self/mountinfo", subsystem));
+            return Status::NotFound("Could not find subsystem {} in /proc/self/mountinfo",
+                                    subsystem);
         }
         // The relevant lines look like below (see proc manpage for full documentation). The
         // first example is running outside of a container, the second example is running
@@ -125,9 +119,9 @@ Status CGroupUtil::find_cgroup_mounts(const string& subsystem, pair<string, stri
         if (!mountinfo.good()) continue;
         std::vector<string> fields = Split(line, " ", SkipWhitespace());
         if (fields.size() < 7) {
-            return Status::InvalidArgument(strings::Substitute(
-                    "Could not parse line from /proc/self/mountinfo - had $0 > 7 tokens: '$1'",
-                    fields.size(), line));
+            return Status::InvalidArgument(
+                    "Could not parse line from /proc/self/mountinfo - had {} > 7 tokens: '{}'",
+                    fields.size(), line);
         }
         if (fields[fields.size() - 3] != "cgroup") continue;
         // This is a cgroup mount. Check if it's the mount we're looking for.
@@ -155,8 +149,8 @@ Status CGroupUtil::find_abs_cgroup_path(const string& subsystem, string* path) {
     const string& mount_path = paths.first;
     const string& system_path = paths.second;
     if (path->compare(0, system_path.size(), system_path) != 0) {
-        return Status::InvalidArgument(strings::Substitute(
-                "Expected CGroup path '$0' to start with '$1'", *path, system_path));
+        return Status::InvalidArgument("Expected CGroup path '{}' to start with '{}'", *path,
+                                       system_path);
     }
     path->replace(0, system_path.size(), mount_path);
     return Status::OK();
diff --git a/be/src/util/dynamic_util.cpp b/be/src/util/dynamic_util.cpp
index 1cf65b6b47..d6ff812493 100644
--- a/be/src/util/dynamic_util.cpp
+++ b/be/src/util/dynamic_util.cpp
@@ -31,9 +31,7 @@ Status dynamic_lookup(void* handle, const char* symbol, void** fn_ptr) {
     char* error = dlerror();
 
     if (error != nullptr) {
-        std::stringstream ss;
-        ss << "Unable to find " << symbol << "\ndlerror: " << error;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unable to find {}\ndlerror: {}", symbol, error);
     }
 
     return Status::OK();
@@ -45,9 +43,7 @@ Status dynamic_open(const char* library, void** handle) {
     *handle = dlopen(library, flags);
 
     if (*handle == nullptr) {
-        std::stringstream ss;
-        ss << "Unable to load " << library << "\ndlerror: " << dlerror();
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Unable to load {}\ndlerror: {}", library, dlerror());
     }
 
     return Status::OK();
diff --git a/be/src/util/file_utils.cpp b/be/src/util/file_utils.cpp
index a971acce86..dc6a6195b0 100644
--- a/be/src/util/file_utils.cpp
+++ b/be/src/util/file_utils.cpp
@@ -167,15 +167,11 @@ Status FileUtils::split_paths(const char* path, std::vector<std::string>* path_v
     // Check if
     std::sort(path_vec->begin(), path_vec->end());
     if (std::unique(path_vec->begin(), path_vec->end()) != path_vec->end()) {
-        std::stringstream ss;
-        ss << "Same path in path.[path=" << path << "]";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Same path in path.[path={}]", path);
     }
 
     if (path_vec->size() == 0) {
-        std::stringstream ss;
-        ss << "Size of vector after split is zero.[path=" << path << "]";
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Size of vector after split is zero.[path={}]", path);
     }
 
     return Status::OK();
diff --git a/be/src/util/filesystem_util.cc b/be/src/util/filesystem_util.cc
index 3a12db628a..22c7df4e97 100644
--- a/be/src/util/filesystem_util.cc
+++ b/be/src/util/filesystem_util.cc
@@ -43,26 +43,22 @@ Status FileSystemUtil::create_directory(const string& directory) {
     // Need to check for no_such_file_or_directory error case - Boost's exists() sometimes
     // returns an error when it should simply return false.
     if (errcode && errcode != std::errc::no_such_file_or_directory) {
-        std::stringstream error_msg;
-        error_msg << "Encountered error checking existence of directory: " << directory << ": "
-                  << errcode.message();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Encountered error checking existence of directory {}:{}",
+                                     directory, errcode.message());
     }
     if (exists) {
         // Attempt to remove the directory and its contents so that we can create a fresh
         // empty directory that we will have permissions for.
         std::filesystem::remove_all(directory, errcode);
         if (errcode) {
-            std::stringstream error_msg;
-            error_msg << "Encountered error removing directory " << directory << errcode.message();
-            return Status::InternalError(error_msg.str());
+            return Status::InternalError("Encountered error removing directory {}:{}", directory,
+                                         errcode.message());
         }
     }
     std::filesystem::create_directories(directory, errcode);
     if (errcode) {
-        std::stringstream error_msg;
-        error_msg << "Encountered error creating directory " << directory << errcode.message();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Encountered error creating directory {}:{}", directory,
+                                     errcode.message());
     }
     return Status::OK();
 }
@@ -72,10 +68,8 @@ Status FileSystemUtil::remove_paths(const vector<string>& directories) {
         error_code errcode;
         std::filesystem::remove_all(directories[i], errcode);
         if (errcode) {
-            std::stringstream error_msg;
-            error_msg << "Encountered error removing directory " << directories[i] << ": "
-                      << errcode.message();
-            return Status::InternalError(error_msg.str());
+            return Status::InternalError("Encountered error removing directory {}:{}",
+                                         directories[i], errcode.message());
         }
     }
 
@@ -119,27 +113,21 @@ Status FileSystemUtil::verify_is_directory(const string& directory_path) {
     error_code errcode;
     bool exists = std::filesystem::exists(directory_path, errcode);
     if (errcode) {
-        std::stringstream error_msg;
-        error_msg << "Encountered exception while verifying existence of directory path "
-                  << directory_path << ": " << errcode.message();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError(
+                "Encountered exception while verifying existence of directory path {}: {}",
+                directory_path, errcode.message());
     }
     if (!exists) {
-        std::stringstream error_msg;
-        error_msg << "Directory path " << directory_path << " does not exist ";
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Directory path {} does not exist.", directory_path);
     }
     bool is_dir = std::filesystem::is_directory(directory_path, errcode);
     if (errcode) {
-        std::stringstream error_msg;
-        error_msg << "Encountered exception while verifying existence of directory path "
-                  << directory_path << ": " << errcode.message();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError(
+                "Encountered exception while verifying existence of directory path {}: {}",
+                directory_path, errcode.message());
     }
     if (!is_dir) {
-        std::stringstream error_msg;
-        error_msg << "Path " << directory_path << " is not a directory";
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError("Path {} is not a directory", directory_path);
     }
     return Status::OK();
 }
@@ -149,10 +137,9 @@ Status FileSystemUtil::get_space_available(const string& directory_path,
     error_code errcode;
     std::filesystem::space_info info = std::filesystem::space(directory_path, errcode);
     if (errcode) {
-        std::stringstream error_msg;
-        error_msg << "Encountered exception while checking available space for path "
-                  << directory_path << ": " << errcode.message();
-        return Status::InternalError(error_msg.str());
+        return Status::InternalError(
+                "Encountered exception while checking available space for path {}: {}",
+                directory_path, errcode.message());
     }
     *available_bytes = info.available;
     return Status::OK();
diff --git a/be/src/util/jni-util.cpp b/be/src/util/jni-util.cpp
index 237139ce80..384af3e414 100644
--- a/be/src/util/jni-util.cpp
+++ b/be/src/util/jni-util.cpp
@@ -149,7 +149,7 @@ Status JniUtil::GetJniExceptionMsg(JNIEnv* env, bool log_stack, const string& pr
     }
 
     env->DeleteLocalRef(exc);
-    return Status::InternalError(strings::Substitute("$0$1", prefix, msg_str_guard.get()));
+    return Status::InternalError("{}{}", prefix, msg_str_guard.get());
 }
 
 Status JniUtil::GetGlobalClassRef(JNIEnv* env, const char* class_str, jclass* class_ref) {
diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h
index ccbd26871b..58610d468c 100644
--- a/be/src/util/jni-util.h
+++ b/be/src/util/jni-util.h
@@ -142,9 +142,9 @@ Status SerializeThriftMsg(JNIEnv* env, T* msg, jbyteArray* serialized_msg) {
     // Make sure that 'size' is within the limit of INT_MAX as the use of
     // 'size' below takes int.
     if (size > INT_MAX) {
-        return Status::InternalError(strings::Substitute(
-                "The length of the serialization buffer ($0 bytes) exceeds the limit of $1 bytes",
-                size, INT_MAX));
+        return Status::InternalError(
+                "The length of the serialization buffer ({} bytes) exceeds the limit of {} bytes",
+                size, INT_MAX);
     }
 
     /// create jbyteArray given buffer
diff --git a/be/src/util/load_error_hub.cpp b/be/src/util/load_error_hub.cpp
index 1a0745038c..1149656d8a 100644
--- a/be/src/util/load_error_hub.cpp
+++ b/be/src/util/load_error_hub.cpp
@@ -67,9 +67,7 @@ Status LoadErrorHub::create_hub(ExecEnv* env, const TLoadErrorHubInfo* t_hub_inf
         hub->reset(tmp_hub);
         break;
     default:
-        std::stringstream err;
-        err << "Unknown hub type." << t_hub_info->type;
-        return Status::InternalError(err.str());
+        return Status::InternalError("Unknown hub type {}", t_hub_info->type);
     }
 
     return Status::OK();
diff --git a/be/src/util/network_util.cpp b/be/src/util/network_util.cpp
index 244bed2304..539baa7db3 100644
--- a/be/src/util/network_util.cpp
+++ b/be/src/util/network_util.cpp
@@ -58,9 +58,7 @@ Status get_hostname(std::string* hostname) {
     int ret = gethostname(name, HOST_NAME_MAX);
 
     if (ret != 0) {
-        std::stringstream ss;
-        ss << "Could not get hostname: errno: " << errno;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Could not get hostname: errno: {}", errno);
     }
 
     *hostname = std::string(name);
@@ -76,9 +74,7 @@ Status hostname_to_ip_addrs(const std::string& name, std::vector<std::string>* a
     struct addrinfo* addr_info;
 
     if (getaddrinfo(name.c_str(), nullptr, &hints, &addr_info) != 0) {
-        std::stringstream ss;
-        ss << "Could not find IPv4 address for: " << name;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Could not find IPv4 address for: {}", name);
     }
 
     addrinfo* it = addr_info;
@@ -89,10 +85,8 @@ Status hostname_to_ip_addrs(const std::string& name, std::vector<std::string>* a
                 inet_ntop(AF_INET, &((sockaddr_in*)it->ai_addr)->sin_addr, addr_buf, 64);
 
         if (result == nullptr) {
-            std::stringstream ss;
-            ss << "Could not convert IPv4 address for: " << name;
             freeaddrinfo(addr_info);
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Could not convert IPv4 address for: {}", name);
         }
 
         addresses->push_back(std::string(addr_buf));
diff --git a/be/src/util/pprof_utils.cpp b/be/src/util/pprof_utils.cpp
index 95f84a3e5e..51c0486f92 100644
--- a/be/src/util/pprof_utils.cpp
+++ b/be/src/util/pprof_utils.cpp
@@ -109,7 +109,7 @@ Status PprofUtils::get_readable_profile(const std::string& file_or_content, bool
     FileUtils::remove(file_or_content);
 
     if (!rc) {
-        return Status::InternalError("Failed to execute command: " + cmd_output);
+        return Status::InternalError("Failed to execute command: {}", cmd_output);
     }
 
     (*output) << "Profile(Sample 30 seconds)" << std::endl;
@@ -147,7 +147,7 @@ Status PprofUtils::generate_flamegraph(int32_t sample_seconds,
     bool rc = util.exec_cmd(cmd.str(), &cmd_output);
     if (!rc) {
         FileUtils::remove(tmp_file.str());
-        return Status::InternalError("Failed to execute perf command: " + cmd_output);
+        return Status::InternalError("Failed to execute perf command: {}", cmd_output);
     }
 
     // generate flamegraph
@@ -164,7 +164,7 @@ Status PprofUtils::generate_flamegraph(int32_t sample_seconds,
         if (!rc) {
             FileUtils::remove(tmp_file.str());
             FileUtils::remove(graph_file.str());
-            return Status::InternalError("Failed to execute perf script command: " + res_content);
+            return Status::InternalError("Failed to execute perf script command: {}", res_content);
         }
         *svg_file_or_content = graph_file.str();
     } else {
@@ -174,7 +174,7 @@ Status PprofUtils::generate_flamegraph(int32_t sample_seconds,
         rc = util.exec_cmd(gen_cmd.str(), &res_content, false);
         if (!rc) {
             FileUtils::remove(tmp_file.str());
-            return Status::InternalError("Failed to execute perf script command: " + res_content);
+            return Status::InternalError("Failed to execute perf script command: {}", res_content);
         }
         *svg_file_or_content = res_content;
     }
diff --git a/be/src/util/proto_util.h b/be/src/util/proto_util.h
index 1e645afd9b..e17e6cc806 100644
--- a/be/src/util/proto_util.h
+++ b/be/src/util/proto_util.h
@@ -123,8 +123,8 @@ inline Status request_embed_attachment(Params* brpc_request, const std::string&
         LOG(WARNING) << "Try to alloc " << data_size
                      << " bytes for append data to attachment failed. "
                      << (p ? p.__cxa_exception_type()->name() : "null");
-        return Status::MemoryAllocFailed(
-                fmt::format("request embed attachment failed to memcpy {} bytes", data_size));
+        return Status::MemoryAllocFailed("request embed attachment failed to memcpy {} bytes",
+                                         data_size);
     }
     // step3: attachment add to closure.
     closure->cntl.request_attachment().swap(attachment);
@@ -174,8 +174,8 @@ inline Status attachment_extract_request(const Params* brpc_request, brpc::Contr
         LOG(WARNING) << "Try to alloc " << data_size
                      << " bytes for extract data from attachment failed. "
                      << (p ? p.__cxa_exception_type()->name() : "null");
-        return Status::MemoryAllocFailed(
-                fmt::format("attachment extract request failed to memcpy {} bytes", data_size));
+        return Status::MemoryAllocFailed("attachment extract request failed to memcpy {} bytes",
+                                         data_size);
     }
     return Status::OK();
 }
diff --git a/be/src/util/s3_storage_backend.cpp b/be/src/util/s3_storage_backend.cpp
index ebbaa01cdd..f074e682c1 100644
--- a/be/src/util/s3_storage_backend.cpp
+++ b/be/src/util/s3_storage_backend.cpp
@@ -46,10 +46,10 @@ namespace doris {
 #endif
 
 #ifndef CHECK_S3_PATH
-#define CHECK_S3_PATH(uri, path)                                      \
-    S3URI uri(path);                                                  \
-    if (!uri.parse()) {                                               \
-        return Status::InvalidArgument("s3 uri is invalid: " + path); \
+#define CHECK_S3_PATH(uri, path)                                       \
+    S3URI uri(path);                                                   \
+    if (!uri.parse()) {                                                \
+        return Status::InvalidArgument("s3 uri is invalid: {}", path); \
     }
 #endif
 
@@ -83,10 +83,10 @@ Status S3StorageBackend::download(const std::string& remote, const std::string&
             local_file << response.GetResult().GetBody().rdbuf();
         }
         if (!local_file.good()) {
-            return Status::InternalError("failed to write file: " + local);
+            return Status::InternalError("failed to write file: {}", local);
         }
     } else {
-        return Status::IOError("s3 download error: " + error_msg(response));
+        return Status::IOError("s3 download error: {}", error_msg(response));
     }
     return Status::OK();
 }
@@ -102,7 +102,7 @@ Status S3StorageBackend::direct_download(const std::string& remote, std::string*
         ss << response.GetResult().GetBody().rdbuf();
         *content = ss.str();
     } else {
-        return Status::IOError("s3 direct_download error: " + error_msg(response));
+        return Status::IOError("s3 direct_download error: {}", error_msg(response));
     }
     return Status::OK();
 }
@@ -119,7 +119,7 @@ Status S3StorageBackend::upload(const std::string& local, const std::string& rem
         request.SetBody(input_data);
     }
     if (!input_data->good()) {
-        return Status::InternalError("failed to read file: " + local);
+        return Status::InternalError("failed to read file: {}", local);
     }
     Aws::S3::Model::PutObjectOutcome response = _client->PutObject(request);
 
@@ -166,7 +166,7 @@ Status S3StorageBackend::list(const std::string& remote_path, bool contain_md5,
         }
         return Status::OK();
     } else {
-        return Status::InternalError("list form s3 error: " + error_msg(response));
+        return Status::InternalError("list form s3 error: {}", error_msg(response));
     }
 }
 
@@ -276,7 +276,7 @@ Status S3StorageBackend::exist(const std::string& path) {
     if (response.IsSuccess()) {
         return Status::OK();
     } else if (response.GetError().GetResponseCode() == Aws::Http::HttpResponseCode::NOT_FOUND) {
-        return Status::NotFound(path + " not exists!");
+        return Status::NotFound("{} not exists!", path);
     } else {
         return Status::InternalError(error_msg(response));
     }
@@ -288,7 +288,7 @@ Status S3StorageBackend::exist_dir(const std::string& path) {
     if (files.size() > 0) {
         return Status::OK();
     }
-    return Status::NotFound(path + " not exists!");
+    return Status::NotFound("{} not exists!", path);
 }
 
 Status S3StorageBackend::upload_with_checksum(const std::string& local, const std::string& remote,
diff --git a/be/src/util/storage_backend_mgr.cpp b/be/src/util/storage_backend_mgr.cpp
index ab907e114f..5179a7f31a 100644
--- a/be/src/util/storage_backend_mgr.cpp
+++ b/be/src/util/storage_backend_mgr.cpp
@@ -37,8 +37,8 @@ Status StorageBackendMgr::init(const std::string& storage_param_dir) {
     if (!exist_status.ok() &&
         (!exist_status.is_not_found() || !Env::Default()->create_dirs(storage_param_dir).ok())) {
         RETURN_NOT_OK_STATUS_WITH_WARN(
-                Status::IOError(strings::Substitute(
-                        "failed to create remote storage_param root path $0", storage_param_dir)),
+                Status::IOError("failed to create remote storage_param root path {}",
+                                storage_param_dir),
                 "create_dirs failed");
     }
 
@@ -151,7 +151,7 @@ Status StorageBackendMgr::get_storage_param(const std::string& storage_name,
                                             StorageParamPB* storage_param) {
     std::shared_lock rdlock(_storage_backend_lock);
     if (_storage_backend_map.find(storage_name) == _storage_backend_map.end()) {
-        return Status::InternalError("storage_name not exist: " + storage_name);
+        return Status::InternalError("storage_name not exist: {}", storage_name);
     }
     *storage_param = _storage_param_map[storage_name];
     return Status::OK();
@@ -160,7 +160,7 @@ Status StorageBackendMgr::get_storage_param(const std::string& storage_name,
 Status StorageBackendMgr::get_root_path(const std::string& storage_name, std::string* root_path) {
     std::shared_lock rdlock(_storage_backend_lock);
     if (_storage_backend_map.find(storage_name) == _storage_backend_map.end()) {
-        return Status::InternalError("storage_name not exist: " + storage_name);
+        return Status::InternalError("storage_name not exist: {}", storage_name);
     }
     *root_path = get_root_path_from_param(_storage_param_map[storage_name]);
     return Status::OK();
@@ -197,7 +197,7 @@ Status StorageBackendMgr::_serialize_param(const StorageParamPB& storage_param_p
     bool serialize_success = storage_param_pb.SerializeToString(param_binary);
     if (!serialize_success) {
         LOG(WARNING) << "failed to serialize storage_param " << storage_param_pb.storage_name();
-        return Status::InternalError("failed to serialize storage_param: " +
+        return Status::InternalError("failed to serialize storage_param: {}",
                                      storage_param_pb.storage_name());
     }
     return Status::OK();
diff --git a/be/src/util/thread.cpp b/be/src/util/thread.cpp
index 3be6d8b0e4..39060d645b 100644
--- a/be/src/util/thread.cpp
+++ b/be/src/util/thread.cpp
@@ -361,7 +361,7 @@ Status Thread::start_thread(const std::string& category, const std::string& name
 
     int ret = pthread_create(&t->_thread, nullptr, &Thread::supervise_thread, t.get());
     if (ret) {
-        return Status::RuntimeError("Could not create thread", ret, strerror(ret));
+        return Status::RuntimeError("Could not create thread. (error {}) {}", ret, strerror(ret));
     }
 
     // The thread has been created and is now joinable.
@@ -457,7 +457,8 @@ ThreadJoiner& ThreadJoiner::give_up_after_ms(int ms) {
 
 Status ThreadJoiner::join() {
     if (Thread::current_thread() && Thread::current_thread()->tid() == _thread->tid()) {
-        return Status::InvalidArgument("Can't join on own thread", -1, _thread->_name);
+        return Status::InvalidArgument("Can't join on own thread. (error {}) {}", -1,
+                                       _thread->_name);
     }
 
     // Early exit: double join is a no-op.
@@ -500,8 +501,7 @@ Status ThreadJoiner::join() {
         }
         waited_ms += wait_for;
     }
-    return Status::Aborted(
-            strings::Substitute("Timed out after $0ms joining on $1", waited_ms, _thread->_name));
+    return Status::Aborted("Timed out after {}ms joining on {}", waited_ms, _thread->_name);
 }
 
 void register_thread_display_page(WebPageHandler* web_page_handler) {
diff --git a/be/src/util/threadpool.cpp b/be/src/util/threadpool.cpp
index ef2757f764..e5dce3adb6 100644
--- a/be/src/util/threadpool.cpp
+++ b/be/src/util/threadpool.cpp
@@ -371,10 +371,10 @@ Status ThreadPool::do_submit(std::shared_ptr<Runnable> r, ThreadPoolToken* token
     int64_t capacity_remaining = static_cast<int64_t>(_max_threads) - _active_threads +
                                  static_cast<int64_t>(_max_queue_size) - _total_queued_tasks;
     if (capacity_remaining < 1) {
-        return Status::ServiceUnavailable(strings::Substitute(
-                "Thread pool is at capacity ($0/$1 tasks running, $2/$3 tasks queued)",
+        return Status::ServiceUnavailable(
+                "Thread pool is at capacity ({}/{} tasks running, {}/{} tasks queued)",
                 _num_threads + _num_threads_pending_start, _max_threads, _total_queued_tasks,
-                _max_queue_size));
+                _max_queue_size);
     }
 
     // Should we create another thread?
diff --git a/be/src/util/thrift_util.h b/be/src/util/thrift_util.h
index 6594bfa6c9..3d9a82421d 100644
--- a/be/src/util/thrift_util.h
+++ b/be/src/util/thrift_util.h
@@ -63,9 +63,7 @@ public:
             _mem_buffer->resetBuffer();
             obj->write(_protocol.get());
         } catch (std::exception& e) {
-            std::stringstream msg;
-            msg << "Couldn't serialize thrift object:\n" << e.what();
-            return Status::InternalError(msg.str());
+            return Status::InternalError("Couldn't serialize thrift object:\n{}", e.what());
         }
 
         _mem_buffer->getBuffer(buffer, len);
@@ -78,9 +76,7 @@ public:
             _mem_buffer->resetBuffer();
             obj->write(_protocol.get());
         } catch (apache::thrift::TApplicationException& e) {
-            std::stringstream msg;
-            msg << "Couldn't serialize thrift object:\n" << e.what();
-            return Status::InternalError(msg.str());
+            return Status::InternalError("Couldn't serialize thrift object:\n{}", e.what());
         }
 
         *result = _mem_buffer->getBufferAsString();
@@ -93,9 +89,7 @@ public:
             _mem_buffer->resetBuffer();
             obj->write(_protocol.get());
         } catch (apache::thrift::TApplicationException& e) {
-            std::stringstream msg;
-            msg << "Couldn't serialize thrift object:\n" << e.what();
-            return Status::InternalError(msg.str());
+            return Status::InternalError("Couldn't serialize thrift object:\n{}", e.what());
         }
 
         return Status::OK();
@@ -138,9 +132,7 @@ Status deserialize_thrift_msg(const uint8_t* buf, uint32_t* len, bool compact,
     try {
         deserialized_msg->read(tproto.get());
     } catch (std::exception& e) {
-        std::stringstream msg;
-        msg << "couldn't deserialize thrift msg:\n" << e.what();
-        return Status::InternalError(msg.str());
+        return Status::InternalError("Couldn't deserialize thrift msg:\n{}", e.what());
     } catch (...) {
         // TODO: Find the right exception for 0 bytes
         return Status::InternalError("Unknown exception");
diff --git a/be/src/util/zip_util.cpp b/be/src/util/zip_util.cpp
index 23023c7774..901230e5cb 100644
--- a/be/src/util/zip_util.cpp
+++ b/be/src/util/zip_util.cpp
@@ -51,21 +51,20 @@ Status ZipFile::extract(const std::string& target_path, const std::string& dir_n
     // check zip file
     _zip_file = unzOpen64(_zip_path.c_str());
     if (_zip_file == nullptr) {
-        return Status::InvalidArgument("open zip file: " + _zip_path + " error");
+        return Status::InvalidArgument("open zip file: {} error", _zip_path);
     }
 
     unz_global_info64 global_info;
     int err = unzGetGlobalInfo64(_zip_file, &global_info);
 
     if (err != UNZ_OK) {
-        return Status::IOError(
-                strings::Substitute("read zip file info $0 error, code: $1", _zip_path, err));
+        return Status::IOError("read zip file info {} error, code: {}", _zip_path, err);
     }
 
     // 0.check target path
     std::string target = target_path + "/" + dir_name;
     if (FileUtils::check_exist(target)) {
-        return Status::AlreadyExist("path already exists: " + target);
+        return Status::AlreadyExist("path already exists: {}", target);
     }
 
     // 1.create temp directory
@@ -97,7 +96,7 @@ Status ZipFile::extract_file(const std::string& target_path) {
                                       DEFAULT_FILE_NAME_SIZE, nullptr, 0, nullptr, 0);
 
     if (err != UNZ_OK) {
-        return Status::IOError(strings::Substitute("read zip file info error, code: $0", err));
+        return Status::IOError("read zip file info error, code: {}", err);
     }
 
     // is directory, mkdir
@@ -112,8 +111,7 @@ Status ZipFile::extract_file(const std::string& target_path) {
     _open_current_file = true;
 
     if (UNZ_OK != err) {
-        return Status::IOError(
-                strings::Substitute("read zip file $0 info error, code: $1", file_name, err));
+        return Status::IOError("read zip file {} info error, code: {}", file_name, err);
     }
 
     ZPOS64_T file_size = std::min(file_info_inzip.uncompressed_size, DEFAULT_UNZIP_BUFFER);
@@ -126,7 +124,7 @@ Status ZipFile::extract_file(const std::string& target_path) {
     do {
         size = unzReadCurrentFile(_zip_file, (voidp)file_data.get(), file_size);
         if (size < 0) {
-            return Status::IOError(strings::Substitute("unzip file $0 failed", file_name));
+            return Status::IOError("unzip file {} failed", file_name);
         }
 
         RETURN_IF_ERROR(wfile->append(Slice(file_data.get(), size)));
diff --git a/be/src/util/zlib.cpp b/be/src/util/zlib.cpp
index ea2ca2de37..c23ca009fb 100644
--- a/be/src/util/zlib.cpp
+++ b/be/src/util/zlib.cpp
@@ -58,7 +58,7 @@ Status ZlibResultToStatus(int rc) {
     case Z_VERSION_ERROR:
         return Status::RuntimeError("zlib error: VERSION_ERROR");
     default:
-        return Status::RuntimeError(strings::Substitute("zlib error: unknown error $0", rc));
+        return Status::RuntimeError("zlib error: unknown error {}", rc);
     }
 }
 } // anonymous namespace
diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp
index 3bcefb0cc9..41d4dfaf40 100644
--- a/be/src/vec/core/block.cpp
+++ b/be/src/vec/core/block.cpp
@@ -635,8 +635,8 @@ Status Block::filter_block(Block* block, int filter_column_id, int column_to_kee
         ColumnUInt8* concrete_column = typeid_cast<ColumnUInt8*>(mutable_holder.get());
         if (!concrete_column) {
             return Status::InvalidArgument(
-                    "Illegal type " + filter_column->get_name() +
-                    " of column for filter. Must be UInt8 or Nullable(UInt8).");
+                    "Illegal type {} of column for filter. Must be UInt8 or Nullable(UInt8).",
+                    filter_column->get_name());
         }
         auto* __restrict null_map = nullable_column->get_null_map_data().data();
         IColumn::Filter& filter = concrete_column->get_data();
@@ -732,8 +732,8 @@ Status Block::serialize(PBlock* pblock, size_t* uncompressed_bytes, size_t* comp
                  << ", compressed size: " << compressed_size;
     }
     if (!allow_transfer_large_data && *compressed_bytes >= std::numeric_limits<int32_t>::max()) {
-        return Status::InternalError(fmt::format(
-                "The block is large than 2GB({}), can not send by Protobuf.", *compressed_bytes));
+        return Status::InternalError("The block is large than 2GB({}), can not send by Protobuf.",
+                                     *compressed_bytes);
     }
     return Status::OK();
 }
diff --git a/be/src/vec/data_types/data_type_array.cpp b/be/src/vec/data_types/data_type_array.cpp
index e39f1c569b..40aa07011b 100644
--- a/be/src/vec/data_types/data_type_array.cpp
+++ b/be/src/vec/data_types/data_type_array.cpp
@@ -152,9 +152,9 @@ Status DataTypeArray::from_string(ReadBuffer& rb, IColumn* column) const {
             if (*rb.position() == ',') {
                 ++rb.position();
             } else {
-                return Status::InvalidArgument(fmt::format(
+                return Status::InvalidArgument(
                         "Cannot read array from text, expected comma or end of array, found '{}'",
-                        *rb.position()));
+                        *rb.position());
             }
         }
         first = false;
diff --git a/be/src/vec/data_types/data_type_date.cpp b/be/src/vec/data_types/data_type_date.cpp
index 3e261d0f99..3697a99418 100644
--- a/be/src/vec/data_types/data_type_date.cpp
+++ b/be/src/vec/data_types/data_type_date.cpp
@@ -64,8 +64,8 @@ Status DataTypeDate::from_string(ReadBuffer& rb, IColumn* column) const {
     auto* column_data = static_cast<ColumnInt64*>(column);
     Int64 val = 0;
     if (!read_date_text_impl<Int64>(val, rb)) {
-        return Status::InvalidArgument(fmt::format("parse date fail, string: '{}'",
-                                                   std::string(rb.position(), rb.count()).c_str()));
+        return Status::InvalidArgument("parse date fail, string: '{}'",
+                                       std::string(rb.position(), rb.count()).c_str());
     }
     column_data->insert_value(val);
     return Status::OK();
diff --git a/be/src/vec/data_types/data_type_date_time.cpp b/be/src/vec/data_types/data_type_date_time.cpp
index b758390576..02daeb478c 100644
--- a/be/src/vec/data_types/data_type_date_time.cpp
+++ b/be/src/vec/data_types/data_type_date_time.cpp
@@ -87,8 +87,8 @@ Status DataTypeDateTime::from_string(ReadBuffer& rb, IColumn* column) const {
     auto* column_data = static_cast<ColumnInt64*>(column);
     Int64 val = 0;
     if (!read_datetime_text_impl<Int64>(val, rb)) {
-        return Status::InvalidArgument(fmt::format("parse datetime fail, string: '{}'",
-                                                   std::string(rb.position(), rb.count()).c_str()));
+        return Status::InvalidArgument("parse datetime fail, string: '{}'",
+                                       std::string(rb.position(), rb.count()).c_str());
     }
     column_data->insert_value(val);
     return Status::OK();
diff --git a/be/src/vec/data_types/data_type_decimal.cpp b/be/src/vec/data_types/data_type_decimal.cpp
index 04b8713da9..d0eae375e9 100644
--- a/be/src/vec/data_types/data_type_decimal.cpp
+++ b/be/src/vec/data_types/data_type_decimal.cpp
@@ -67,8 +67,8 @@ Status DataTypeDecimal<T>::from_string(ReadBuffer& rb, IColumn* column) const {
     auto& column_data = static_cast<ColumnType&>(*column).get_data();
     T val = 0;
     if (!read_decimal_text_impl<T>(val, rb)) {
-        return Status::InvalidArgument(fmt::format("parse decimal fail, string: '{}'",
-                                                   std::string(rb.position(), rb.count()).c_str()));
+        return Status::InvalidArgument("parse decimal fail, string: '{}'",
+                                       std::string(rb.position(), rb.count()).c_str());
     }
     column_data.emplace_back(val);
     return Status::OK();
diff --git a/be/src/vec/data_types/data_type_number_base.cpp b/be/src/vec/data_types/data_type_number_base.cpp
index a6111412df..b3052e9043 100644
--- a/be/src/vec/data_types/data_type_number_base.cpp
+++ b/be/src/vec/data_types/data_type_number_base.cpp
@@ -64,17 +64,15 @@ Status DataTypeNumberBase<T>::from_string(ReadBuffer& rb, IColumn* column) const
     } else if constexpr (std::is_same_v<T, float> || std::is_same_v<T, double>) {
         T val = 0;
         if (!read_float_text_fast_impl(val, rb)) {
-            return Status::InvalidArgument(
-                    fmt::format("parse number fail, string: '{}'",
-                                std::string(rb.position(), rb.count()).c_str()));
+            return Status::InvalidArgument("parse number fail, string: '{}'",
+                                           std::string(rb.position(), rb.count()).c_str());
         }
         column_data->insert_value(val);
     } else if constexpr (std::is_integral<T>::value) {
         T val = 0;
         if (!read_int_text_impl(val, rb)) {
-            return Status::InvalidArgument(
-                    fmt::format("parse number fail, string: '{}'",
-                                std::string(rb.position(), rb.count()).c_str()));
+            return Status::InvalidArgument("parse number fail, string: '{}'",
+                                           std::string(rb.position(), rb.count()).c_str());
         }
         column_data->insert_value(val);
     } else {
diff --git a/be/src/vec/exec/vaggregation_node.cpp b/be/src/vec/exec/vaggregation_node.cpp
index bc699b855c..68c48897ec 100644
--- a/be/src/vec/exec/vaggregation_node.cpp
+++ b/be/src/vec/exec/vaggregation_node.cpp
@@ -257,7 +257,7 @@ Status AggregationNode::prepare(RuntimeState* state) {
             size_t alignment_of_next_state =
                     _aggregate_evaluators[i + 1]->function()->align_of_data();
             if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0) {
-                return Status::RuntimeError(fmt::format("Logical error: align_of_data is not 2^N"));
+                return Status::RuntimeError("Logical error: align_of_data is not 2^N");
             }
 
             /// Extend total_size to next alignment requirement
diff --git a/be/src/vec/exec/vanalytic_eval_node.cpp b/be/src/vec/exec/vanalytic_eval_node.cpp
index 49d9b11d51..23e1274dd4 100644
--- a/be/src/vec/exec/vanalytic_eval_node.cpp
+++ b/be/src/vec/exec/vanalytic_eval_node.cpp
@@ -174,7 +174,7 @@ Status VAnalyticEvalNode::prepare(RuntimeState* state) {
         if (i + 1 < _agg_functions_size) {
             size_t alignment_of_next_state = _agg_functions[i + 1]->function()->align_of_data();
             if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0) {
-                return Status::RuntimeError(fmt::format("Logical error: align_of_data is not 2^N"));
+                return Status::RuntimeError("Logical error: align_of_data is not 2^N");
             }
             /// Extend total_size to next alignment requirement
             /// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
diff --git a/be/src/vec/exec/varrow_scanner.cpp b/be/src/vec/exec/varrow_scanner.cpp
index 237c65fb62..05329f1d04 100644
--- a/be/src/vec/exec/varrow_scanner.cpp
+++ b/be/src/vec/exec/varrow_scanner.cpp
@@ -79,9 +79,8 @@ Status VArrowScanner::_open_next_reader() {
             continue;
         } else {
             if (!status.ok()) {
-                std::stringstream ss;
-                ss << " file: " << range.path << " error:" << status.get_error_msg();
-                return Status::InternalError(ss.str());
+                return Status::InternalError(" file: {} error:{}", range.path,
+                                             status.get_error_msg());
             } else {
                 return status;
             }
diff --git a/be/src/vec/exec/vassert_num_rows_node.cpp b/be/src/vec/exec/vassert_num_rows_node.cpp
index 9bc7db4077..7a246df17e 100644
--- a/be/src/vec/exec/vassert_num_rows_node.cpp
+++ b/be/src/vec/exec/vassert_num_rows_node.cpp
@@ -87,9 +87,8 @@ Status VAssertNumRowsNode::get_next(RuntimeState* state, Block* block, bool* eos
         };
         LOG(INFO) << "Expected " << to_string_lambda(_assertion) << " " << _desired_num_rows
                   << " to be returned by expression " << _subquery_string;
-        return Status::Cancelled(strings::Substitute(
-                "Expected $0 $1 to be returned by expression $2", to_string_lambda(_assertion),
-                _desired_num_rows, _subquery_string));
+        return Status::Cancelled("Expected {} {} to be returned by expression {}",
+                                 to_string_lambda(_assertion), _desired_num_rows, _subquery_string);
     }
     COUNTER_SET(_rows_returned_counter, _num_rows_returned);
     return Status::OK();
diff --git a/be/src/vec/exec/vbroker_scan_node.cpp b/be/src/vec/exec/vbroker_scan_node.cpp
index 0d639b7077..212c82d157 100644
--- a/be/src/vec/exec/vbroker_scan_node.cpp
+++ b/be/src/vec/exec/vbroker_scan_node.cpp
@@ -64,18 +64,14 @@ Status VBrokerScanNode::prepare(RuntimeState* state) {
     _runtime_state = state;
     _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
     if (_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Failed to get tuple descriptor, _tuple_id=" << _tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to get tuple descriptor, _tuple_id={}", _tuple_id);
     }
 
     // Initialize slots map
     for (auto slot : _tuple_desc->slots()) {
         auto pair = _slots_map.emplace(slot->col_name(), slot);
         if (!pair.second) {
-            std::stringstream ss;
-            ss << "Failed to insert slot, col_name=" << slot->col_name();
-            return Status::InternalError(ss.str());
+            return Status::InternalError("Failed to insert slot, col_name={}", slot->col_name());
         }
     }
 
diff --git a/be/src/vec/exec/ves_http_scan_node.cpp b/be/src/vec/exec/ves_http_scan_node.cpp
index b5dcf02840..4b9d0cd250 100644
--- a/be/src/vec/exec/ves_http_scan_node.cpp
+++ b/be/src/vec/exec/ves_http_scan_node.cpp
@@ -72,9 +72,7 @@ Status VEsHttpScanNode::prepare(RuntimeState* state) {
     _runtime_state = state;
     _tuple_desc = state->desc_tbl().get_tuple_descriptor(_tuple_id);
     if (_tuple_desc == nullptr) {
-        std::stringstream ss;
-        ss << "Failed to get tuple descriptor, _tuple_id=" << _tuple_id;
-        return Status::InternalError(ss.str());
+        return Status::InternalError("Failed to get tuple descriptor, _tuple_id=i{}", _tuple_id);
     }
 
     // set up column name vector for ESScrollQueryBuilder
diff --git a/be/src/vec/exec/vmysql_scan_node.cpp b/be/src/vec/exec/vmysql_scan_node.cpp
index 3ba8a4e3b4..954a28bd72 100644
--- a/be/src/vec/exec/vmysql_scan_node.cpp
+++ b/be/src/vec/exec/vmysql_scan_node.cpp
@@ -192,10 +192,9 @@ Status VMysqlScanNode::get_next(RuntimeState* state, vectorized::Block* block, b
                                 reinterpret_cast<vectorized::ColumnNullable*>(columns[i].get());
                         nullable_column->insert_data(nullptr, 0);
                     } else {
-                        std::stringstream ss;
-                        ss << "nonnull column contains NULL. table=" << _table_name
-                           << ", column=" << slot_desc->col_name();
-                        return Status::InternalError(ss.str());
+                        return Status::InternalError(
+                                "nonnull column contains NULL. table={}, column={}", _table_name,
+                                slot_desc->col_name());
                     }
                 } else {
                     RETURN_IF_ERROR(
diff --git a/be/src/vec/exec/vrepeat_node.cpp b/be/src/vec/exec/vrepeat_node.cpp
index 636ad5dd43..1595aef71c 100644
--- a/be/src/vec/exec/vrepeat_node.cpp
+++ b/be/src/vec/exec/vrepeat_node.cpp
@@ -57,7 +57,10 @@ Status VRepeatNode::prepare(RuntimeState* state) {
        << " is not equal to the sum of child_slots_size " << _child_slots.size()
        << ",virtual_slots_size " << _virtual_tuple_desc->slots().size();
     if (_output_slots.size() != (_child_slots.size() + _virtual_tuple_desc->slots().size())) {
-        return Status::InternalError(ss.str());
+        return Status::InternalError(
+                "The output slots size {} is not equal to the sum of child_slots_size {}"
+                ",virtual_slots_size {}",
+                _output_slots.size(), _child_slots.size(), _virtual_tuple_desc->slots().size());
     }
 
     _child_block.reset(new Block());
diff --git a/be/src/vec/exec/vschema_scan_node.cpp b/be/src/vec/exec/vschema_scan_node.cpp
index 4df6bc4151..f2e29eb7e9 100644
--- a/be/src/vec/exec/vschema_scan_node.cpp
+++ b/be/src/vec/exec/vschema_scan_node.cpp
@@ -278,10 +278,9 @@ Status VSchemaScanNode::get_next(RuntimeState* state, vectorized::Block* block,
                                 reinterpret_cast<vectorized::ColumnNullable*>(columns[i].get());
                         nullable_column->insert_data(nullptr, 0);
                     } else {
-                        std::stringstream ss;
-                        ss << "nonnull column contains NULL. table=" << _table_name
-                           << ", column=" << slot_desc->col_name();
-                        return Status::InternalError(ss.str());
+                        return Status::InternalError(
+                                "nonnull column contains NULL. table={}, column={}", _table_name,
+                                slot_desc->col_name());
                     }
                 } else {
                     RETURN_IF_ERROR(write_slot_to_vectorized_column(
diff --git a/be/src/vec/exprs/table_function/vexplode.cpp b/be/src/vec/exprs/table_function/vexplode.cpp
index adb0db076d..2669213119 100644
--- a/be/src/vec/exprs/table_function/vexplode.cpp
+++ b/be/src/vec/exprs/table_function/vexplode.cpp
@@ -45,9 +45,8 @@ Status VExplodeTableFunction::process_init(vectorized::Block* block) {
                 check_and_get_column<ColumnArray>(*block->get_by_position(value_column_idx).column);
     }
     if (!_array_column) {
-        return Status::NotSupported("column type " +
-                                    block->get_by_position(value_column_idx).column->get_name() +
-                                    " not supported now");
+        return Status::NotSupported("column type {} not supported now",
+                                    block->get_by_position(value_column_idx).column->get_name());
     }
 
     return Status::OK();
diff --git a/be/src/vec/exprs/vcase_expr.cpp b/be/src/vec/exprs/vcase_expr.cpp
index ba45fafa64..f11c9ef1e5 100644
--- a/be/src/vec/exprs/vcase_expr.cpp
+++ b/be/src/vec/exprs/vcase_expr.cpp
@@ -56,8 +56,8 @@ Status VCaseExpr::prepare(doris::RuntimeState* state, const doris::RowDescriptor
     _function = SimpleFunctionFactory::instance().get_function(_function_name, argument_template,
                                                                _data_type);
     if (_function == nullptr) {
-        return Status::NotSupported(
-                fmt::format("vcase_expr Function {} is not implemented", _fn.name.function_name));
+        return Status::NotSupported("vcase_expr Function {} is not implemented",
+                                    _fn.name.function_name);
     }
 
     VExpr::register_function_context(state, context);
diff --git a/be/src/vec/exprs/vcast_expr.cpp b/be/src/vec/exprs/vcast_expr.cpp
index f268b12ec7..b30cd14c40 100644
--- a/be/src/vec/exprs/vcast_expr.cpp
+++ b/be/src/vec/exprs/vcast_expr.cpp
@@ -51,8 +51,7 @@ doris::Status VCastExpr::prepare(doris::RuntimeState* state, const doris::RowDes
                                                                _data_type);
 
     if (_function == nullptr) {
-        return Status::NotSupported(
-                fmt::format("Function {} is not implemented", _fn.name.function_name));
+        return Status::NotSupported("Function {} is not implemented", _fn.name.function_name);
     }
     VExpr::register_function_context(state, context);
     _expr_name = fmt::format("(CAST {}, TO {})", child_name, _target_data_type_name);
diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp b/be/src/vec/exprs/vectorized_agg_fn.cpp
index bf6deae4ad..ad7066a9a4 100644
--- a/be/src/vec/exprs/vectorized_agg_fn.cpp
+++ b/be/src/vec/exprs/vectorized_agg_fn.cpp
@@ -98,8 +98,7 @@ Status AggFnEvaluator::prepare(RuntimeState* state, const RowDescriptor& desc, M
                 _fn.name.function_name, argument_types, params, _data_type->is_nullable());
     }
     if (_function == nullptr) {
-        return Status::InternalError(
-                fmt::format("Agg Function {} is not implemented", _fn.name.function_name));
+        return Status::InternalError("Agg Function {} is not implemented", _fn.name.function_name);
     }
 
     _expr_name = fmt::format("{}({})", _fn.name.function_name, child_expr_name);
diff --git a/be/src/vec/exprs/vectorized_fn_call.cpp b/be/src/vec/exprs/vectorized_fn_call.cpp
index 9afacc748b..99641090a8 100644
--- a/be/src/vec/exprs/vectorized_fn_call.cpp
+++ b/be/src/vec/exprs/vectorized_fn_call.cpp
@@ -58,8 +58,7 @@ doris::Status VectorizedFnCall::prepare(doris::RuntimeState* state,
                                                                    argument_template, _data_type);
     }
     if (_function == nullptr) {
-        return Status::InternalError(
-                fmt::format("Function {} is not implemented", _fn.name.function_name));
+        return Status::InternalError("Function {} is not implemented", _fn.name.function_name);
     }
     VExpr::register_function_context(state, context);
     _expr_name = fmt::format("{}({})", _fn.name.function_name, child_expr_name);
diff --git a/be/src/vec/exprs/vexpr.cpp b/be/src/vec/exprs/vexpr.cpp
index 058895a6fb..c2da03f0a0 100644
--- a/be/src/vec/exprs/vexpr.cpp
+++ b/be/src/vec/exprs/vexpr.cpp
@@ -155,8 +155,7 @@ Status VExpr::create_expr(doris::ObjectPool* pool, const doris::TExprNode& texpr
         break;
     }
     default:
-        return Status::InternalError(
-                fmt::format("Unknown expr node type: {}", texpr_node.node_type));
+        return Status::InternalError("Unknown expr node type: {}", texpr_node.node_type);
     }
     return Status::OK();
 }
diff --git a/be/src/vec/exprs/vin_predicate.cpp b/be/src/vec/exprs/vin_predicate.cpp
index b2fa7ed053..0d6cceac4f 100644
--- a/be/src/vec/exprs/vin_predicate.cpp
+++ b/be/src/vec/exprs/vin_predicate.cpp
@@ -60,8 +60,7 @@ Status VInPredicate::prepare(RuntimeState* state, const RowDescriptor& desc,
     _function = SimpleFunctionFactory::instance().get_function(real_function_name,
                                                                argument_template, _data_type);
     if (_function == nullptr) {
-        return Status::NotSupported(
-                fmt::format("Function {} is not implemented", real_function_name));
+        return Status::NotSupported("Function {} is not implemented", real_function_name);
     }
 
     VExpr::register_function_context(state, context);
diff --git a/be/src/vec/exprs/vslot_ref.cpp b/be/src/vec/exprs/vslot_ref.cpp
index aa728b9c39..2732a341e7 100644
--- a/be/src/vec/exprs/vslot_ref.cpp
+++ b/be/src/vec/exprs/vslot_ref.cpp
@@ -48,7 +48,7 @@ Status VSlotRef::prepare(doris::RuntimeState* state, const doris::RowDescriptor&
     }
     const SlotDescriptor* slot_desc = state->desc_tbl().get_slot_descriptor(_slot_id);
     if (slot_desc == NULL) {
-        return Status::InternalError(fmt::format("couldn't resolve slot descriptor {}", _slot_id));
+        return Status::InternalError("couldn't resolve slot descriptor {}", _slot_id);
     }
     _column_id = desc.get_column_id(_slot_id);
     _column_name = &slot_desc->col_name();
diff --git a/be/src/vec/functions/array/function_array_aggregation.cpp b/be/src/vec/functions/array/function_array_aggregation.cpp
index 8095ff75ad..422cc0e3c1 100644
--- a/be/src/vec/functions/array/function_array_aggregation.cpp
+++ b/be/src/vec/functions/array/function_array_aggregation.cpp
@@ -163,7 +163,7 @@ struct ArrayAggregateImpl {
             block.replace_by_position(result, std::move(res));
             return Status::OK();
         } else {
-            return Status::RuntimeError("Unexpected column for aggregation: " + data->get_name());
+            return Status::RuntimeError("Unexpected column for aggregation: {}", data->get_name());
         }
     }
 
diff --git a/be/src/vec/functions/array/function_array_element.h b/be/src/vec/functions/array/function_array_element.h
index 4f6eb12a07..579d45f53f 100644
--- a/be/src/vec/functions/array/function_array_element.h
+++ b/be/src/vec/functions/array/function_array_element.h
@@ -71,10 +71,9 @@ public:
 
         auto res_column = _execute_non_nullable(args, input_rows_count, src_null_map, dst_null_map);
         if (!res_column) {
-            return Status::RuntimeError(
-                    fmt::format("unsupported types for function {}({}, {})", get_name(),
-                                block.get_by_position(arguments[0]).type->get_name(),
-                                block.get_by_position(arguments[1]).type->get_name()));
+            return Status::RuntimeError("unsupported types for function {}({}, {})", get_name(),
+                                        block.get_by_position(arguments[0]).type->get_name(),
+                                        block.get_by_position(arguments[1]).type->get_name());
         }
         block.replace_by_position(
                 result, ColumnNullable::create(std::move(res_column), std::move(dst_null_column)));
diff --git a/be/src/vec/functions/array/function_array_index.h b/be/src/vec/functions/array/function_array_index.h
index 241bbda6b2..34eb56d416 100644
--- a/be/src/vec/functions/array/function_array_index.h
+++ b/be/src/vec/functions/array/function_array_index.h
@@ -267,10 +267,10 @@ private:
             block.replace_by_position(result, std::move(return_column));
             return Status::OK();
         }
-        return Status::RuntimeError(
-                fmt::format("execute failed or unsupported types for function {}({}, {})",
-                            get_name(), block.get_by_position(arguments[0]).type->get_name(),
-                            block.get_by_position(arguments[1]).type->get_name()));
+        return Status::RuntimeError("execute failed or unsupported types for function {}({}, {})",
+                                    get_name(),
+                                    block.get_by_position(arguments[0]).type->get_name(),
+                                    block.get_by_position(arguments[1]).type->get_name());
     }
 };
 
diff --git a/be/src/vec/functions/array/function_array_size.h b/be/src/vec/functions/array/function_array_size.h
index 1988c5b66a..1e375814b7 100644
--- a/be/src/vec/functions/array/function_array_size.h
+++ b/be/src/vec/functions/array/function_array_size.h
@@ -50,9 +50,8 @@ public:
                 block.get_by_position(arguments[0]).column->convert_to_full_column_if_const();
         const auto array_column = check_and_get_column<ColumnArray>(*left_column);
         if (!array_column) {
-            return Status::RuntimeError(
-                    fmt::format("unsupported types for function {}({})", get_name(),
-                                block.get_by_position(arguments[0]).type->get_name()));
+            return Status::RuntimeError("unsupported types for function {}({})", get_name(),
+                                        block.get_by_position(arguments[0]).type->get_name());
         }
         const auto& offsets = array_column->get_offsets();
 
diff --git a/be/src/vec/functions/date_time_transforms.h b/be/src/vec/functions/date_time_transforms.h
index 5942d72de1..1e8d7962bb 100644
--- a/be/src/vec/functions/date_time_transforms.h
+++ b/be/src/vec/functions/date_time_transforms.h
@@ -358,9 +358,9 @@ struct DateTimeTransformImpl {
             block.replace_by_position(
                     result, ColumnNullable::create(std::move(col_to), std::move(null_map)));
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of first argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), Transform::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        Transform::name);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function.cpp b/be/src/vec/functions/function.cpp
index bda271be0d..cb53206e37 100644
--- a/be/src/vec/functions/function.cpp
+++ b/be/src/vec/functions/function.cpp
@@ -135,8 +135,8 @@ Status PreparedFunctionImpl::default_implementation_for_constant_arguments(
     for (auto arg_num : arguments_to_remain_constants) {
         if (arg_num < args.size() &&
             !is_column_const(*block.get_by_position(args[arg_num]).column)) {
-            return Status::RuntimeError(fmt::format(
-                    "Argument at index {} for function {}  must be constant", arg_num, get_name()));
+            return Status::RuntimeError("Argument at index {} for function {}  must be constant",
+                                        arg_num, get_name());
         }
     }
 
@@ -169,9 +169,9 @@ Status PreparedFunctionImpl::default_implementation_for_constant_arguments(
       */
     if (!have_converted_columns) {
         return Status::RuntimeError(
-                fmt::format("Number of arguments for function {} doesn't match: the function "
-                            "requires more arguments",
-                            get_name()));
+                "Number of arguments for function {} doesn't match: the function "
+                "requires more arguments",
+                get_name());
     }
 
     temporary_block.insert(block.get_by_position(result));
diff --git a/be/src/vec/functions/function_always_not_nullable.h b/be/src/vec/functions/function_always_not_nullable.h
index 76b2b2a600..1c476c56c0 100644
--- a/be/src/vec/functions/function_always_not_nullable.h
+++ b/be/src/vec/functions/function_always_not_nullable.h
@@ -69,9 +69,9 @@ public:
             block.replace_by_position(result, std::move(column_result));
             return Status::OK();
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), get_name()));
+            return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        get_name());
         }
 
         block.replace_by_position(result, std::move(column_result));
diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h
index b3bb43a803..3091ab2dde 100644
--- a/be/src/vec/functions/function_binary_arithmetic.h
+++ b/be/src/vec/functions/function_binary_arithmetic.h
@@ -842,8 +842,8 @@ public:
                     return false;
                 });
         if (!valid) {
-            return Status::RuntimeError(
-                    fmt::format("{}'s arguments do not match the expected data types", get_name()));
+            return Status::RuntimeError("{}'s arguments do not match the expected data types",
+                                        get_name());
         }
 
         return Status::OK();
diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h
index 47e33f58ff..ae394fcb10 100644
--- a/be/src/vec/functions/function_case.h
+++ b/be/src/vec/functions/function_case.h
@@ -321,8 +321,7 @@ public:
                                                   input_rows_count);
         TYPE_TO_COLUMN_TYPE(DISPATCH)
 #undef DISPATCH
-        return Status::NotSupported(
-                fmt::format("argument_type {} not supported", data_type->get_name()));
+        return Status::NotSupported("argument_type {} not supported", data_type->get_name());
     }
 
     Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h
index 490d414418..1531b77a24 100644
--- a/be/src/vec/functions/function_cast.h
+++ b/be/src/vec/functions/function_cast.h
@@ -86,9 +86,8 @@ struct ConvertImpl {
             if constexpr (!(IsDataTypeDecimalOrNumber<FromDataType> || IsTimeType<FromDataType> ||
                             IsDateV2Type<FromDataType>) ||
                           !IsDataTypeDecimalOrNumber<ToDataType>)
-                return Status::RuntimeError(
-                        fmt::format("Illegal column {} of first argument of function {}",
-                                    named_from.column->get_name(), Name::name));
+                return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                            named_from.column->get_name(), Name::name);
         }
 
         if (const ColVecFrom* col_from =
@@ -168,9 +167,8 @@ struct ConvertImpl {
 
             block.replace_by_position(result, std::move(col_to));
         } else {
-            return Status::RuntimeError(
-                    fmt::format("Illegal column {} of first argument of function {}",
-                                named_from.column->get_name(), Name::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        named_from.column->get_name(), Name::name);
         }
         return Status::OK();
     }
@@ -241,9 +239,8 @@ struct ConvertImplToTimeType {
             block.get_by_position(result).column =
                     ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
         } else {
-            return Status::RuntimeError(
-                    fmt::format("Illegal column {} of first argument of function {}",
-                                named_from.column->get_name(), Name::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        named_from.column->get_name(), Name::name);
         }
 
         return Status::OK();
@@ -297,9 +294,9 @@ struct ConvertImplGenericFromString {
             }
             block.replace_by_position(result, std::move(col_to));
         } else {
-            return Status::RuntimeError(fmt::format(
+            return Status::RuntimeError(
                     "Illegal column {} of first argument of conversion function from string",
-                    col_from.get_name()));
+                    col_from.get_name());
         }
         return Status::OK();
     }
@@ -599,8 +596,7 @@ private:
     Status executeInternal(Block& block, const ColumnNumbers& arguments, size_t result,
                            size_t input_rows_count) {
         if (!arguments.size()) {
-            return Status::RuntimeError(
-                    fmt::format("Function {} expects at least 1 arguments", get_name()));
+            return Status::RuntimeError("Function {} expects at least 1 arguments", get_name());
         }
 
         const IDataType* from_type = block.get_by_position(arguments[0]).type.get();
@@ -618,8 +614,8 @@ private:
                 // now, cast to decimal do not execute the code
                 if constexpr (IsDataTypeDecimal<RightDataType>) {
                     if (arguments.size() != 2) {
-                        ret_status = Status::RuntimeError(fmt::format(
-                                "Function {} expects 2 arguments for Decimal.", get_name()));
+                        ret_status = Status::RuntimeError(
+                                "Function {} expects 2 arguments for Decimal.", get_name());
                         return true;
                     }
 
@@ -636,9 +632,9 @@ private:
 
             bool done = call_on_index_and_data_type<ToDataType>(from_type->get_type_id(), call);
             if (!done) {
-                ret_status = Status::RuntimeError(fmt::format(
+                ret_status = Status::RuntimeError(
                         "Illegal type {} of argument of function {}",
-                        block.get_by_position(arguments[0]).type->get_name(), get_name()));
+                        block.get_by_position(arguments[0]).type->get_name(), get_name());
             }
             return ret_status;
         }
@@ -802,9 +798,8 @@ struct ConvertThroughParsing {
         const ColumnString* col_from_string = check_and_get_column<ColumnString>(col_from);
 
         if (std::is_same_v<FromDataType, DataTypeString> && !col_from_string) {
-            return Status::RuntimeError(
-                    fmt::format("Illegal column {} of first argument of function {}",
-                                col_from->get_name(), Name::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        col_from->get_name(), Name::name);
         }
 
         size_t size = input_rows_count;
@@ -895,11 +890,11 @@ public:
                     block, arguments, result, input_rows_count);
         }
 
-        return Status::RuntimeError(fmt::format(
+        return Status::RuntimeError(
                 "Illegal type {} of argument of function {} . Only String or FixedString "
                 "argument is accepted for try-conversion function. For other arguments, use "
                 "function without 'orZero' or 'orNull'.",
-                block.get_by_position(arguments[0]).type->get_name(), get_name()));
+                block.get_by_position(arguments[0]).type->get_name(), get_name());
     }
 };
 
@@ -939,9 +934,9 @@ public:
 
         bool done = call_on_index_and_number_data_type<ToDataType>(from_type->get_type_id(), call);
         if (!done) {
-            return Status::RuntimeError(
-                    fmt::format("Illegal type {} of argument of function {}",
-                                block.get_by_position(arguments[0]).type->get_name(), get_name()));
+            return Status::RuntimeError("Illegal type {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).type->get_name(),
+                                        get_name());
         }
 
         return ret_status;
@@ -1084,8 +1079,8 @@ private:
             /// Additionally check if call_on_index_and_data_type wasn't called at all.
             if (!res) {
                 auto to = DataTypeDecimal<FieldType>(precision, scale);
-                return Status::RuntimeError(fmt::format("Conversion from {} to {} is not supported",
-                                                        getTypeName(type_index), to.get_name()));
+                return Status::RuntimeError("Conversion from {} to {} is not supported",
+                                            getTypeName(type_index), to.get_name());
             }
             return Status::OK();
         };
@@ -1167,8 +1162,8 @@ private:
                 block.get_by_position(result).column = ColumnArray::create(
                         nested_result_column, from_col_array->get_offsets_ptr());
             } else {
-                return Status::RuntimeError(fmt::format(
-                        "Illegal column {} for function CAST AS Array", from_column->get_name()));
+                return Status::RuntimeError("Illegal column {} for function CAST AS Array",
+                                            from_column->get_name());
             }
             return Status::OK();
         };
@@ -1236,10 +1231,10 @@ private:
 
                 /// May happen in fuzzy tests. For debug purpose.
                 if (!tmp_res.column.get()) {
-                    return Status::RuntimeError(fmt::format(
+                    return Status::RuntimeError(
                             "Couldn't convert {} to {} in prepare_remove_nullable wrapper.",
                             block.get_by_position(arguments[0]).type->get_name(),
-                            nested_type->get_name()));
+                            nested_type->get_name());
                 }
 
                 res.column = wrap_in_nullable(tmp_res.column,
@@ -1265,7 +1260,7 @@ private:
 
                     if (!memory_is_zero(null_map.data(), null_map.size())) {
                         return Status::RuntimeError(
-                                fmt::format("Cannot convert NULL value to non-Nullable type"));
+                                "Cannot convert NULL value to non-Nullable type");
                     }
                 }
 
diff --git a/be/src/vec/functions/function_coalesce.cpp b/be/src/vec/functions/function_coalesce.cpp
index 8ed7bf51f7..e90e054fd1 100644
--- a/be/src/vec/functions/function_coalesce.cpp
+++ b/be/src/vec/functions/function_coalesce.cpp
@@ -227,8 +227,7 @@ public:
         DECIMAL_TYPE_TO_COLUMN_TYPE(DISPATCH)
         TIME_TYPE_TO_COLUMN_TYPE(DISPATCH)
 #undef DISPATCH
-        return Status::NotSupported(
-                fmt::format("argument_type {} not supported", data_type->get_name()));
+        return Status::NotSupported("argument_type {} not supported", data_type->get_name());
     }
 };
 
diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h
index f748d5ad6d..69be1e6bae 100644
--- a/be/src/vec/functions/function_date_or_datetime_computation.h
+++ b/be/src/vec/functions/function_date_or_datetime_computation.h
@@ -371,9 +371,9 @@ struct DateTimeAddIntervalImpl {
             block.get_by_position(result).column =
                     ColumnNullable::create(std::move(col_to), std::move(null_map));
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of first argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), Transform::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        Transform::name);
         }
         return Status::OK();
     }
@@ -484,9 +484,9 @@ public:
             return DateTimeAddIntervalImpl<DataTypeDateV2::FieldType, Transform>::execute(
                     block, arguments, result);
         } else {
-            return Status::RuntimeError(
-                    fmt::format("Illegal type {} of argument of function {}",
-                                block.get_by_position(arguments[0]).type->get_name(), get_name()));
+            return Status::RuntimeError("Illegal type {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).type->get_name(),
+                                        get_name());
         }
     }
 };
diff --git a/be/src/vec/functions/function_date_or_datetime_to_string.h b/be/src/vec/functions/function_date_or_datetime_to_string.h
index 8b06869489..b4f2b80175 100644
--- a/be/src/vec/functions/function_date_or_datetime_to_string.h
+++ b/be/src/vec/functions/function_date_or_datetime_to_string.h
@@ -69,9 +69,9 @@ public:
             block.replace_by_position(
                     result, ColumnNullable::create(std::move(col_res), std::move(null_map)));
         } else {
-            return Status::InternalError("Illegal column " +
-                                         block.get_by_position(arguments[0]).column->get_name() +
-                                         " of first argument of function " + name);
+            return Status::InternalError("Illegal column {} of first argument of function {}",
+                                         block.get_by_position(arguments[0]).column->get_name(),
+                                         name);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_datetime_floor_ceil.cpp b/be/src/vec/functions/function_datetime_floor_ceil.cpp
index 42e331b9f0..9003b932df 100644
--- a/be/src/vec/functions/function_datetime_floor_ceil.cpp
+++ b/be/src/vec/functions/function_datetime_floor_ceil.cpp
@@ -101,9 +101,9 @@ public:
             block.get_by_position(result).column =
                     ColumnNullable::create(std::move(col_to), std::move(null_map));
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of first argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), Impl::name));
+            return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        Impl::name);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_datetime_string_to_string.h b/be/src/vec/functions/function_datetime_string_to_string.h
index 0cb2d9df96..ca33ebd854 100644
--- a/be/src/vec/functions/function_datetime_string_to_string.h
+++ b/be/src/vec/functions/function_datetime_string_to_string.h
@@ -78,9 +78,8 @@ public:
                             col_res->get_chars(), col_res->get_offsets(), vec_null_map_to);
                 } else {
                     return Status::InternalError(
-                            "Illegal column " +
-                            block.get_by_position(arguments[1]).column->get_name() +
-                            " is not const" + name);
+                            "Illegal column {} is not const {}",
+                            block.get_by_position(arguments[1]).column->get_name(), name);
                 }
             } else {
                 TransformerToStringTwoArgument<Transform>::vector_constant(
@@ -97,9 +96,9 @@ public:
             block.get_by_position(result).column =
                     ColumnNullable::create(std::move(col_res), std::move(col_null_map_to));
         } else {
-            return Status::InternalError("Illegal column " +
-                                         block.get_by_position(arguments[0]).column->get_name() +
-                                         " of first argument of function " + name);
+            return Status::InternalError("Illegal column {} of first argument of function {}",
+                                         block.get_by_position(arguments[0]).column->get_name(),
+                                         name);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_fake.h b/be/src/vec/functions/function_fake.h
index 2dbf4f263f..475850671e 100644
--- a/be/src/vec/functions/function_fake.h
+++ b/be/src/vec/functions/function_fake.h
@@ -50,7 +50,7 @@ public:
 
     Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments,
                         size_t result, size_t input_rows_count) override {
-        return Status::NotSupported(fmt::format("Fake function {} do not support execute", name));
+        return Status::NotSupported("Fake function {} do not support execute", name);
     }
 };
 
diff --git a/be/src/vec/functions/function_hash.cpp b/be/src/vec/functions/function_hash.cpp
index 4a6235c9a6..9f3101b3a3 100644
--- a/be/src/vec/functions/function_hash.cpp
+++ b/be/src/vec/functions/function_hash.cpp
@@ -84,8 +84,8 @@ struct MurmurHash2Impl64 {
             }
         } else {
             DCHECK(false);
-            return Status::NotSupported(fmt::format("Illegal column {} of argument of function {}",
-                                                    column->get_name(), name));
+            return Status::NotSupported("Illegal column {} of argument of function {}",
+                                        column->get_name(), name);
         }
         return Status::OK();
     }
@@ -130,8 +130,8 @@ struct MurmurHash2Impl64 {
             }
         } else {
             DCHECK(false);
-            return Status::NotSupported(fmt::format("Illegal column {} of argument of function {}",
-                                                    column->get_name(), name));
+            return Status::NotSupported("Illegal column {} of argument of function {}",
+                                        column->get_name(), name);
         }
         return Status::OK();
     }
@@ -149,8 +149,7 @@ struct MurmurHash2Impl64 {
         return execute_int_type<TYPE, first>(icolumn, col_to, input_rows_count);
         NUMERIC_TYPE_TO_COLUMN_TYPE(DISPATCH)
 #undef DISPATCH
-        return Status::NotSupported(
-                fmt::format("argument_type {} not supported", from_type->get_name()));
+        return Status::NotSupported("argument_type {} not supported", from_type->get_name());
     }
 };
 using FunctionMurmurHash2_64 = FunctionVariadicArgumentsBase<DataTypeUInt64, MurmurHash2Impl64>;
@@ -215,8 +214,8 @@ struct MurmurHash3Impl32 {
             }
         } else {
             DCHECK(false);
-            return Status::NotSupported(fmt::format("Illegal column {} of argument of function {}",
-                                                    column->get_name(), name));
+            return Status::NotSupported("Illegal column {} of argument of function {}",
+                                        column->get_name(), name);
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_math_unary.h b/be/src/vec/functions/function_math_unary.h
index bbcb011bd8..455d1f1e02 100644
--- a/be/src/vec/functions/function_math_unary.h
+++ b/be/src/vec/functions/function_math_unary.h
@@ -142,8 +142,8 @@ private:
         };
 
         if (!call_on_basic_type<void, true, true, true, false>(col.type->get_type_id(), call)) {
-            return Status::InvalidArgument("Illegal column " + col.column->get_name() +
-                                           " of argument of function " + get_name());
+            return Status::InvalidArgument("Illegal column {} of argument of function {}",
+                                           col.column->get_name(), get_name());
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_math_unary_to_null_type.h b/be/src/vec/functions/function_math_unary_to_null_type.h
index 68930a78b7..80386b35a9 100644
--- a/be/src/vec/functions/function_math_unary_to_null_type.h
+++ b/be/src/vec/functions/function_math_unary_to_null_type.h
@@ -115,8 +115,8 @@ private:
         };
 
         if (!call_on_basic_type<void, true, true, true, false>(col.type->get_type_id(), call)) {
-            return Status::InvalidArgument("Illegal column " + col.column->get_name() +
-                                           " of argument of function " + get_name());
+            return Status::InvalidArgument("Illegal column {} of argument of function {}",
+                                           col.column->get_name(), get_name());
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h
index 3c29cb8745..4d4fdc566f 100644
--- a/be/src/vec/functions/function_string.h
+++ b/be/src/vec/functions/function_string.h
@@ -625,7 +625,7 @@ public:
             }
         }
 
-        return Status::RuntimeError(fmt::format("not support {}", get_name()));
+        return Status::RuntimeError("not support {}", get_name());
     }
 
     void vector_vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets,
@@ -964,9 +964,9 @@ public:
                 offsets_list[i] = &col_str->get_offsets();
                 chars_list[i] = &col_str->get_chars();
             } else {
-                return Status::RuntimeError(fmt::format(
-                        "Illegal column {} of argument of function {}",
-                        block.get_by_position(arguments[0]).column->get_name(), get_name()));
+                return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                            block.get_by_position(arguments[0]).column->get_name(),
+                                            get_name());
             }
         }
 
@@ -1071,10 +1071,10 @@ public:
             if (!success) {
                 // url is malformed, or url_part is invalid.
                 if (url_part == UrlParser::INVALID) {
-                    return Status::RuntimeError(fmt::format(
+                    return Status::RuntimeError(
                             "Invalid URL part: {}\n{}", std::string(part.data, part.size),
                             "(Valid URL parts are 'PROTOCOL', 'HOST', 'PATH', 'REF', 'AUTHORITY', "
-                            "'FILE', 'USERINFO', 'PORT' and 'QUERY')"));
+                            "'FILE', 'USERINFO', 'PORT' and 'QUERY')");
                 } else {
                     StringOP::push_null_string(i, res_chars, res_offsets, null_map_data);
                     continue;
diff --git a/be/src/vec/functions/function_string_to_string.h b/be/src/vec/functions/function_string_to_string.h
index 78e81f73d2..15435a5192 100644
--- a/be/src/vec/functions/function_string_to_string.h
+++ b/be/src/vec/functions/function_string_to_string.h
@@ -69,9 +69,9 @@ public:
                          col_res->get_offsets());
             block.replace_by_position(result, std::move(col_res));
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), get_name()));
+            return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        get_name());
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_totype.h b/be/src/vec/functions/function_totype.h
index 27a96b03ad..033e168a8e 100644
--- a/be/src/vec/functions/function_totype.h
+++ b/be/src/vec/functions/function_totype.h
@@ -86,9 +86,9 @@ private:
             }
         }
 
-        return Status::RuntimeError(
-                fmt::format("Illegal column {} of argument of function {}",
-                            block.get_by_position(arguments[0]).column->get_name(), get_name()));
+        return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                    block.get_by_position(arguments[0]).column->get_name(),
+                                    get_name());
     }
     template <typename T, std::enable_if_t<!std::is_same_v<T, DataTypeString>, T>* = nullptr>
     Status execute_impl(Block& block, const ColumnNumbers& arguments, size_t result,
@@ -119,9 +119,9 @@ private:
                 return Status::OK();
             }
         }
-        return Status::RuntimeError(
-                fmt::format("Illegal column {} of argument of function {}",
-                            block.get_by_position(arguments[0]).column->get_name(), get_name()));
+        return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                    block.get_by_position(arguments[0]).column->get_name(),
+                                    get_name());
     }
 };
 
@@ -177,7 +177,7 @@ public:
                 return Status::OK();
             }
         }
-        return Status::RuntimeError(fmt::format("unimplements function {}", get_name()));
+        return Status::RuntimeError("unimplements function {}", get_name());
     }
 };
 
@@ -240,7 +240,7 @@ private:
                 return Status::OK();
             }
         }
-        return Status::RuntimeError(fmt::format("unimplements function {}", get_name()));
+        return Status::RuntimeError("unimplements function {}", get_name());
     }
 
     template <typename ReturnDataType,
@@ -262,7 +262,7 @@ private:
                 return Status::OK();
             }
         }
-        return Status::RuntimeError(fmt::format("unimplements function {}", get_name()));
+        return Status::RuntimeError("unimplements function {}", get_name());
     }
 };
 
@@ -333,7 +333,7 @@ public:
                 return Status::OK();
             }
         }
-        return Status::RuntimeError(fmt::format("unimplements function {}", get_name()));
+        return Status::RuntimeError("unimplements function {}", get_name());
     }
 };
 
@@ -434,9 +434,9 @@ public:
             block.replace_by_position(
                     result, ColumnNullable::create(std::move(col_res), std::move(null_map)));
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), get_name()));
+            return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        get_name());
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/function_unary_arithmetic.h b/be/src/vec/functions/function_unary_arithmetic.h
index 7bbee9afb7..3c1f3739ac 100644
--- a/be/src/vec/functions/function_unary_arithmetic.h
+++ b/be/src/vec/functions/function_unary_arithmetic.h
@@ -141,8 +141,8 @@ public:
                     return false;
                 });
         if (!valid) {
-            return Status::RuntimeError(
-                    fmt::format("{}'s argument does not match the expected data type", get_name()));
+            return Status::RuntimeError("{}'s argument does not match the expected data type",
+                                        get_name());
         }
         return Status::OK();
     }
diff --git a/be/src/vec/functions/functions_comparison.h b/be/src/vec/functions/functions_comparison.h
index f481438537..8ef6fd3e7d 100644
--- a/be/src/vec/functions/functions_comparison.h
+++ b/be/src/vec/functions/functions_comparison.h
@@ -285,9 +285,8 @@ private:
         };
 
         if (!call_on_basic_types<true, false, true, false>(left_number, right_number, call)) {
-            return Status::RuntimeError(fmt::format("Wrong call for {} with {} and {}", get_name(),
-                                                    col_left.type->get_name(),
-                                                    col_right.type->get_name()));
+            return Status::RuntimeError("Wrong call for {} with {} and {}", get_name(),
+                                        col_left.type->get_name(), col_right.type->get_name());
         }
         return Status::OK();
     }
@@ -402,14 +401,12 @@ public:
                   execute_num_left_type<Float64>(block, result, col_left_untyped,
                                                  col_right_untyped)))
 
-                return Status::RuntimeError(
-                        fmt::format("Illegal column {} of first argument of function {}",
-                                    col_left_untyped->get_name(), get_name()));
+                return Status::RuntimeError("Illegal column {} of first argument of function {}",
+                                            col_left_untyped->get_name(), get_name());
         } else if (is_decimal(left_type) || is_decimal(right_type)) {
             if (!allow_decimal_comparison(left_type, right_type)) {
-                return Status::RuntimeError(fmt::format("No operation {} between {} and {}",
-                                                        get_name(), left_type->get_name(),
-                                                        right_type->get_name()));
+                return Status::RuntimeError("No operation {} between {} and {}", get_name(),
+                                            left_type->get_name(), right_type->get_name());
             }
             return execute_decimal(block, result, col_with_type_and_name_left,
                                    col_with_type_and_name_right);
diff --git a/be/src/vec/functions/hll_cardinality.cpp b/be/src/vec/functions/hll_cardinality.cpp
index aea5cbe668..4c192ca5e1 100644
--- a/be/src/vec/functions/hll_cardinality.cpp
+++ b/be/src/vec/functions/hll_cardinality.cpp
@@ -96,9 +96,9 @@ public:
             block.replace_by_position(result, std::move(column_result));
             return Status::OK();
         } else {
-            return Status::RuntimeError(fmt::format(
-                    "Illegal column {} of argument of function {}",
-                    block.get_by_position(arguments[0]).column->get_name(), get_name()));
+            return Status::RuntimeError("Illegal column {} of argument of function {}",
+                                        block.get_by_position(arguments[0]).column->get_name(),
+                                        get_name());
         }
 
         block.replace_by_position(result, std::move(column_result));
diff --git a/be/src/vec/functions/if.cpp b/be/src/vec/functions/if.cpp
index d132cfe3e6..cf9a56ddcc 100644
--- a/be/src/vec/functions/if.cpp
+++ b/be/src/vec/functions/if.cpp
@@ -290,9 +290,10 @@ public:
                             make_nullable_column_if_not(arg_else.column);
                 }
             } else {
-                status = Status::InternalError("Illegal column " + arg_cond.column->get_name() +
-                                               " of first argument of function " + get_name() +
-                                               ". Must be ColumnUInt8 or ColumnConstUInt8.");
+                status = Status::InternalError(
+                        "Illegal column {} of first argument of function {}. Must be ColumnUInt8 "
+                        "or ColumnConstUInt8.",
+                        arg_cond.column->get_name(), get_name());
             }
             return true;
         }
@@ -334,9 +335,10 @@ public:
                                     input_rows_count);
                 }
             } else {
-                status = Status::InternalError("Illegal column " + arg_cond.column->get_name() +
-                                               " of first argument of function " + get_name() +
-                                               ". Must be ColumnUInt8 or ColumnConstUInt8.");
+                status = Status::InternalError(
+                        "Illegal column {} of first argument of function {}. Must be ColumnUInt8 "
+                        "or ColumnConstUInt8.",
+                        arg_cond.column->get_name(), get_name());
             }
             return true;
         }
@@ -457,9 +459,10 @@ public:
         }
 
         if (!cond_col) {
-            return Status::InvalidArgument("Illegal column " + arg_cond.column->get_name() +
-                                           " of first argument of function " + get_name() +
-                                           ",Must be ColumnUInt8 or ColumnConstUInt8.");
+            return Status::InvalidArgument(
+                    "Illegal column {} of first argument of function {},Must be ColumnUInt8 or "
+                    "ColumnConstUInt8.",
+                    arg_cond.column->get_name(), get_name());
         }
 
         WhichDataType which_type(arg_then.type);
diff --git a/be/src/vec/functions/in.cpp b/be/src/vec/functions/in.cpp
index 301d9b4e2f..942a0d8062 100644
--- a/be/src/vec/functions/in.cpp
+++ b/be/src/vec/functions/in.cpp
@@ -92,8 +92,8 @@ public:
         auto in_state = reinterpret_cast<InState*>(
                 context->get_function_state(FunctionContext::FRAGMENT_LOCAL));
         if (!in_state) {
-            return Status::RuntimeError(
-                    fmt::format("funciton context for function '{}' must have Set;", get_name()));
+            return Status::RuntimeError("funciton context for function '{}' must have Set;",
+                                        get_name());
         }
         auto res = ColumnUInt8::create();
         ColumnUInt8::Container& vec_res = res->get_data();
diff --git a/be/src/vec/functions/like.cpp b/be/src/vec/functions/like.cpp
index e11e27c5e7..6a04e654ab 100644
--- a/be/src/vec/functions/like.cpp
+++ b/be/src/vec/functions/like.cpp
@@ -155,7 +155,7 @@ Status FunctionLike::like_fn(LikeSearchState* state, const StringValue& val,
         *result = RE2::FullMatch(re2::StringPiece(val.ptr, val.len), re);
         return Status::OK();
     } else {
-        return Status::RuntimeError(fmt::format("Invalid pattern: {}", pattern.debug_string()));
+        return Status::RuntimeError("Invalid pattern: {}", pattern.debug_string());
     }
 }
 
@@ -247,8 +247,7 @@ Status FunctionLike::prepare(FunctionContext* context, FunctionContext::Function
             opts.set_dot_nl(true);
             state->search_state.regex = std::make_unique<RE2>(re_pattern, opts);
             if (!state->search_state.regex->ok()) {
-                return Status::InternalError(
-                        fmt::format("Invalid regex expression: {}", pattern_str));
+                return Status::InternalError("Invalid regex expression: {}", pattern_str);
             }
             state->function = constant_regex_full_fn;
         }
@@ -288,8 +287,7 @@ Status FunctionRegexp::prepare(FunctionContext* context,
             opts.set_dot_nl(true);
             state->search_state.regex = std::make_unique<RE2>(pattern_str, opts);
             if (!state->search_state.regex->ok()) {
-                return Status::InternalError(
-                        fmt::format("Invalid regex expression: {}", pattern_str));
+                return Status::InternalError("Invalid regex expression: {}", pattern_str);
             }
             state->function = constant_regex_partial_fn;
         }
@@ -315,7 +313,7 @@ Status FunctionRegexp::regexp_fn(LikeSearchState* state, const StringValue& val,
         *result = RE2::PartialMatch(re2::StringPiece(val.ptr, val.len), re);
         return Status::OK();
     } else {
-        return Status::RuntimeError(fmt::format("Invalid pattern: {}", pattern.debug_string()));
+        return Status::RuntimeError("Invalid pattern: {}", pattern.debug_string());
     }
 }
 
diff --git a/be/src/vec/runtime/vfile_result_writer.cpp b/be/src/vec/runtime/vfile_result_writer.cpp
index 912275c974..40f8472fef 100644
--- a/be/src/vec/runtime/vfile_result_writer.cpp
+++ b/be/src/vec/runtime/vfile_result_writer.cpp
@@ -120,8 +120,7 @@ Status VFileResultWriter::_create_file_writer(const std::string& file_name) {
         return Status::NotSupported("Parquet Writer is not supported yet!");
         break;
     default:
-        return Status::InternalError(
-                strings::Substitute("unsupported file format: $0", _file_opts->file_format));
+        return Status::InternalError("unsupported file format: {}", _file_opts->file_format);
     }
     LOG(INFO) << "create file for exporting query result. file name: " << file_name
               << ". query id: " << print_id(_state->query_id())
@@ -466,9 +465,9 @@ Status VFileResultWriter::_fill_result_block() {
             break;
         }
         default:
-            return Status::InternalError(strings::Substitute(
-                    "Invalid type to print: $0",
-                    _output_row_descriptor.tuple_descriptors()[0]->slots()[i]->type().type));
+            return Status::InternalError(
+                    "Invalid type to print: {}",
+                    _output_row_descriptor.tuple_descriptors()[0]->slots()[i]->type().type);
         }
     }
     return Status::OK();
diff --git a/be/src/vec/sink/vtablet_sink.cpp b/be/src/vec/sink/vtablet_sink.cpp
index ef334df09e..99fc7213ac 100644
--- a/be/src/vec/sink/vtablet_sink.cpp
+++ b/be/src/vec/sink/vtablet_sink.cpp
@@ -146,7 +146,7 @@ Status VNodeChannel::add_row(const BlockRow& block_row, int64_t tablet_id) {
     if (!st.ok()) {
         if (_cancelled) {
             std::lock_guard<SpinLock> l(_cancel_msg_lock);
-            return Status::InternalError("add row failed. " + _cancel_msg);
+            return Status::InternalError("add row failed. {}", _cancel_msg);
         } else {
             return st.clone_and_prepend("already stopped, can't add row. cancelled/eos: ");
         }


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